Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[#5315] doc(iceberg): add credential vending document for Iceberg REST Server #5314

Merged
merged 7 commits into from
Oct 29, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ public void initialize(Map<String, String> properties) {
S3CredentialConfig s3CredentialConfig = new S3CredentialConfig(properties);
this.roleArn = s3CredentialConfig.s3RoleArn();
this.externalID = s3CredentialConfig.externalID();
this.tokenExpireSecs = s3CredentialConfig.tokenExpireSecs();
this.tokenExpireSecs = s3CredentialConfig.tokenExpireInSecs();
this.stsClient = createStsClient(s3CredentialConfig);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
public class CredentialConstants {
public static final String CREDENTIAL_PROVIDER_TYPE = "credential-provider-type";
public static final String S3_TOKEN_CREDENTIAL_PROVIDER = "s3-token";
public static final String TOKEN_EXPIRE_TIME = "token-expire-time";
public static final String S3_TOKEN_EXPIRE_IN_SECS = "s3-token-expire-in-secs";

public static final String GCS_TOKEN_CREDENTIAL_PROVIDER_TYPE = "gcs-token";

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,9 +69,9 @@ public class S3CredentialConfig extends Config {
.stringConf()
.create();

public static final ConfigEntry<Integer> S3_TOKEN_EXPIRE_SECS =
new ConfigBuilder(CredentialConstants.TOKEN_EXPIRE_TIME)
.doc("S3 token expire seconds")
public static final ConfigEntry<Integer> S3_TOKEN_EXPIRE_IN_SECS =
new ConfigBuilder(CredentialConstants.S3_TOKEN_EXPIRE_IN_SECS)
.doc("S3 token expire in seconds")
.version(ConfigConstants.VERSION_0_7_0)
.intConf()
.createWithDefault(3600);
Expand Down Expand Up @@ -104,7 +104,7 @@ public String externalID() {
return this.get(S3_EXTERNAL_ID);
}

public Integer tokenExpireSecs() {
return this.get(S3_TOKEN_EXPIRE_SECS);
public Integer tokenExpireInSecs() {
return this.get(S3_TOKEN_EXPIRE_IN_SECS);
}
}
37 changes: 24 additions & 13 deletions docs/iceberg-rest-service.md
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ The Apache Gravitino Iceberg REST Server follows the [Apache Iceberg REST API sp
- multi table transaction
- pagination
- Works as a catalog proxy, supporting `Hive` and `JDBC` as catalog backend.
- Supports credential vending for `S3` and `GCS`.
- Supports different storages like `S3`, `HDFS`, `OSS`, `GCS` and provides the capability to support other storages.
- Supports event listener.
- Supports OAuth2 and HTTPS.
Expand Down Expand Up @@ -101,18 +102,24 @@ The detailed configuration items are as follows:

#### S3 configuration

Gravitino Iceberg REST service supports using static access-key-id and secret-access-key to access S3 data.
Gravitino Iceberg REST service supports using static S3 secret key or generating temporary token to access S3 data.

| Configuration item | Description | Default value | Required | Since Version |
|-----------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|---------------|----------|------------------|
| `gravitino.iceberg-rest.io-impl` | The IO implementation for `FileIO` in Iceberg, use `org.apache.iceberg.aws.s3.S3FileIO` for S3. | (none) | No | 0.6.0-incubating |
| `gravitino.iceberg-rest.s3-access-key-id` | The static access key ID used to access S3 data. | (none) | No | 0.6.0-incubating |
| `gravitino.iceberg-rest.s3-secret-access-key` | The static secret access key used to access S3 data. | (none) | No | 0.6.0-incubating |
| `gravitino.iceberg-rest.s3-endpoint` | An alternative endpoint of the S3 service, This could be used for S3FileIO with any s3-compatible object storage service that has a different endpoint, or access a private S3 endpoint in a virtual private cloud. | (none) | No | 0.6.0-incubating |
| `gravitino.iceberg-rest.s3-region` | The region of the S3 service, like `us-west-2`. | (none) | No | 0.6.0-incubating |
| Configuration item | Description | Default value | Required | Since Version |
|---------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|---------------|----------------------------------------------------|------------------|
| `gravitino.iceberg-rest.io-impl` | The IO implementation for `FileIO` in Iceberg, use `org.apache.iceberg.aws.s3.S3FileIO` for S3. | (none) | No | 0.6.0-incubating |
| `gravitino.iceberg-rest.credential-provider-type` | Supports `s3-token` and `s3-secret-key` for S3. `s3-token` generates a temporary token according to the query data path while `s3-secret-key` using the s3 secret access key to access S3 data. | (none) | No | 0.7.0-incubating |
| `gravitino.iceberg-rest.s3-access-key-id` | The static access key ID used to access S3 data. | (none) | No | 0.6.0-incubating |
| `gravitino.iceberg-rest.s3-secret-access-key` | The static secret access key used to access S3 data. | (none) | No | 0.6.0-incubating |
| `gravitino.iceberg-rest.s3-endpoint` | An alternative endpoint of the S3 service, This could be used for S3FileIO with any s3-compatible object storage service that has a different endpoint, or access a private S3 endpoint in a virtual private cloud. | (none) | No | 0.6.0-incubating |
| `gravitino.iceberg-rest.s3-region` | The region of the S3 service, like `us-west-2`. | (none) | No | 0.6.0-incubating |
| `gravitino.iceberg-rest.s3-role-arn` | The ARN of the role to access the S3 data. | (none) | Yes, when `credential-provider-type` is `s3-token` | 0.7.0-incubating |
| `gravitino.iceberg-rest.s3-external-id` | The S3 external id to generate token, only used when `credential-provider-type` is `s3-token`. | (none) | No | 0.7.0-incubating |
| `gravitino.iceberg-rest.s3-token-expire-in-secs` | The S3 session token expire time in secs, it couldn't exceed the max session time of the assumed role, only used when `credential-provider-type` is `s3-token`. | 3600 | No | 0.7.0-incubating |

For other Iceberg s3 properties not managed by Gravitino like `s3.sse.type`, you could config it directly by `gravitino.iceberg-rest.s3.sse.type`.

If you set `credential-provider-type` explicitly, please downloading [Gravitino AWS bundle jar](https://mvnrepository.com/artifact/org.apache.gravitino/aws-bundle), and place it to the classpath of Iceberg REST server.

:::info
To configure the JDBC catalog backend, set the `gravitino.iceberg-rest.warehouse` parameter to `s3://{bucket_name}/${prefix_name}`. For the Hive catalog backend, set `gravitino.iceberg-rest.warehouse` to `s3a://{bucket_name}/${prefix_name}`. Additionally, download the [Iceberg AWS bundle](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws-bundle) and place it in the classpath of Iceberg REST server.
:::
Expand All @@ -136,14 +143,18 @@ Please set the `gravitino.iceberg-rest.warehouse` parameter to `oss://{bucket_na

#### GCS

Supports using google credential file to access GCS data.
Supports using static GCS credential file or generating GCS token to access GCS data.

| Configuration item | Description | Default value | Required | Since Version |
|----------------------------------|----------------------------------------------------------------------------------------------------|---------------|----------|------------------|
| `gravitino.iceberg-rest.io-impl` | The io implementation for `FileIO` in Iceberg, use `org.apache.iceberg.gcp.gcs.GCSFileIO` for GCS. | (none) | No | 0.6.0-incubating |
| Configuration item | Description | Default value | Required | Since Version |
|---------------------------------------------------|----------------------------------------------------------------------------------------------------|---------------|----------|------------------|
| `gravitino.iceberg-rest.io-impl` | The io implementation for `FileIO` in Iceberg, use `org.apache.iceberg.gcp.gcs.GCSFileIO` for GCS. | (none) | No | 0.6.0-incubating |
| `gravitino.iceberg-rest.credential-provider-type` | Supports `gcs-token`, generates a temporary token according to the query data path. | (none) | No | 0.7.0-incubating |
| `gravitino.iceberg-rest.gcs-credential-file-path` | The location of GCS credential file, only used when `credential-provider-type` is `gcs-token`. | (none) | No | 0.7.0-incubating |

For other Iceberg GCS properties not managed by Gravitino like `gcs.project-id`, you could config it directly by `gravitino.iceberg-rest.gcs.project-id`.

If you set `credential-provider-type` explicitly, please downloading [Gravitino GCP bundle jar](https://mvnrepository.com/artifact/org.apache.gravitino/gcp-bundle), and place it to the classpath of Iceberg REST server.

Please make sure the credential file is accessible by Gravitino, like using `export GOOGLE_APPLICATION_CREDENTIALS=/xx/application_default_credentials.json` before Gravitino Iceberg REST server is started.

:::info
Expand Down Expand Up @@ -367,7 +378,7 @@ For example, we can configure Spark catalog options to use Gravitino Iceberg RES
--conf spark.sql.catalog.rest.uri=http://127.0.0.1:9001/iceberg/
```

You may need to adjust the Iceberg Spark runtime jar file name according to the real version number in your environment. If you want to access the data stored in cloud, you need to download corresponding jars (please refer to the cloud storage part) and place it in the classpath of Spark, no extra config is needed because related properties is transferred from Iceberg REST server to Iceberg REST client automatically.
You may need to adjust the Iceberg Spark runtime jar file name according to the real version number in your environment. If you want to access the data stored in cloud, you need to download corresponding jars (please refer to the cloud storage part) and place it in the classpath of Spark. If you want to enable credential vending, please set `credential-provider-type` to a proper value in the server side, set `spark.sql.catalog.rest.header.X-Iceberg-Access-Delegation` = `vended-credentials` in the client side.

For other storages not managed by Gravitino, the properties wouldn't transfer from the server to client automatically, if you want to pass custom properties to initialize `FileIO`, you could add it by `spark.sql.catalog.${iceberg_catalog_name}.${configuration_key}` = `{property_value}`.

Expand Down
Loading