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

jshao pushed a commit to branch branch-0.9
in repository https://gitbox.apache.org/repos/asf/gravitino.git


The following commit(s) were added to refs/heads/branch-0.9 by this push:
     new 52e9f6e002 [#6936] improvement(doc): update doc for fileset multiple 
locations (#7036)
52e9f6e002 is described below

commit 52e9f6e002c0199eba073c3ff3a9ca7832984d6c
Author: github-actions[bot] 
<41898282+github-actions[bot]@users.noreply.github.com>
AuthorDate: Tue Apr 22 16:35:13 2025 +0800

    [#6936] improvement(doc): update doc for fileset multiple locations (#7036)
    
    ### What changes were proposed in this pull request?
    
     update doc for fileset multiple locations
    
    ### Why are the changes needed?
    
    Fix: #6936
    
    ### Does this PR introduce _any_ user-facing change?
    
    no
    
    ### How was this patch tested?
    
    no need
    
    Co-authored-by: mchades <[email protected]>
    Co-authored-by: Qiming Teng <[email protected]>
---
 docs/hadoop-catalog.md                          |  44 ++--
 docs/how-to-use-gvfs.md                         |  53 ++++-
 docs/manage-fileset-metadata-using-gravitino.md | 257 +++++++++++++++++++++---
 3 files changed, 301 insertions(+), 53 deletions(-)

diff --git a/docs/hadoop-catalog.md b/docs/hadoop-catalog.md
index cea25e6490..cf57367950 100644
--- a/docs/hadoop-catalog.md
+++ b/docs/hadoop-catalog.md
@@ -25,7 +25,8 @@ Besides the [common catalog 
properties](./gravitino-server-config.md#apache-grav
 
 | Property Name                        | Description                           
                                                                                
                                                                                
                                                                                
                             | Default Value   | Required | Since Version    |
 
|--------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-----------------|----------|------------------|
-| `location`                           | The storage location managed by 
Hadoop catalog.                                                                 
                                                                                
                                                                                
                                   | (none)          | No       | 0.5.0         
   |
+| `location`                           | The storage location managed by 
Hadoop catalog. Its location name is `unknown`.                                 
                                                                                
                                                                                
                                   | (none)          | No       | 0.5.0         
   |
+| `location-`                          | The property prefix. User can use 
`location-{name}={path}` to set multiple locations with different names for the 
catalog.                                                                        
                                                                                
                                 | (none)          | No       | 
0.9.0-incubating |
 | `default-filesystem-provider`        | The default filesystem provider of 
this Hadoop catalog if users do not specify the scheme in the URI. Candidate 
values are 'builtin-local', 'builtin-hdfs', 's3', 'gcs', 'abs' and 'oss'. 
Default value is `builtin-local`. For S3, if we set this value to 's3', we can 
omit the prefix 's3a://' in the location. | `builtin-local` | No       | 
0.7.0-incubating |
 | `filesystem-providers`               | The file system providers to add. 
Users need to set this configuration to support cloud storage or custom HCFS. 
For instance, set it to `s3` or a comma separated string that contains `s3` 
like `gs,s3` to support multiple kinds of fileset including `s3`.               
                                       | (none)          | Yes      | 
0.7.0-incubating |
 | `credential-providers`               | The credential provider types, 
separated by comma.                                                             
                                                                                
                                                                                
                                    | (none)          | No       | 
0.8.0-incubating |
@@ -104,19 +105,27 @@ The Hadoop catalog supports creating, updating, deleting, 
and listing schema.
 
 ### Schema properties
 
-| Property name                         | Description                          
                                                                          | 
Default value             | Required | Since Version    |
-|---------------------------------------|----------------------------------------------------------------------------------------------------------------|---------------------------|----------|------------------|
-| `location`                            | The storage location managed by 
Hadoop schema.                                                                 
| (none)                    | No       | 0.5.0            |
-| `authentication.impersonation-enable` | Whether to enable impersonation for 
this schema of the Hadoop catalog.                                         | 
The parent(catalog) value | No       | 0.6.0-incubating |
-| `authentication.type`                 | The type of authentication for this 
schema of Hadoop catalog , currently we only support `kerberos`, `simple`. | 
The parent(catalog) value | No       | 0.6.0-incubating |
-| `authentication.kerberos.principal`   | The principal of the Kerberos 
authentication for this schema.                                                 
 | The parent(catalog) value | No       | 0.6.0-incubating |
-| `authentication.kerberos.keytab-uri`  | The URI of The keytab for the 
Kerberos authentication for this schema.                                        
 | The parent(catalog) value | No       | 0.6.0-incubating |
-| `credential-providers`                | The credential provider types, 
separated by comma.                                                             
| (none)                    | No       | 0.8.0-incubating |
+| Property name                         | Description                          
                                                                                
     | Default value             | Required | Since Version    |
+|---------------------------------------|---------------------------------------------------------------------------------------------------------------------------|---------------------------|----------|------------------|
+| `location`                            | The storage location managed by 
Hadoop schema. Its location name is `unknown`.                                  
          | (none)                    | No       | 0.5.0            |
+| `location-`                           | The property prefix. User can use 
`location-{name}={path}` to set multiple locations with different names for the 
schema. | (none)                    | No       | 0.9.0-incubating |
+| `authentication.impersonation-enable` | Whether to enable impersonation for 
this schema of the Hadoop catalog.                                              
      | The parent(catalog) value | No       | 0.6.0-incubating |
+| `authentication.type`                 | The type of authentication for this 
schema of Hadoop catalog , currently we only support `kerberos`, `simple`.      
      | The parent(catalog) value | No       | 0.6.0-incubating |
+| `authentication.kerberos.principal`   | The principal of the Kerberos 
authentication for this schema.                                                 
            | The parent(catalog) value | No       | 0.6.0-incubating |
+| `authentication.kerberos.keytab-uri`  | The URI of The keytab for the 
Kerberos authentication for this schema.                                        
            | The parent(catalog) value | No       | 0.6.0-incubating |
+| `credential-providers`                | The credential provider types, 
separated by comma.                                                             
           | (none)                    | No       | 0.8.0-incubating |
 
 ### Schema operations
 
 Refer to [Schema 
operation](./manage-fileset-metadata-using-gravitino.md#schema-operations) for 
more details.
 
+:::note
+During schema creation or deletion, Gravitino automatically creates or removes 
the corresponding filesystem directories for the schema locations. 
+This behavior is skipped in either of these cases:
+1. When the catalog property `disable-filesystem-ops` is set to `true`
+2. When the location contains 
[placeholders](./manage-fileset-metadata-using-gravitino.md#placeholder)
+:::
+
 ## Fileset
 
 ### Fileset capabilities
@@ -125,14 +134,15 @@ Refer to [Schema 
operation](./manage-fileset-metadata-using-gravitino.md#schema-
 
 ### Fileset properties
 
-| Property name                         | Description                          
                                                                  | Default 
value            | Required | Immutable | Since Version    |
-|---------------------------------------|--------------------------------------------------------------------------------------------------------|--------------------------|----------|-----------|------------------|
-| `authentication.impersonation-enable` | Whether to enable impersonation for 
the Hadoop catalog fileset.                                        | The 
parent(schema) value | No       | Yes       | 0.6.0-incubating |
-| `authentication.type`                 | The type of authentication for 
Hadoop catalog fileset, currently we only support `kerberos`, `simple`. | The 
parent(schema) value | No       | No        | 0.6.0-incubating |
-| `authentication.kerberos.principal`   | The principal of the Kerberos 
authentication for the fileset.                                          | The 
parent(schema) value | No       | No        | 0.6.0-incubating |
-| `authentication.kerberos.keytab-uri`  | The URI of The keytab for the 
Kerberos authentication for the fileset.                                 | The 
parent(schema) value | No       | No        | 0.6.0-incubating |
-| `credential-providers`                | The credential provider types, 
separated by comma.                                                     | 
(none)                   | No       | No        | 0.8.0-incubating |
-| `placeholder-`                        | Properties that start with 
`placeholder-` are used to replace placeholders in the location.            | 
(none)                   | No       | Yes       | 0.9.0-incubating |
+| Property name                         | Description                          
                                                                                
| Default value                                                                 
                                 | Required                                   | 
Immutable | Since Version    |
+|---------------------------------------|----------------------------------------------------------------------------------------------------------------------|----------------------------------------------------------------------------------------------------------------|--------------------------------------------|-----------|------------------|
+| `authentication.impersonation-enable` | Whether to enable impersonation for 
the Hadoop catalog fileset.                                                     
 | The parent(schema) value                                                     
                                  | No                                         
| Yes       | 0.6.0-incubating |
+| `authentication.type`                 | The type of authentication for 
Hadoop catalog fileset, currently we only support `kerberos`, `simple`.         
      | The parent(schema) value                                                
                                       | No                                     
    | No        | 0.6.0-incubating |
+| `authentication.kerberos.principal`   | The principal of the Kerberos 
authentication for the fileset.                                                 
       | The parent(schema) value                                               
                                        | No                                    
     | No        | 0.6.0-incubating |
+| `authentication.kerberos.keytab-uri`  | The URI of The keytab for the 
Kerberos authentication for the fileset.                                        
       | The parent(schema) value                                               
                                        | No                                    
     | No        | 0.6.0-incubating |
+| `credential-providers`                | The credential provider types, 
separated by comma.                                                             
      | (none)                                                                  
                                       | No                                     
    | No        | 0.8.0-incubating |
+| `placeholder-`                        | Properties that start with 
`placeholder-` are used to replace placeholders in the location.                
          | (none)                                                              
                                           | No                                 
        | Yes       | 0.9.0-incubating |
+| `default-location-name`               | The name of the default location of 
the fileset, mainly used for GVFS operations without specifying a location 
name. | When the fileset has only one location, its location name will be 
automatically selected as the default value. | Yes, if the fileset has multiple 
locations | Yes       | 0.9.0-incubating |
 
 Some properties are reserved and cannot be set by users:
 
diff --git a/docs/how-to-use-gvfs.md b/docs/how-to-use-gvfs.md
index 6b3e4a0a04..996041462b 100644
--- a/docs/how-to-use-gvfs.md
+++ b/docs/how-to-use-gvfs.md
@@ -138,6 +138,13 @@ two ways:
        ./gradlew :clients:filesystem-hadoop3-runtime:build -x test
     ```
 
+:::note
+For fileset with multiple locations, you can specify which location to access 
using one of these methods (in priority order):
+1. Set the `fs.gravitino.current.location.name` configuration property
+2. Export the environment variable `CURRENT_LOCATION_NAME`
+3. If neither is specified, the system will use the value of 
`default-location-name` from the fileset properties
+:::
+
 #### Via Hadoop shell command
 
 You can use the Hadoop shell command to perform operations on the fileset 
storage. For example:
@@ -145,6 +152,9 @@ You can use the Hadoop shell command to perform operations 
on the fileset storag
 ```shell
 # 1. Configure the hadoop `core-site.xml` configuration
 # You should put the required properties into this file
+
+# set the location name if you want to access a specific location
+# export CURRENT_LOCATION_NAME=${the_fileset_location_name}
 vi ${HADOOP_HOME}/etc/hadoop/core-site.xml
 
 # 2. Place the GVFS runtime jar into your Hadoop environment
@@ -172,6 +182,8 @@ 
conf.set("fs.AbstractFileSystem.gvfs.impl","org.apache.gravitino.filesystem.hado
 
conf.set("fs.gvfs.impl","org.apache.gravitino.filesystem.hadoop.GravitinoVirtualFileSystem");
 conf.set("fs.gravitino.server.uri","http://localhost:8090";);
 conf.set("fs.gravitino.client.metalake","test_metalake");
+// set the location name if you want to access a specific location
+// conf.set("fs.gravitino.current.location.name","test_location_name");
 Path filesetPath = new 
Path("gvfs://fileset/test_catalog/test_schema/test_fileset_1");
 FileSystem fs = filesetPath.getFileSystem(conf);
 fs.getFileStatus(filesetPath);
@@ -199,6 +211,8 @@ fs.getFileStatus(filesetPath);
     --conf 
spark.hadoop.fs.gvfs.impl=org.apache.gravitino.filesystem.hadoop.GravitinoVirtualFileSystem
     --conf spark.hadoop.fs.gravitino.server.uri=${your_gravitino_server_uri}
     --conf spark.hadoop.fs.gravitino.client.metalake=${your_gravitino_metalake}
+    # set the location name if you want to access a specific location
+    # --conf 
spark.hadoop.fs.gravitino.current.location.name=${the_fileset_location_name}
     ```
 
 3. Perform operations on the fileset storage in your code.
@@ -236,6 +250,8 @@ For Tensorflow to support GVFS, you need to recompile the 
[tensorflow-io](https:
    ```shell
    export HADOOP_HOME=${your_hadoop_home}
    export HADOOP_CONF_DIR=${your_hadoop_conf_home}
+   # set the location name if you want to access a specific location
+   # export CURRENT_LOCATION_NAME=${the_fileset_location_name} 
    export PATH=$PATH:$HADOOP_HOME/libexec/hadoop-config.sh
    export LD_LIBRARY_PATH=$LD_LIBRARY_PATH:$JAVA_HOME/jre/lib/amd64/server
    export PATH=$PATH:$HADOOP_HOME/bin:$HADOOP_HOME/sbin
@@ -378,6 +394,13 @@ Gravitino python client does not support [customized file 
systems](hadoop-catalo
 
 ### Usage examples
 
+:::note
+For fileset with multiple locations, you can specify which location to access 
using one of these methods (in priority order):
+1. Set the `current_location_name` configuration property
+2. Export the environment variable `CURRENT_LOCATION_NAME`
+3. If neither is specified, the system will use the value of 
`default-location-name` from the fileset properties
+:::
+
 1. Make sure to obtain the Gravitino library.
    You can get it by [pip](https://pip.pypa.io/en/stable/installation/):
 
@@ -407,6 +430,13 @@ Gravitino python client does not support [customized file 
systems](hadoop-catalo
       <name>hadoop.client.keytab.file</name>
       <value>/tmp/xxx.keytab</value>
     </property>
+   
+   <!-- Optional, if you want to access a specific location -->
+   <property>
+      <name>fs.gravitino.current.location.name</name>
+      <value>location-name</value>
+   </property>
+  
     # Configure Hadoop env in Linux
     export HADOOP_HOME=${YOUR_HADOOP_PATH}
     export HADOOP_CONF_DIR=${YOUR_HADOOP_PATH}/etc/hadoop
@@ -423,7 +453,11 @@ For example:
 from gravitino import gvfs
 
 # init the gvfs
-fs = gvfs.GravitinoVirtualFileSystem(server_uri="http://localhost:8090";, 
metalake_name="test_metalake")
+fs = gvfs.GravitinoVirtualFileSystem(
+   server_uri="http://localhost:8090";,
+   metalake_name="test_metalake",
+   # set the location name if you want to access a specific location
+   options={"current_location_name": "the_location_name"})
 
 # list file infos under the fileset
 fs.ls(path="gvfs://fileset/fileset_catalog/tmp/tmp_fileset/sub_dir")
@@ -514,7 +548,10 @@ import pyarrow.dataset as dt
 import pyarrow.parquet as pq
 
 fs = gvfs.GravitinoVirtualFileSystem(
-    server_uri="http://localhost:8090";, metalake_name="test_metalake"
+    server_uri="http://localhost:8090";,
+    metalake_name="test_metalake",
+    # set the location name if you want to access a specific location
+    options={"current_location_name": "the_location_name"}
 )
 
 # read a parquet file as arrow dataset
@@ -531,7 +568,10 @@ from gravitino import gvfs
 import ray
 
 fs = gvfs.GravitinoVirtualFileSystem(
-    server_uri="http://localhost:8090";, metalake_name="test_metalake"
+    server_uri="http://localhost:8090";,
+    metalake_name="test_metalake",
+    # set the location name if you want to access a specific location
+    options={"current_location_name": "the_location_name"},
 )
 
 # read a parquet file as ray dataset
@@ -544,7 +584,12 @@ ds = 
ray.data.read_parquet("gvfs://fileset/fileset_catalog/tmp/tmp_fileset/test.
 from gravitino import gvfs
 from llama_index.core import SimpleDirectoryReader
 
-fs = gvfs.GravitinoVirtualFileSystem(server_uri=server_uri, 
metalake_name=metalake_name)
+fs = gvfs.GravitinoVirtualFileSystem(
+   server_uri=server_uri,
+   metalake_name=metalake_name,
+   # set the location name if you want to access a specific location
+   options={"current_location_name": "the_location_name"},
+)
 
 # read all document files like csv files under the fileset sub dir
 reader = SimpleDirectoryReader(
diff --git a/docs/manage-fileset-metadata-using-gravitino.md 
b/docs/manage-fileset-metadata-using-gravitino.md
index d4fde25a1d..12bb9a64c3 100644
--- a/docs/manage-fileset-metadata-using-gravitino.md
+++ b/docs/manage-fileset-metadata-using-gravitino.md
@@ -247,7 +247,7 @@ same.
 
 You can create a fileset by sending a `POST` request to the 
`/api/metalakes/{metalake_name}
 /catalogs/{catalog_name}/schemas/{schema_name}/filesets` endpoint or just use 
the Gravitino Java
-client. The following is an example of creating a fileset:
+client. The following is an example of creating a fileset with single storage 
location:
 
 <Tabs groupId="language" queryString>
 <TabItem value="shell" label="Shell">
@@ -315,16 +315,45 @@ Currently, Gravitino supports two **types** of filesets:
    specified as `EXTERNAL`, the files of the fileset will **not** be deleted 
when
    the fileset is dropped.
 
-**storageLocation**
+:::note
+During fileset creation or deletion, Gravitino automatically creates or 
removes the corresponding filesystem directories for the fileset locations.
+This behavior is skipped in either of these cases:
+1. When the catalog property `disable-filesystem-ops` is set to `true`
+2. When the location contains 
[placeholders](./manage-fileset-metadata-using-gravitino.md#placeholder)
+:::
+
+#### storageLocation
 
 The `storageLocation` is the physical location of the fileset. Users can 
specify this location
 when creating a fileset, or follow the rules of the catalog/schema location if 
not specified.
 
+For a `MANAGED` fileset, the storage location is determined in the following 
priority order:
+
+1. If the user specifies `storageLocation` during fileset creation:
+   - This location is used, with any [placeholders](#placeholder) replaced by 
the corresponding fileset property values.
+
+2. If the user doesn't specify `storageLocation`:
+   - If schema property `location` is specified:
+      - Use `<schema location>/<fileset name>` if schema location has no 
placeholders
+      - Use `<schema location>` with placeholders replaced by fileset property 
values
+
+   - Otherwise, if catalog property `location` is specified:
+      - Use `<catalog location>/<schema name>/<fileset name>` if catalog 
location has no placeholders
+      - Use `<catalog location>` with placeholders replaced by fileset 
property values
+
+   - If neither schema nor catalog location is specified:
+      - The user must provide `storageLocation` during fileset creation
+
+For an `EXTERNAL` fileset, the user must always specify `storageLocation` 
during fileset creation. 
+If the provided location contains placeholders, they will be replaced by the 
corresponding fileset property values.
+
+#### placeholder
+
 The `storageLocation` in each level can contain **placeholders**, format as 
`{{name}}`, which will
 be replaced by the corresponding fileset property value when the fileset 
object is created. The
 placeholder property in the fileset object is formed as 
"placeholder-{{name}}". For example, if
-the `storageLocation` is `file:///tmp/{{schema}}-{{fileset}}-{{verion}}`, and 
the fileset object 
-named "catalog1.schema1.fileset1" contains the properties 
`placeholder-version=v1`, 
+the `storageLocation` is `file:///tmp/{{schema}}-{{fileset}}-{{verion}}`, and 
the fileset object
+named "catalog1.schema1.fileset1" contains the properties 
`placeholder-version=v1`,
 the actual `storageLocation` will be `file:///tmp/schema1-fileset1-v1`.
 
 The following is an example of creating a fileset with placeholders in the 
`storageLocation`:
@@ -429,34 +458,198 @@ 
catalog.as_fileset_catalog().create_fileset(ident=NameIdentifier.of("test_schema
 </TabItem>
 </Tabs>
 
-The value of `storageLocation` depends on the configuration settings of the 
catalog:
-- If this is a local fileset catalog, the `storageLocation` should be in the 
format of `file:///path/to/fileset`.
-- If this is a HDFS fileset catalog, the `storageLocation` should be in the 
format of `hdfs://namenode:port/path/to/fileset`.
-
-For a `MANAGED` fileset, the storage location is:
-
-1. The one specified by the user during the fileset creation, and the 
placeholder will be replaced by the
-   corresponding fileset property value.
-2. When the catalog property `location` is specified but the schema property 
`location` isn't specified, the storage location is:
-   1. `catalog location/schema name/fileset name` if `catalog location` does 
not contain any placeholder. 
-   2. `catalog location` - placeholders in the catalog location will be 
replaced by the corresponding fileset property value.
-
-3. When the catalog property `location` isn't specified but the schema 
property `location` is specified,
-   the storage location is:
-   1. `schema location/fileset name` if `schema location` does not contain any 
placeholder.
-   2. `schema location` - placeholders in the schema location will be replaced 
by the corresponding fileset property value.
-   
-4. When both the catalog property `location` and the schema property 
`location` are specified, the storage
-   location is:
-   1. `schema location/fileset name` if `schema location` does not contain any 
placeholder.
-   2. `schema location` - placeholders in the schema location will be replaced 
by the corresponding fileset property value.
-
-5. When both the catalog property `location` and schema property `location` 
isn't specified, the user
-   should specify the `storageLocation` in the fileset creation.
-
-For `EXTERNAL` fileset, users should specify `storageLocation` during the 
fileset creation,
-otherwise, Gravitino will throw an exception. If the `storageLocation` 
contains placeholders, the
-placeholder will be replaced by the corresponding fileset property value.
+#### storageLocations
+You can also create a fileset with multiple storage locations. The 
`storageLocations` is a map of location name to storage location.
+The generation rules of each location follow the generation rules of a single 
location.
+The following is an example of creating a fileset with multiple storage 
locations:
+
+<Tabs groupId="language" queryString>
+<TabItem value="shell" label="Shell">
+
+```shell
+# create a catalog first
+curl -X POST -H "Accept: application/vnd.gravitino.v1+json" \
+-H "Content-Type: application/json" -d '{
+  "name": "test_catalog",
+  "type": "FILESET",
+  "comment": "comment",
+  "provider": "hadoop",
+  "properties": {
+    "filesystem-providers": "builtin-local,builtin-hdfs,s3,gcs",
+    "location-l1": 
"file:///{{catalog}}/{{schema}}/workspace_{{project}}/{{user}}",
+    "location-l2": 
"hdfs:///{{catalog}}/{{schema}}/workspace_{{project}}/{{user}}"
+  }
+}' http://localhost:8090/api/metalakes/metalake/catalogs
+
+# create a schema under the catalog
+curl -X POST -H "Accept: application/vnd.gravitino.v1+json" \
+-H "Content-Type: application/json" -d '{
+  "name": "test_schema",
+  "comment": "comment",
+  "properties": {
+    "location-l3": 
"s3a://myBucket/{{catalog}}/{{schema}}/workspace_{{project}}/{{user}}"
+  }
+}' http://localhost:8090/api/metalakes/metalake/catalogs/test_catalog/schemas
+
+# create a fileset by placeholders
+curl -X POST -H "Accept: application/vnd.gravitino.v1+json" \
+-H "Content-Type: application/json" -d '{
+  "name": "example_fileset",
+  "comment": "This is an example fileset",
+  "type": "MANAGED",
+  "storageLocations": {
+    "l4": "gs://myBucket/{{catalog}}/{{schema}}/workspace_{{project}}/{{user}}"
+  },
+  "properties": {
+    "placeholder-project": "test_project",
+    "placeholder-user": "test_user",
+    "default-location-name": "l1"
+  }
+}' 
http://localhost:8090/api/metalakes/metalake/catalogs/test_catalog/schemas/test_schema/filesets
+
+# the fileset will be created with 4 storage locations:
+{
+  "name": "example_fileset",
+  "comment": "This is an example fileset",
+  "type": "MANAGED",
+  "storageLocation": null,
+  "storageLocations": {
+    "l1": "file:///test_catalog/test_schema/workspace_test_project/test_user",
+    "l2": "hdfs:///test_catalog/test_schema/workspace_test_project/test_user",
+    "l3": 
"s3a://myBucket/test_catalog/test_schema/workspace_test_project/test_user",
+    "l4": 
"gs://myBucket/test_catalog/test_schema/workspace_test_project/test_user"
+  },
+  "properties": {
+    "placeholder-project": "test_project",
+    "placeholder-user": "test_user",
+    "default-location-name": "l1"
+  }
+}
+```
+
+</TabItem>
+<TabItem value="java" label="Java">
+
+```java
+GravitinoClient gravitinoClient = GravitinoClient
+    .builder("http://localhost:8090";)
+    .withMetalake("metalake")
+    .build();
+// create a catalog first
+Catalog catalog = gravitinoClient.createCatalog(
+    "test_catalog",
+    Type.FILESET,
+    "hadoop", // provider
+    "comment",
+    ImmutableMap.of(
+        "filesystem-providers", "builtin-local,builtin-hdfs,s3,gcs",
+        "location-l1", 
"file:///{{catalog}}/{{schema}}/workspace_{{project}}/{{user}}",
+        "location-l2", 
"hdfs:///{{catalog}}/{{schema}}/workspace_{{project}}/{{user}}"));
+FilesetCatalog filesetCatalog = catalog.asFilesetCatalog();
+
+// create a schema under the catalog
+filesetCatalog.createSchema(
+    "test_schema",
+    "comment",
+    ImmutableMap.of("location-l3", 
"s3a://myBucket/{{catalog}}/{{schema}}/workspace_{{project}}/{{user}}"));
+
+// create a fileset by placeholders
+filesetCatalog.createMultipleLocationFileset(
+  NameIdentifier.of("test_schema", "example_fileset"),
+  "This is an example fileset",
+  Fileset.Type.MANAGED,
+  ImmutableMap.of("l4", 
"gs://myBucket/{{catalog}}/{{schema}}/workspace_{{project}}/{{user}}"),
+  ImmutableMap.of(
+      "placeholder-project", "test_project", 
+      "placeholder-user", "test_user",
+      "default-location-name", "l1")
+);
+
+// the fileset will be created with 4 storage locations:
+{
+  "name": "example_fileset",
+  "comment": "This is an example fileset",
+  "type": "MANAGED",
+  "storageLocation": null,
+  "storageLocations": {
+    "l1": "file:///test_catalog/test_schema/workspace_test_project/test_user",
+    "l2": "hdfs:///test_catalog/test_schema/workspace_test_project/test_user",
+    "l3": 
"s3a://myBucket/test_catalog/test_schema/workspace_test_project/test_user",
+    "l4": 
"gs://myBucket/test_catalog/test_schema/workspace_test_project/test_user"
+  },
+  "properties": {
+    "placeholder-project": "test_project",
+    "placeholder-user": "test_user",
+    "default-location-name": "l1"
+  }
+}
+```
+
+</TabItem>
+<TabItem value="python" label="Python">
+
+```python
+gravitino_client: GravitinoClient = 
GravitinoClient(uri="http://localhost:8090";, metalake_name="metalake")
+
+# create a catalog first
+catalog: Catalog = gravitino_client.create_catalog(
+   name="test_catalog",
+   catalog_type=Catalog.Type.FILESET,
+   provider="hadoop",
+   comment="comment",
+   properties={
+      "filesystem-providers": "builtin-local,builtin-hdfs,s3,gcs",
+      "location-l1": 
"file:///{{catalog}}/{{schema}}/workspace_{{project}}/{{user}}",
+      "location-l2": 
"hdfs:///{{catalog}}/{{schema}}/workspace_{{project}}/{{user}}",
+    }
+)
+
+# create a schema under the catalog
+catalog.as_schemas().create_schema(
+   name="test_schema",
+   comment="comment",
+   properties={
+      "location-l3": 
"s3a://myBucket/{{catalog}}/{{schema}}/workspace_{{project}}/{{user}}",
+   }
+)
+
+# create a fileset by placeholders
+catalog.as_fileset_catalog().create_multiple_location_fileset(
+    ident=NameIdentifier.of("test_schema", "example_fileset"),
+    type=Fileset.Type.MANAGED,
+    comment="This is an example fileset",
+    storage_locations={
+        "l4": 
"gs://myBucket/{{catalog}}/{{schema}}/workspace_{{project}}/{{user}}",
+    },
+    roperties={
+       "placeholder-project": "test_project",
+       "placeholder-user": "test_user",
+       "default-location-name": "l1",
+    }
+)
+
+# the fileset will be created with 4 storage locations:
+{
+  "name": "example_fileset",
+  "comment": "This is an example fileset",
+  "type": "MANAGED",
+  "storageLocation": null,
+  "storageLocations": {
+    "l1": "file:///test_catalog/test_schema/workspace_test_project/test_user",
+    "l2": "hdfs:///test_catalog/test_schema/workspace_test_project/test_user",
+    "l3": 
"s3a://myBucket/test_catalog/test_schema/workspace_test_project/test_user",
+    "l4": 
"gs://myBucket/test_catalog/test_schema/workspace_test_project/test_user"
+  },
+  "properties": {
+    "placeholder-project": "test_project",
+    "placeholder-user": "test_user",
+    "default-location-name": "l1"
+  }
+}
+```
+
+</TabItem>
+</Tabs>
 
 ### Alter a fileset
 

Reply via email to