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

Add support for REST catalog in Iceberg connector #22417

Merged
merged 2 commits into from
May 24, 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
1 change: 1 addition & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,7 @@
<dep.docker-java.version>3.3.0</dep.docker-java.version>
<dep.jayway.version>2.6.0</dep.jayway.version>
<dep.ratis.version>2.2.0</dep.ratis.version>
<dep.jackson.version>2.11.0</dep.jackson.version>
<!--
America/Bahia_Banderas has:
- offset change since 1970 (offset Jan 1970: -08:00, offset Jan 2018: -06:00)
Expand Down
48 changes: 45 additions & 3 deletions presto-docs/src/main/sphinx/connector/iceberg.rst
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@ Iceberg tables store most of the metadata in the metadata files, along with the
filesystem, but it still requires a central place to find the current location of the
current metadata pointer for a table. This central place is called the ``Iceberg Catalog``.
The Presto Iceberg connector supports different types of Iceberg Catalogs : ``Hive Metastore``,
``GLUE``, ``NESSIE``, and ``HADOOP``.
``GLUE``, ``NESSIE``, ``REST`` and ``HADOOP``.

To configure the Iceberg connector, create a catalog properties file
``etc/catalog/iceberg.properties``. To define the catalog type, ``iceberg.catalog.type`` property
Expand Down Expand Up @@ -144,12 +144,49 @@ If an error similar to the following example is displayed, this is probably beca
at org.projectnessie.client.http.impl.jdk8.UrlConnectionRequest.executeRequest(UrlConnectionRequest.java:71)
... 42 more

REST catalog
^^^^^^^^^^^^

To use a REST catalog, configure the catalog type as
``iceberg.catalog.type=rest``. A minimal configuration includes:

.. code-block:: none

connector.name=iceberg
iceberg.catalog.type=rest
iceberg.rest.uri=https://localhost:8181

Additional supported properties for the REST catalog:

==================================================== ============================================================
Property Name Description
==================================================== ============================================================
``iceberg.rest.uri`` REST API endpoint URI (required).
Example: ``https://localhost:8181``

``iceberg.rest.auth.type`` The authentication type to use.
Available values are ``NONE`` or ``OAUTH2`` (default: ``NONE``).
``OAUTH2`` requires either a credential or token.

``iceberg.rest.auth.oauth2.credential`` The credential to use for OAUTH2 authentication.
Example: ``key:secret``

``iceberg.rest.auth.oauth2.token`` The Bearer token to use for OAUTH2 authentication.
Example: ``SXVLUXUhIExFQ0tFUiEK``

``iceberg.rest.session.type`` The session type to use when communicating with the REST catalog.
Available values are ``NONE`` or ``USER`` (default: ``NONE``).

``iceberg.catalog.warehouse`` A catalog warehouse root path for Iceberg tables (optional).
Example: ``s3://warehouse/``

==================================================== ============================================================

Hadoop catalog
^^^^^^^^^^^^^^

To use a Hadoop catalog, configure the catalog type as
``iceberg.catalog.type=hadoop``
``iceberg.catalog.type=hadoop``. A minimal configuration includes:

.. code-block:: none

Expand Down Expand Up @@ -190,7 +227,7 @@ Property Name Description

Example: ``hdfs://nn:8020/warehouse/path``
This property is required if the ``iceberg.catalog.type`` is
``hadoop``. Otherwise, it will be ignored.
``hadoop``.

``iceberg.catalog.cached-catalog-num`` The number of Iceberg catalogs to cache. This property is ``10``
required if the ``iceberg.catalog.type`` is ``hadoop``.
Expand Down Expand Up @@ -898,6 +935,11 @@ in the case where a specific metadata file contains the targeted table state::

CALL iceberg.system.register_table('schema_name', 'table_name', 'hdfs://localhost:9000/path/to/iceberg/table/metadata/dir', '00000-35a08aed-f4b0-4010-95d2-9d73ef4be01c.metadata.json')

.. note::

The Iceberg REST catalog may not support table register depending on the
type of the backing catalog.

.. note::

When registering a table with the Hive metastore, the user calling the
Expand Down
58 changes: 55 additions & 3 deletions presto-iceberg/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -192,14 +192,12 @@
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
<version>2.10.2</version>
<scope>compile</scope>
</dependency>

<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
<version>2.10.2</version>
<scope>compile</scope>
</dependency>

Expand Down Expand Up @@ -475,6 +473,21 @@
<artifactId>datasketches-memory</artifactId>
</dependency>

<dependency>
<groupId>io.jsonwebtoken</groupId>
<artifactId>jjwt-api</artifactId>
</dependency>
<dependency>
<groupId>io.jsonwebtoken</groupId>
<artifactId>jjwt-impl</artifactId>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>io.jsonwebtoken</groupId>
<artifactId>jjwt-jackson</artifactId>
<scope>runtime</scope>
</dependency>

<!-- for testing -->
<dependency>
<groupId>org.openjdk.jmh</groupId>
Expand Down Expand Up @@ -576,6 +589,40 @@
<artifactId>presto-jmx</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.iceberg</groupId>
<artifactId>iceberg-core</artifactId>
<version>1.5.0</version>
<classifier>tests</classifier>
<scope>test</scope>
<exclusions>
<exclusion>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-column</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-jdk14</artifactId>
</exclusion>
<exclusion>
<groupId>io.airlift</groupId>
<artifactId>aircompressor</artifactId>
</exclusion>
<exclusion>
<groupId>org.roaringbitmap</groupId>
<artifactId>RoaringBitmap</artifactId>
</exclusion>
</exclusions>
</dependency>
</dependencies>

