Skip to content

Commit

Permalink
Support additional configurations for Nessie catalog in Iceberg conne…
Browse files Browse the repository at this point in the history
…ctor

To reduce the review effort, only the basic Nessie configurations were supported in trinodb#11701.
Nessie server can be deployed with Auth mode like keycloak. So, need to expose the Nessie client configurations to handle the Auth.
Along with that, some common Nessie server configurations like read-timeout-ms, connect-timeout-ms and compression-enabled properties
are exposed to have finer control over the Nessie commits.
  • Loading branch information
ajantha-bhat committed Jun 1, 2023
1 parent f680380 commit c0ae156
Show file tree
Hide file tree
Showing 5 changed files with 141 additions and 10 deletions.
21 changes: 20 additions & 1 deletion docs/src/main/sphinx/connector/iceberg.rst
Original file line number Diff line number Diff line change
Expand Up @@ -245,6 +245,24 @@ Property Name Description
``iceberg.nessie-catalog.default-warehouse-dir`` Default warehouse directory for schemas created without an
explicit ``location`` property.
Example: ``/tmp``

``iceberg.nessie-catalog.auth.type`` The authentication type to use.
Available values are ``NONE`` or ``BEARER``.
Example: ``BEARER``

``iceberg.nessie-catalog.auth.bearer.token`` The token to use with ``BEARER`` authentication.
Example: ``SXVLUXUhIExFQ0tFUiEK``

``iceberg.nessie-catalog.read-timeout-ms`` The read timeout in milliseconds for requests
to the Nessie server.
Example: ``5000``

``iceberg.nessie-catalog.connect-timeout-ms`` The connection timeout in milliseconds for connection
connection requests to the Nessie server.
Example: ``10000``

``iceberg.nessie-catalog.compression-enabled`` Configure whether compression should be enabled or not for
requests to the Nessie server, defaults to ``true``.
==================================================== ============================================================

.. code-block:: text
Expand All @@ -253,7 +271,8 @@ Property Name Description
iceberg.catalog.type=nessie
iceberg.nessie-catalog.uri=https://localhost:19120/api/v1
iceberg.nessie-catalog.default-warehouse-dir=/tmp

iceberg.nessie-catalog.auth.type=BEARER
iceberg.nessie-catalog.auth.bearer.token=SXVLUXUhIExFQ0tFUiEK

