Skip to content
Draft
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
@@ -0,0 +1,147 @@
/*
* 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.iceberg.common;

import java.io.Closeable;
import java.io.File;
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
import java.util.Map;
import java.util.function.Function;
import org.apache.gravitino.iceberg.common.authentication.AuthenticationConfig;
import org.apache.gravitino.iceberg.common.authentication.SupportsKerberos;
import org.apache.gravitino.iceberg.common.authentication.kerberos.KerberosClient;
import org.apache.gravitino.utils.PrincipalUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.jdbc.JdbcCatalog;
import org.apache.iceberg.jdbc.JdbcClientPool;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* ClosableHiveCatalog is a wrapper class to wrap Iceberg HiveCatalog to do some clean-up work like
* closing resources.
*/
public class ClosableJdbcCatalog extends JdbcCatalog implements Closeable, SupportsKerberos {

private static final Logger LOGGER= LoggerFactory.getLogger(ClosableJdbcCatalog.class);

private KerberosClient kerberosClient;

private Configuration hadoopConf;

public ClosableJdbcCatalog() {
super();
}

public ClosableJdbcCatalog(
Function<Map<String, String>, FileIO> ioBuilder,
Function<Map<String, String>, JdbcClientPool> clientPoolBuilder,
boolean initializeCatalogTables) {
super(ioBuilder, clientPoolBuilder, initializeCatalogTables);
}

/**
* Initialize the ClosableHiveCatalog with the given input name and properties.
*
* <p>Note: This method can only be called once as it will create new client pools.
*
* @param inputName name of the catalog
* @param properties properties for the catalog
*/
@Override
public void initialize(String inputName, Map<String, String> properties) {
super.initialize(inputName, properties);

AuthenticationConfig authenticationConfig = new AuthenticationConfig(properties);
if (authenticationConfig.isKerberosAuth()) {
this.kerberosClient = initKerberosClient();
}
}

public Configuration getHadoopConf() {
return hadoopConf;
}

public void setHadoopConf(Configuration hadoopConf) {
this.hadoopConf = hadoopConf;
}

@Override
public void close() {
if (kerberosClient != null) {
try {
kerberosClient.close();
} catch (Exception e) {
LOGGER.warn("Failed to close KerberosClient", e);
}
}
}

@Override
public <R> R doKerberosOperations(Executable<R> executable) throws Throwable {
Map<String, String> properties = this.properties();
AuthenticationConfig authenticationConfig = new AuthenticationConfig(properties);

final String finalPrincipalName;
String proxyKerberosPrincipalName = PrincipalUtils.getCurrentPrincipal().getName();

if (!proxyKerberosPrincipalName.contains("@")) {
finalPrincipalName =
String.format("%s@%s", proxyKerberosPrincipalName, kerberosClient.getRealm());
} else {
finalPrincipalName = proxyKerberosPrincipalName;
}

UserGroupInformation realUser =
authenticationConfig.isImpersonationEnabled()
? UserGroupInformation.createProxyUser(
finalPrincipalName, kerberosClient.getLoginUser())
: kerberosClient.getLoginUser();

return realUser.doAs(
(PrivilegedExceptionAction<R>)
() -> {
try {
return executable.execute();
} catch (Throwable e) {
if (RuntimeException.class.isAssignableFrom(e.getClass())) {
throw (RuntimeException) e;
}
throw new RuntimeException("Failed to invoke method", e);
}
});
}

private KerberosClient initKerberosClient() {
try {
KerberosClient kerberosClient = new KerberosClient(this.properties(), this.getHadoopConf());
// catalog_uuid always exists for Gravitino managed catalogs, `0` is just a fallback value.
String catalogUUID = properties().getOrDefault("catalog_uuid", "0");
File keytabFile = kerberosClient.saveKeyTabFileFromUri(Long.parseLong(catalogUUID));
kerberosClient.login(keytabFile.getAbsolutePath());
return kerberosClient;
} catch (IOException e) {
throw new RuntimeException("Failed to login with kerberos", e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConstants;
import org.apache.gravitino.exceptions.ConnectionFailedException;
import org.apache.gravitino.iceberg.common.ClosableHiveCatalog;
import org.apache.gravitino.iceberg.common.ClosableJdbcCatalog;
import org.apache.gravitino.iceberg.common.IcebergConfig;
import org.apache.gravitino.iceberg.common.authentication.AuthenticationConfig;
import org.apache.hadoop.hdfs.HdfsConfiguration;
Expand Down Expand Up @@ -101,15 +102,28 @@ private static JdbcCatalog loadJdbcCatalog(IcebergConfig icebergConfig) {
} catch (ClassNotFoundException e) {
throw new IllegalArgumentException("Couldn't load jdbc driver " + driverClassName);
}
JdbcCatalog jdbcCatalog =
ClosableJdbcCatalog jdbcCatalog =
new JdbcCatalogWithMetadataLocationSupport(
icebergConfig.get(IcebergConfig.JDBC_INIT_TABLES));

HdfsConfiguration hdfsConfiguration = new HdfsConfiguration();
properties.forEach(hdfsConfiguration::set);
jdbcCatalog.setConf(hdfsConfiguration);
AuthenticationConfig authenticationConfig = new AuthenticationConfig(properties);
try {
jdbcCatalog.initialize(icebergCatalogName, properties);
if (authenticationConfig.isSimpleAuth()) {
jdbcCatalog.setConf(hdfsConfiguration);
jdbcCatalog.setHadoopConf(hdfsConfiguration);
jdbcCatalog.initialize(icebergCatalogName, properties);
} else if (authenticationConfig.isKerberosAuth()) {
hdfsConfiguration.set(HADOOP_SECURITY_AUTHORIZATION, "true");
hdfsConfiguration.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
jdbcCatalog.setConf(hdfsConfiguration);
jdbcCatalog.setHadoopConf(hdfsConfiguration);
jdbcCatalog.initialize(icebergCatalogName, properties);
} else {
throw new UnsupportedOperationException(
"Unsupported authentication method: " + authenticationConfig.getAuthType());
}
} catch (UncheckedSQLException e) {
if (e.getCause() instanceof SQLException
&& e.getCause().getMessage().contains("Access denied")) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,12 +22,13 @@
import com.google.common.base.Preconditions;
import java.util.Map;
import org.apache.commons.lang3.reflect.FieldUtils;
import org.apache.gravitino.iceberg.common.ClosableJdbcCatalog;
import org.apache.gravitino.iceberg.common.cache.SupportsMetadataLocation;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.jdbc.JdbcUtil.SchemaVersion;

// Use Iceberg package to reuse JdbcUtil related classes.
public class JdbcCatalogWithMetadataLocationSupport extends JdbcCatalog
public class JdbcCatalogWithMetadataLocationSupport extends ClosableJdbcCatalog
implements SupportsMetadataLocation {
private String jdbcCatalogName;
private JdbcClientPool jdbcConnections;
Expand Down