Skip to content
Closed
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
Expand Up @@ -33,6 +33,7 @@
import org.apache.doris.datasource.ExternalDatabase;
import org.apache.doris.datasource.operations.ExternalMetadataOps;

import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.catalog.Catalog;
Expand All @@ -42,6 +43,7 @@
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;

import java.lang.reflect.UndeclaredThrowableException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -74,7 +76,16 @@ public void close() {

@Override
public boolean tableExist(String dbName, String tblName) {
return catalog.tableExists(TableIdentifier.of(dbName, tblName));
try {
return catalog.tableExists(TableIdentifier.of(dbName, tblName));
} catch (UndeclaredThrowableException e) {
// avoid to miss exception when get table reflect call
if (e.getCause() instanceof NoSuchObjectException) {
return false;
} else {
throw e;
}
}
}

public boolean databaseExist(String dbName) {
Expand All @@ -83,7 +94,7 @@ public boolean databaseExist(String dbName) {

public List<String> listDatabaseNames() {
return nsCatalog.listNamespaces().stream()
.map(e -> e.toString())
.map(Namespace::toString)
.collect(Collectors.toList());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,12 +90,7 @@ protected FileIO initializeFileIO(Map<String, String> properties, Configuration
* so HadoopFileIO is used in the superclass by default
* we can add better implementations to derived class just like the implementation in DLFCatalog.
*/
FileIO io;
try {
io = new IcebergHadoopFileIO(hadoopConf, this.fs.rawFileSystem());
} catch (IOException e) {
throw new RuntimeException(e);
}
FileIO io = new IcebergHadoopFileIO(hadoopConf, this.fs);
io.initialize(properties);
return io;
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,11 @@

package org.apache.doris.datasource.iceberg.hadoop;

import org.apache.doris.fs.remote.dfs.DFSFileSystem;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.iceberg.exceptions.RuntimeIOException;
import org.apache.iceberg.hadoop.HadoopFileIO;
import org.apache.iceberg.io.BulkDeletionFailureException;
import org.apache.iceberg.io.InputFile;
Expand All @@ -30,37 +31,44 @@

public class IcebergHadoopFileIO extends HadoopFileIO {

private FileSystem fs;
private Configuration hadoopConf;
private final DFSFileSystem fs;
private final Configuration hadoopConf;

public IcebergHadoopFileIO(Configuration hadoopConf, FileSystem fs) {
public IcebergHadoopFileIO(Configuration hadoopConf, DFSFileSystem fs) {
this.hadoopConf = hadoopConf;
this.fs = fs;
}

@Override
public InputFile newInputFile(String path) {
return new IcebergHadoopInputFile(this.fs, path, this.hadoopConf);
return new IcebergHadoopInputFile(getFs(), path, this.hadoopConf);
}

private FileSystem getFs() {
try {
return this.fs.rawFileSystem();
} catch (IOException e) {
throw new RuntimeException(e);
}
}

@Override
public InputFile newInputFile(String path, long length) {
return new IcebergHadoopInputFile(this.fs, path, length, this.hadoopConf);
return new IcebergHadoopInputFile(getFs(), path, length, this.hadoopConf);
}

@Override
public OutputFile newOutputFile(String path) {
return new IcebergHadoopOutputFile(this.fs, new Path(path), this.hadoopConf);
return new IcebergHadoopOutputFile(getFs(), new Path(path), this.hadoopConf);
}

@Override
public void deleteFile(String path) {
Path toDelete = new Path(path);
try {
fs.delete(toDelete, false);
} catch (IOException var5) {
IOException e = var5;
throw new RuntimeIOException(e, "Failed to delete file: %s", path);
} catch (IOException e) {
throw new RuntimeException("Failed to delete file: " + path, e);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,27 +17,32 @@

package org.apache.doris.datasource.iceberg.hive;

import org.apache.doris.common.security.authentication.AuthenticationConfig;
import org.apache.doris.common.security.authentication.HadoopAuthenticator;

import com.github.benmanes.caffeine.cache.Cache;
import com.github.benmanes.caffeine.cache.Caffeine;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.metastore.IMetaStoreClient;
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.ClientPool;
import org.apache.iceberg.hive.HiveClientPool;
import org.apache.iceberg.util.PropertyUtil;
import shade.doris.hive.org.apache.thrift.TException;

import java.io.IOException;
import java.lang.reflect.UndeclaredThrowableException;
import java.util.Map;
import java.util.concurrent.TimeUnit;

public class HCachedClientPool implements ClientPool<IMetaStoreClient, TException> {

private static volatile Cache<String, HiveClientPool> clientPoolCache;
private static volatile Cache<String, HClientPool> clientPoolCache;
private static final Object clientPoolCacheLock = new Object();
private final String catalogName;
private final Configuration conf;
private final int clientPoolSize;
private final long evictionInterval;
private final HadoopAuthenticator authenticator;

public HCachedClientPool(String catalogName, Configuration conf, Map<String, String> properties) {
this.catalogName = catalogName;
Expand All @@ -59,25 +64,45 @@ public HCachedClientPool(String catalogName, Configuration conf, Map<String, Str
clientPoolCache =
Caffeine.newBuilder()
.expireAfterAccess(evictionInterval, TimeUnit.MILLISECONDS)
.removalListener((key, value, cause) -> ((HiveClientPool) value).close())
.removalListener((key, value, cause) -> ((HClientPool) value).close())
.build();
}
}
}
AuthenticationConfig authConfig = AuthenticationConfig.getKerberosConfig(conf);
authenticator = HadoopAuthenticator.getHadoopAuthenticator(authConfig);
}

protected HiveClientPool clientPool() {
return clientPoolCache.get(this.catalogName, (k) -> new HiveClientPool(this.clientPoolSize, this.conf));
protected HClientPool clientPool() {
return clientPoolCache.get(this.catalogName, (k) -> new HClientPool(this.clientPoolSize, this.conf));
}

@Override
public <R> R run(Action<R, IMetaStoreClient, TException> action) throws TException, InterruptedException {
return clientPool().run(action);
try {
return authenticator.doAs(() -> clientPool().run(action));
} catch (IOException e) {
throw new TException(e);
} catch (UndeclaredThrowableException e) {
if (e.getCause() instanceof TException) {
throw (TException) e.getCause();
}
throw e;
}
}

@Override
public <R> R run(Action<R, IMetaStoreClient, TException> action, boolean retry)
throws TException, InterruptedException {
return clientPool().run(action, retry);
try {
return authenticator.doAs(() -> clientPool().run(action, retry));
} catch (IOException e) {
throw new TException(e);
} catch (UndeclaredThrowableException e) {
if (e.getCause() instanceof TException) {
throw (TException) e.getCause();
}
throw e;
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
// 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.doris.datasource.iceberg.hive;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
import org.apache.hadoop.hive.metastore.IMetaStoreClient;
import org.apache.hadoop.hive.metastore.RetryingMetaStoreClient;
import org.apache.hadoop.hive.metastore.api.MetaException;
import org.apache.iceberg.ClientPoolImpl;
import org.apache.iceberg.hive.RuntimeMetaException;
import shade.doris.hive.org.apache.thrift.TException;
import shade.doris.hive.org.apache.thrift.transport.TTransportException;

public class HClientPool extends ClientPoolImpl<IMetaStoreClient, TException> {
private final HiveConf hiveConf;

public HClientPool(int poolSize, Configuration conf) {
super(poolSize, TTransportException.class, false);
this.hiveConf = new HiveConf(conf, org.apache.iceberg.hive.HiveClientPool.class);
this.hiveConf.addResource(conf);
}

protected IMetaStoreClient newClient() {
try {
try {
return RetryingMetaStoreClient.getProxy(hiveConf, t -> null, HiveMetaStoreClient.class.getName());
} catch (RuntimeException e) {
if (e.getCause() instanceof MetaException) {
throw (MetaException) e.getCause();
} else {
throw e;
}
}
} catch (MetaException e) {
throw new RuntimeMetaException(e, "Failed to connect to Hive Metastore");
} catch (Throwable t) {
if (t.getMessage().contains("Another instance of Derby may have already booted")) {
throw new RuntimeMetaException(t,
"Embedded Derby supports only one client at a time."
+ "To fix this, use a metastore that supports multiple clients.");
} else {
throw new RuntimeMetaException(t, "Failed to connect to Hive Metastore");
}
}
}

protected IMetaStoreClient reconnect(IMetaStoreClient client) {
try {
client.close();
client.reconnect();
return client;
} catch (MetaException var3) {
MetaException e = var3;
throw new RuntimeMetaException(e, "Failed to reconnect to Hive Metastore", new Object[0]);
}
}

protected boolean isConnectionException(Exception e) {
return super.isConnectionException(e)
|| e instanceof MetaException
&& e.getMessage().contains("Got exception: org.apache.thrift.transport.TTransportException");
}

protected void close(IMetaStoreClient client) {
client.close();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@

package org.apache.doris.datasource.iceberg.hive;

import org.apache.doris.datasource.iceberg.hadoop.IcebergHadoopFileIO;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
Expand Down Expand Up @@ -57,17 +55,7 @@ protected FileIO initializeFileIO(Map<String, String> properties) {
* so HadoopFileIO is used in the superclass by default
* we can add better implementations to derived class just like the implementation in DLFCatalog.
*/
FileIO io;
try {
FileSystem fs = getFileSystem();
if (fs == null) {
io = new HadoopFileIO(getConf());
} else {
io = new IcebergHadoopFileIO(getConf(), getFileSystem());
}
} catch (IOException e) {
throw new RuntimeException(e);
}
FileIO io = new HadoopFileIO(getConf());
io.initialize(properties);
return io;
} else {
Expand Down
Loading