<build>
Expand All @@ -585,10 +632,15 @@
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-dependency-plugin</artifactId>
<configuration>
<!-- Ignore these because they are picked up as false-positives when configuring logging in the IcebergQueryRunner-->
<ignoredUsedUndeclaredDependencies>
<!-- Ignore these because they are picked up as false-positives when configuring logging in the IcebergQueryRunner -->
<dependency>org.glassfish.jersey.core:jersey-common:jar</dependency>
<dependency>org.eclipse.jetty:jetty-server:jar</dependency>
<!-- Ignore these because they are picked up as false-positives when configuring REST catalog testing http server -->
<dependency>com.facebook.airlift:http-server:jar</dependency>
<dependency>com.facebook.airlift:node:jar</dependency>
<dependency>javax.servlet:javax.servlet-api:jar</dependency>
<dependency>org.apache.httpcomponents.core5:httpcore5:jar</dependency>
</ignoredUsedUndeclaredDependencies>
</configuration>
</plugin>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,14 @@
import org.apache.iceberg.hadoop.HadoopCatalog;
import org.apache.iceberg.hive.HiveCatalog;
import org.apache.iceberg.nessie.NessieCatalog;
import org.apache.iceberg.rest.RESTCatalog;

public enum CatalogType
{
HADOOP(HadoopCatalog.class.getName()),
HIVE(HiveCatalog.class.getName()),
NESSIE(NessieCatalog.class.getName()),

REST(RESTCatalog.class.getName())
/**/;

private final String catalogImpl;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
/*
* Licensed 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 com.facebook.presto.iceberg;

import com.facebook.airlift.configuration.AbstractConfigurationAwareModule;
import com.facebook.presto.hive.metastore.ExtendedHiveMetastore;
import com.facebook.presto.iceberg.hadoop.IcebergHadoopCatalogModule;
import com.facebook.presto.iceberg.nessie.IcebergNessieCatalogModule;
import com.facebook.presto.iceberg.rest.IcebergRestCatalogModule;
import com.google.inject.Binder;
import com.google.inject.Module;

import java.util.Optional;

import static com.facebook.airlift.configuration.ConditionalModule.installModuleIf;
import static com.facebook.presto.iceberg.CatalogType.HADOOP;
import static com.facebook.presto.iceberg.CatalogType.HIVE;
import static com.facebook.presto.iceberg.CatalogType.NESSIE;
import static com.facebook.presto.iceberg.CatalogType.REST;

public class IcebergCatalogModule
extends AbstractConfigurationAwareModule
{
private final String connectorId;
private final Optional<ExtendedHiveMetastore> metastore;

public IcebergCatalogModule(String connectorId, Optional<ExtendedHiveMetastore> metastore)
{
this.connectorId = connectorId;
this.metastore = metastore;
}

@Override
protected void setup(Binder binder)
{
bindCatalogModule(HIVE, new IcebergHiveModule(connectorId, metastore));
bindCatalogModule(HADOOP, new IcebergHadoopCatalogModule());
bindCatalogModule(NESSIE, new IcebergNessieCatalogModule());
bindCatalogModule(REST, new IcebergRestCatalogModule());
}

private void bindCatalogModule(CatalogType catalogType, Module module)
{
install(installModuleIf(
IcebergConfig.class,
config -> config.getCatalogType().equals(catalogType),
module));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@
import com.facebook.presto.hive.gcs.GcsConfigurationInitializer;
import com.facebook.presto.hive.gcs.HiveGcsConfig;
import com.facebook.presto.hive.gcs.HiveGcsConfigurationInitializer;
import com.facebook.presto.iceberg.nessie.NessieConfig;
import com.facebook.presto.iceberg.nessie.IcebergNessieConfig;
import com.facebook.presto.iceberg.optimizer.IcebergPlanOptimizerProvider;
import com.facebook.presto.iceberg.procedure.ExpireSnapshotsProcedure;
import com.facebook.presto.iceberg.procedure.RegisterTableProcedure;
Expand Down Expand Up @@ -90,6 +90,7 @@
import static com.facebook.airlift.json.JsonCodecBinder.jsonCodecBinder;
import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService;
import static com.google.inject.multibindings.Multibinder.newSetBinder;
import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder;
import static java.lang.Math.toIntExact;
import static java.util.concurrent.Executors.newFixedThreadPool;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
Expand Down Expand Up @@ -124,12 +125,12 @@ public void setup(Binder binder)
binder.bind(CacheFactory.class).in(Scopes.SINGLETON);
binder.bind(IcebergTransactionManager.class).in(Scopes.SINGLETON);
binder.bind(IcebergCatalogName.class).toInstance(new IcebergCatalogName(connectorId));
binder.bind(IcebergResourceFactory.class).in(Scopes.SINGLETON);

configBinder(binder).bindConfig(IcebergConfig.class);
configBinder(binder).bindConfig(NessieConfig.class);

binder.bind(IcebergSessionProperties.class).in(Scopes.SINGLETON);
newOptionalBinder(binder, IcebergNessieConfig.class); // bind optional Nessie config to IcebergSessionProperties

binder.bind(IcebergTableProperties.class).in(Scopes.SINGLETON);

binder.bind(ConnectorSplitManager.class).to(IcebergSplitManager.class).in(Scopes.SINGLETON);
Expand Down
Loading
Loading