Skip to content

Commit

Permalink
Support flink iceberg catalog #3515
Browse files Browse the repository at this point in the history
  • Loading branch information
sunxiaojian committed Dec 19, 2024
1 parent a185d10 commit ef294ba
Show file tree
Hide file tree
Showing 16 changed files with 1,096 additions and 41 deletions.
83 changes: 83 additions & 0 deletions docs/flink-connector/flink-catalog-iceberg.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
---
title: "Flink connector Iceberg catalog"
slug: /flink-connector/flink-catalog-iceberg
keyword: flink connector iceberg catalog
license: "This software is licensed under the Apache License version 2."
---

The Apache Gravitino Flink connector offers the capability to read and write Iceberg tables, with the metadata managed by the Gravitino server. To enable the use of the Iceberg catalog within the Flink connector, you must download the Iceberg Flink runtime JAR to the Flink classpath.

## Capabilities

#### Supported DML and DDL operations:

- `CREATE CATALOG`
- `CREATE DATABASE`
- `CREATE TABLE`
- `DROP TABLE`
- `ALTER TABLE`
- `INSERT INTO & OVERWRITE`
- `SELECT`

#### Not supported operations:

- Partition operations
- View operations
- Metadata tables, like:
- `{iceberg_catalog}.{iceberg_database}.{iceberg_table}&snapshots`
- Querying UDF
- `UPDATE` clause
- `DELETE` clause
- `CREATE TABLE LIKE` clause

## SQL example
```sql

-- Suppose iceberg_a is the Iceberg catalog name managed by Gravitino

USE iceberg_a;

CREATE DATABASE IF NOT EXISTS mydatabase;
USE mydatabase;

CREATE TABLE sample (
id BIGINT COMMENT 'unique id',
data STRING NOT NULL
) PARTITIONED BY (data)
WITH ('format-version'='2');

INSERT INTO sample
VALUES (1, 'A'), (2, 'B');

SELECT * FROM sample WHERE data = 'B';

```

## Catalog properties

Gravitino Flink connector will transform the following property names defined in catalog properties to Flink Iceberg connector configuration.