.. _iceberg-jdbc-catalog:

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,19 @@
/*
* 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.nessie;

public enum AuthenticationType
{
NONE, BEARER
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,18 @@
import javax.validation.constraints.NotNull;

import java.net.URI;
import java.util.Optional;

public class IcebergNessieCatalogConfig
{
private String defaultReferenceName = "main";
private String defaultWarehouseDir;
private URI serverUri;
private Optional<AuthenticationType> authenticationType = Optional.empty();
private Optional<String> bearerToken = Optional.empty();
private Optional<Integer> readTimeoutMillis = Optional.empty();
private Optional<Integer> connectTimeoutMillis = Optional.empty();
private boolean compressionEnabled = true;

@NotNull
public String getDefaultReferenceName()
Expand Down Expand Up @@ -68,4 +74,69 @@ public IcebergNessieCatalogConfig setDefaultWarehouseDir(String defaultWarehouse
this.defaultWarehouseDir = defaultWarehouseDir;
return this;
}

@Config("iceberg.nessie-catalog.auth.type")
@ConfigDescription("The authentication type to use. Available values are NONE | BEARER. Default: NONE")
public IcebergNessieCatalogConfig setAuthenticationType(AuthenticationType authenticationType)
{
this.authenticationType = Optional.ofNullable(authenticationType);
return this;
}

public Optional<AuthenticationType> getAuthenticationType()
{
return authenticationType;
}

@Config("iceberg.nessie-catalog.auth.bearer.token")
@ConfigDescription("The token to use with BEARER authentication")
public IcebergNessieCatalogConfig setBearerToken(String token)
{
this.bearerToken = Optional.ofNullable(token);
return this;
}

public Optional<String> getBearerToken()
{
return bearerToken;
}

@Config("iceberg.nessie-catalog.read-timeout-ms")
@ConfigDescription("The read timeout in milliseconds for the client. Default: 25000")
public IcebergNessieCatalogConfig setReadTimeoutMillis(Integer readTimeoutMillis)
{
this.readTimeoutMillis = Optional.ofNullable(readTimeoutMillis);
return this;
}

public Optional<Integer> getReadTimeoutMillis()
{
return readTimeoutMillis;
}

@Config("iceberg.nessie-catalog.connect-timeout-ms")
@ConfigDescription("The connection timeout in milliseconds for the client. Default: 5000")
public IcebergNessieCatalogConfig setConnectTimeoutMillis(Integer connectTimeoutMillis)
{
this.connectTimeoutMillis = Optional.ofNullable(connectTimeoutMillis);
return this;
}

public Optional<Integer> getConnectTimeoutMillis()
{
return connectTimeoutMillis;
}

@Config("iceberg.nessie-catalog.compression-enabled")
@ConfigDescription("Configure whether compression should be enabled or not. Default: true")
public IcebergNessieCatalogConfig setCompressionEnabled(boolean compressionEnabled)
{
this.compressionEnabled = compressionEnabled;
return this;
}

public boolean isCompressionEnabled()
{
return compressionEnabled;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,9 +23,11 @@
import io.trino.plugin.iceberg.catalog.TrinoCatalogFactory;
import org.apache.iceberg.nessie.NessieIcebergClient;
import org.projectnessie.client.api.NessieApiV1;
import org.projectnessie.client.auth.BearerAuthenticationProvider;
import org.projectnessie.client.http.HttpClientBuilder;

import static io.airlift.configuration.ConfigBinder.configBinder;
import static io.trino.plugin.iceberg.catalog.nessie.AuthenticationType.BEARER;
import static org.weakref.jmx.guice.ExportBinder.newExporter;

public class IcebergNessieCatalogModule
Expand All @@ -43,14 +45,19 @@ protected void setup(Binder binder)

@Provides
@Singleton
public static NessieIcebergClient createNessieIcebergClient(IcebergNessieCatalogConfig icebergNessieCatalogConfig)
public static NessieIcebergClient createNessieIcebergClient(IcebergNessieCatalogConfig config)
{
return new NessieIcebergClient(
HttpClientBuilder.builder()
.withUri(icebergNessieCatalogConfig.getServerUri())
.build(NessieApiV1.class),
icebergNessieCatalogConfig.getDefaultReferenceName(),
null,
ImmutableMap.of());
HttpClientBuilder builder = HttpClientBuilder.builder()
.withUri(config.getServerUri())
.withDisableCompression(!config.isCompressionEnabled());

config.getReadTimeoutMillis().ifPresent(builder::withReadTimeout);
config.getConnectTimeoutMillis().ifPresent(builder::withConnectionTimeout);
config.getAuthenticationType().ifPresent(type -> {
if (type.equals(BEARER)) {
config.getBearerToken().ifPresent(token -> builder.withAuthentication(BearerAuthenticationProvider.create(token)));
}
});
return new NessieIcebergClient(builder.build(NessieApiV1.class), config.getDefaultReferenceName(), null, ImmutableMap.of());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,11 @@ public void testDefaults()
assertRecordedDefaults(recordDefaults(IcebergNessieCatalogConfig.class)
.setDefaultWarehouseDir(null)
.setServerUri(null)
.setAuthenticationType(null)
.setBearerToken(null)
.setReadTimeoutMillis(null)
.setConnectTimeoutMillis(null)
.setCompressionEnabled(true)
.setDefaultReferenceName("main"));
}

Expand All @@ -41,12 +46,22 @@ public void testExplicitPropertyMapping()
.put("iceberg.nessie-catalog.default-warehouse-dir", "/tmp")
.put("iceberg.nessie-catalog.uri", "http://localhost:xxx/api/v1")
.put("iceberg.nessie-catalog.ref", "someRef")
.put("iceberg.nessie-catalog.auth.type", "NONE")
.put("iceberg.nessie-catalog.auth.bearer.token", "bearerToken")
.put("iceberg.nessie-catalog.compression-enabled", "false")
.put("iceberg.nessie-catalog.connect-timeout-ms", "123")
.put("iceberg.nessie-catalog.read-timeout-ms", "456")
.buildOrThrow();

IcebergNessieCatalogConfig expected = new IcebergNessieCatalogConfig()
.setDefaultWarehouseDir("/tmp")
.setServerUri(URI.create("http://localhost:xxx/api/v1"))
.setDefaultReferenceName("someRef");
.setDefaultReferenceName("someRef")
.setAuthenticationType(AuthenticationType.NONE)
.setBearerToken("bearerToken")
.setCompressionEnabled(false)
.setConnectTimeoutMillis(123)
.setReadTimeoutMillis(456);

assertFullMapping(properties, expected);
}
Expand Down

0 comments on commit c0ae156

Please sign in to comment.