Skip to content

Commit

Permalink
Support JDBC catalog in Iceberg connector
Browse files Browse the repository at this point in the history
  • Loading branch information
ebyhr committed Jun 13, 2022
1 parent e071da4 commit a14c771
Show file tree
Hide file tree
Showing 27 changed files with 1,696 additions and 88 deletions.
25 changes: 22 additions & 3 deletions docs/src/main/sphinx/connector/iceberg.rst
Original file line number Diff line number Diff line change
Expand Up @@ -45,10 +45,10 @@ To use Iceberg, you need:
Configuration
-------------

The connector supports two Iceberg catalog types, you may use either a Hive
metastore service (HMS) or AWS Glue. The catalog type is determined by the
The connector supports several Iceberg catalog types, you may use either a Hive
metastore service (HMS), AWS Glue or JDBC. The catalog type is determined by the
``iceberg.catalog.type`` property, it can be set to either ``HIVE_METASTORE``
or ``GLUE``.
, ``GLUE`` or ``JDBC``.

Hive metastore catalog
^^^^^^^^^^^^^^^^^^^^^^
Expand Down Expand Up @@ -77,6 +77,25 @@ configuration properties as the Hive connector's Glue setup. See
iceberg.catalog.type=glue
JDBC catalog
^^^^^^^^^^^^

At a minimum, ``iceberg.metastore.jdbc.connection-url`` and
``iceberg.metastore.jdbc.catalogid`` must be configured.
A jar file of JDBC driver needs to be located in the plugin directory
when using it.

.. code-block:: text
connector.name=iceberg
iceberg.catalog.type=jdbc
iceberg.metastore.jdbc.catalogid=test
iceberg.metastore.jdbc.connection-url=jdbc:postgresql://example.net:5432/database
iceberg.metastore.jdbc.connection-user=root
iceberg.metastore.jdbc.connection-password=secret
iceberg.metastore.jdbc.default-warehouse-dir=s3://bucket
General configuration
^^^^^^^^^^^^^^^^^^^^^

Expand Down
23 changes: 23 additions & 0 deletions plugin/trino-iceberg/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -197,6 +197,11 @@
</exclusions>
</dependency>

<dependency>
<groupId>org.jdbi</groupId>
<artifactId>jdbi3-core</artifactId>
</dependency>

<dependency>
<groupId>org.weakref</groupId>
<artifactId>jmxutils</artifactId>
Expand Down Expand Up @@ -247,6 +252,12 @@
<scope>runtime</scope>
</dependency>

<dependency>
<groupId>org.postgresql</groupId>
<artifactId>postgresql</artifactId>
<scope>runtime</scope>
</dependency>

<!-- Trino SPI -->
<dependency>
<groupId>io.trino</groupId>
Expand Down Expand Up @@ -354,6 +365,18 @@
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>postgresql</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>testcontainers</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.testng</groupId>
<artifactId>testng</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,5 +18,6 @@ public enum CatalogType
TESTING_FILE_METASTORE,
HIVE_METASTORE,
GLUE,
JDBC,
/**/;
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,12 +30,14 @@
import io.trino.plugin.iceberg.catalog.glue.IcebergGlueCatalogModule;
import io.trino.plugin.iceberg.catalog.hms.IcebergHiveMetastoreCatalogModule;
import io.trino.plugin.iceberg.catalog.hms.TrinoHiveCatalogFactory;
import io.trino.plugin.iceberg.catalog.jdbc.IcebergJdbcCatalogModule;

import java.util.Optional;

import static io.airlift.configuration.ConditionalModule.conditionalModule;
import static io.trino.plugin.iceberg.CatalogType.GLUE;
import static io.trino.plugin.iceberg.CatalogType.HIVE_METASTORE;
import static io.trino.plugin.iceberg.CatalogType.JDBC;
import static io.trino.plugin.iceberg.CatalogType.TESTING_FILE_METASTORE;
import static java.util.Objects.requireNonNull;

Expand Down Expand Up @@ -63,6 +65,7 @@ protected void setup(Binder binder)
bindCatalogModule(HIVE_METASTORE, new IcebergHiveMetastoreCatalogModule());
bindCatalogModule(TESTING_FILE_METASTORE, new IcebergFileMetastoreCatalogModule());
bindCatalogModule(GLUE, new IcebergGlueCatalogModule());
bindCatalogModule(JDBC, new IcebergJdbcCatalogModule());
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
/*
* 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 io.trino.plugin.iceberg.catalog.jdbc;

import com.google.inject.Binder;
import com.google.inject.Provides;
import com.google.inject.Scopes;
import com.google.inject.Singleton;
import io.airlift.configuration.AbstractConfigurationAwareModule;
import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider;
import io.trino.plugin.iceberg.catalog.TrinoCatalogFactory;

import static io.airlift.configuration.ConfigBinder.configBinder;
import static org.weakref.jmx.guice.ExportBinder.newExporter;

public class IcebergJdbcCatalogModule
extends AbstractConfigurationAwareModule
{
@Override
protected void setup(Binder binder)
{
configBinder(binder).bindConfig(JdbcIcebergConfig.class);
binder.bind(TrinoJdbcCatalogFactory.class);
binder.bind(IcebergTableOperationsProvider.class).to(JdbcIcebergTableOperationsProvider.class).in(Scopes.SINGLETON);
newExporter(binder).export(IcebergTableOperationsProvider.class).withGeneratedName();
binder.bind(TrinoCatalogFactory.class).to(TrinoJdbcCatalogFactory.class).in(Scopes.SINGLETON);
newExporter(binder).export(TrinoJdbcCatalogFactory.class).withGeneratedName();
}

@Provides
@Singleton
public static JdbcIcebergClient createJdbcIcebergClient(JdbcIcebergConfig config)
{
return new JdbcIcebergClient(
new JdbcIcebergConnectionFactory(
config.getConnectionUrl(),
config.getConnectionUser(),
config.getConnectionPassword()),
config.getCatalogId());
}
}
Loading

0 comments on commit a14c771

Please sign in to comment.