| Gravitino catalog property name | Flink Iceberg connector configuration | Description | Since Version |
|---------------------------------|---------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------------------|
| `catalog-backend` | `catalog-type` | Catalog backend type | 0.8.0-incubating |
| `uri` | `uri` | Catalog backend URI | 0.8.0-incubating |
| `warehouse` | `warehouse` | Catalog backend warehouse | 0.8.0-incubating |
| `io-impl` | `io-impl` | The IO implementation for `FileIO` in Iceberg. | 0.8.0-incubating |
| `s3-endpoint` | `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. | 0.8.0-incubating |
| `s3-region` | `client.region` | The region of the S3 service, like `us-west-2`. | 0.8.0-incubating |
| `oss-endpoint` | `oss.endpoint` | The endpoint of Aliyun OSS service. | 0.8.0-incubating |


## Storage

### S3

You need to add s3 secret to the Flink configuration using `s3.access-key-id` and `s3.secret-access-key`. Additionally, download the [Iceberg AWS bundle](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws-bundle) and place it in the classpath of Flink.

### OSS

You need to add OSS secret key to the Flink configuration using `client.access-key-id` and `client.access-key-secret`. Additionally, download the [Aliyun OSS SDK](https://gosspublic.alicdn.com/sdks/java/aliyun_java_sdk_3.10.2.zip) and copy `aliyun-sdk-oss-3.10.2.jar`, `hamcrest-core-1.1.jar`, `jdom2-2.0.6.jar` in the classpath of Flink.

### GCS

No extra configuration is needed. Please make sure the credential file is accessible by Flink, like using `export GOOGLE_APPLICATION_CREDENTIALS=/xx/application_default_credentials.json`, and download [Iceberg GCP bundle](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-gcp-bundle) and place it to the classpath of Flink.
6 changes: 6 additions & 0 deletions flink-connector/flink/build.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,8 @@ repositories {
val flinkVersion: String = libs.versions.flink.get()
val flinkMajorVersion: String = flinkVersion.substringBeforeLast(".")

val icebergVersion: String = libs.versions.iceberg.get()

// The Flink only support scala 2.12, and all scala api will be removed in a future version.
// You can find more detail at the following issues:
// https://issues.apache.org/jira/browse/FLINK-23986,
Expand All @@ -43,6 +45,8 @@ dependencies {

compileOnly(project(":clients:client-java-runtime", configuration = "shadow"))

compileOnly("org.apache.iceberg:iceberg-flink-runtime-$flinkMajorVersion:$icebergVersion")

compileOnly("org.apache.flink:flink-connector-hive_$scalaVersion:$flinkVersion")
compileOnly("org.apache.flink:flink-table-common:$flinkVersion")
compileOnly("org.apache.flink:flink-table-api-java:$flinkVersion")
Expand Down Expand Up @@ -86,7 +90,9 @@ dependencies {
testImplementation(libs.testcontainers)
testImplementation(libs.testcontainers.junit.jupiter)
testImplementation(libs.testcontainers.mysql)
testImplementation(libs.metrics.core)

testImplementation("org.apache.iceberg:iceberg-flink-runtime-$flinkMajorVersion:$icebergVersion")
testImplementation("org.apache.flink:flink-connector-hive_$scalaVersion:$flinkVersion")
testImplementation("org.apache.flink:flink-table-common:$flinkVersion")
testImplementation("org.apache.flink:flink-table-api-java:$flinkVersion")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,12 +35,10 @@ public class GravitinoCatalogManager {
private static GravitinoCatalogManager gravitinoCatalogManager;

private volatile boolean isClosed = false;
private final String metalakeName;
private final GravitinoMetalake metalake;
private final GravitinoAdminClient gravitinoClient;

private GravitinoCatalogManager(String gravitinoUri, String metalakeName) {
this.metalakeName = metalakeName;
this.gravitinoClient = GravitinoAdminClient.builder(gravitinoUri).build();
this.metalake = gravitinoClient.loadMetalake(metalakeName);
}
Expand Down Expand Up @@ -99,15 +97,6 @@ public Catalog getGravitinoCatalogInfo(String name) {
return catalog;
}

/**
* Get the metalake.
*
* @return the metalake name.
*/
public String getMetalakeName() {
return metalakeName;
}

/**
* Create catalog in Gravitino.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.gravitino.flink.connector.iceberg;

import java.util.Map;
import java.util.Optional;
import org.apache.flink.table.catalog.AbstractCatalog;
import org.apache.flink.table.catalog.exceptions.CatalogException;
import org.apache.flink.table.factories.Factory;
import org.apache.gravitino.flink.connector.PartitionConverter;
import org.apache.gravitino.flink.connector.PropertiesConverter;
import org.apache.gravitino.flink.connector.catalog.BaseCatalog;
import org.apache.iceberg.flink.FlinkCatalog;
import org.apache.iceberg.flink.FlinkCatalogFactory;

/** Gravitino Iceberg Catalog. */
public class GravitinoIcebergCatalog extends BaseCatalog {

private final FlinkCatalog icebergCatalog;

protected GravitinoIcebergCatalog(
String catalogName,
String defaultDatabase,
PropertiesConverter propertiesConverter,
PartitionConverter partitionConverter,
Map<String, String> properties) {
super(catalogName, defaultDatabase, propertiesConverter, partitionConverter);
FlinkCatalogFactory flinkCatalogFactory = new FlinkCatalogFactory();
this.icebergCatalog = (FlinkCatalog) flinkCatalogFactory.createCatalog(catalogName, properties);
}

@Override
public void open() throws CatalogException {
icebergCatalog.open();
}

@Override
public void close() throws CatalogException {
icebergCatalog.close();
}

@Override
public Optional<Factory> getFactory() {
return icebergCatalog.getFactory();
}

@Override
protected AbstractCatalog realCatalog() {
return icebergCatalog;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,95 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.gravitino.flink.connector.iceberg;

import java.util.Collections;
import java.util.Set;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.table.catalog.Catalog;
import org.apache.flink.table.factories.FactoryUtil;
import org.apache.gravitino.flink.connector.DefaultPartitionConverter;
import org.apache.gravitino.flink.connector.PartitionConverter;
import org.apache.gravitino.flink.connector.PropertiesConverter;
import org.apache.gravitino.flink.connector.catalog.BaseCatalogFactory;
import org.apache.gravitino.flink.connector.utils.FactoryUtils;

public class GravitinoIcebergCatalogFactory implements BaseCatalogFactory {

@Override
public Catalog createCatalog(Context context) {
final FactoryUtil.CatalogFactoryHelper helper =
FactoryUtils.createCatalogFactoryHelper(this, context);
return new GravitinoIcebergCatalog(
context.getName(),
helper.getOptions().get(GravitinoIcebergCatalogFactoryOptions.DEFAULT_DATABASE),
propertiesConverter(),
partitionConverter(),
context.getOptions());
}

@Override
public String factoryIdentifier() {
return GravitinoIcebergCatalogFactoryOptions.IDENTIFIER;
}

@Override
public Set<ConfigOption<?>> requiredOptions() {
return Collections.emptySet();
}

@Override
public Set<ConfigOption<?>> optionalOptions() {
return Collections.emptySet();
}

/**
* Define gravitino catalog provider.
*
* @return
*/
@Override
public String gravitinoCatalogProvider() {
return "lakehouse-iceberg";
}

/**
* Define gravitino catalog type.
*
* @return
*/
@Override
public org.apache.gravitino.Catalog.Type gravitinoCatalogType() {
return org.apache.gravitino.Catalog.Type.RELATIONAL;
}

/**
* Define properties converter.
*
* @return
*/
@Override
public PropertiesConverter propertiesConverter() {
return IcebergPropertiesConverter.INSTANCE;
}

@Override
public PartitionConverter partitionConverter() {
return DefaultPartitionConverter.INSTANCE;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.gravitino.flink.connector.iceberg;

import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
import org.apache.iceberg.flink.FlinkCatalogFactory;

public class GravitinoIcebergCatalogFactoryOptions {

public static final String IDENTIFIER = "gravitino-iceberg";
public static final ConfigOption<String> DEFAULT_DATABASE =
ConfigOptions.key(FlinkCatalogFactory.DEFAULT_DATABASE)
.stringType()
.defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_NAME);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.gravitino.flink.connector.iceberg;

import com.google.common.annotations.VisibleForTesting;
import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConstants;
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.CatalogUtil;
import org.apache.iceberg.flink.FlinkCatalogFactory;

public interface IcebergPropertiesConstants {
@VisibleForTesting String GRAVITINO_ICEBERG_CATALOG_BACKEND = IcebergConstants.CATALOG_BACKEND;

String ICEBERG_CATALOG_TYPE = FlinkCatalogFactory.ICEBERG_CATALOG_TYPE;

String GRAVITINO_ICEBERG_CATALOG_WAREHOUSE = IcebergConstants.WAREHOUSE;

String ICEBERG_CATALOG_WAREHOUSE = CatalogProperties.WAREHOUSE_LOCATION;

String GRAVITINO_ICEBERG_CATALOG_URI = IcebergConstants.URI;

String ICEBERG_CATALOG_URI = CatalogProperties.URI;

@VisibleForTesting String ICEBERG_CATALOG_BACKEND_HIVE = CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE;

String GRAVITINO_ICEBERG_CATALOG_BACKEND_HIVE = "hive";

@VisibleForTesting String ICEBERG_CATALOG_BACKEND_REST = CatalogUtil.ICEBERG_CATALOG_TYPE_REST;
}
Loading

0 comments on commit ef294ba

Please sign in to comment.