Skip to content

Commit

Permalink
Rename metastore client creator classes
Browse files Browse the repository at this point in the history
  • Loading branch information
phd3 authored and dain committed Sep 5, 2022
1 parent aa60366 commit fc11270
Show file tree
Hide file tree
Showing 11 changed files with 83 additions and 83 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -26,16 +26,16 @@
import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory;
import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastoreFactory;
import io.trino.plugin.hive.metastore.thrift.DefaultThriftMetastoreClientFactory;
import io.trino.plugin.hive.metastore.thrift.MetastoreLocator;
import io.trino.plugin.hive.metastore.thrift.IdentityAwareMetastoreClientFactory;
import io.trino.plugin.hive.metastore.thrift.StaticMetastoreConfig;
import io.trino.plugin.hive.metastore.thrift.StaticMetastoreLocator;
import io.trino.plugin.hive.metastore.thrift.StaticTokenAwareMetastoreClientFactory;
import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastore;
import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastoreFactory;
import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreAuthenticationModule;
import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreClientFactory;
import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig;
import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreFactory;
import io.trino.plugin.hive.metastore.thrift.TokenDelegationThriftMetastoreFactory;
import io.trino.plugin.hive.metastore.thrift.TokenAwareMetastoreClientFactory;
import io.trino.spi.security.ConnectorIdentity;
import io.trino.testing.DistributedQueryRunner;
import io.trino.tpch.TpchEntity;
Expand Down Expand Up @@ -97,10 +97,10 @@ private DistributedQueryRunner createQueryRunner(boolean enablePerTransactionHiv
protected void setup(Binder binder)
{
newOptionalBinder(binder, ThriftMetastoreClientFactory.class).setDefault().to(DefaultThriftMetastoreClientFactory.class).in(Scopes.SINGLETON);
binder.bind(MetastoreLocator.class).to(StaticMetastoreLocator.class).in(Scopes.SINGLETON);
binder.bind(TokenAwareMetastoreClientFactory.class).to(StaticTokenAwareMetastoreClientFactory.class).in(Scopes.SINGLETON);
configBinder(binder).bindConfig(StaticMetastoreConfig.class);
configBinder(binder).bindConfig(ThriftMetastoreConfig.class);
binder.bind(TokenDelegationThriftMetastoreFactory.class);
binder.bind(IdentityAwareMetastoreClientFactory.class);
binder.bind(ThriftMetastoreFactory.class).to(ThriftHiveMetastoreFactory.class).in(Scopes.SINGLETON);
newExporter(binder).export(ThriftMetastoreFactory.class)
.as(generator -> generator.generatedNameOf(ThriftHiveMetastore.class));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,20 +34,20 @@
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.MILLISECONDS;

public class TokenDelegationThriftMetastoreFactory
public class IdentityAwareMetastoreClientFactory
{
private final MetastoreLocator clientProvider;
private final TokenAwareMetastoreClientFactory clientProvider;
private final boolean impersonationEnabled;
private final boolean authenticationEnabled;
private final NonEvictableLoadingCache<String, String> delegationTokenCache;

@Inject
public TokenDelegationThriftMetastoreFactory(
MetastoreLocator metastoreLocator,
public IdentityAwareMetastoreClientFactory(
TokenAwareMetastoreClientFactory tokenAwareMetastoreClientFactory,
ThriftMetastoreConfig thriftConfig,
ThriftMetastoreAuthenticationConfig authenticationConfig)
{
this.clientProvider = requireNonNull(metastoreLocator, "metastoreLocator is null");
this.clientProvider = requireNonNull(tokenAwareMetastoreClientFactory, "tokeAwareMetastoreClientFactory is null");
this.impersonationEnabled = thriftConfig.isImpersonationEnabled();
this.authenticationEnabled = authenticationConfig.getAuthenticationType() != ThriftMetastoreAuthenticationType.NONE;

Expand All @@ -64,7 +64,7 @@ private ThriftMetastoreClient createMetastoreClient()
return clientProvider.createMetastoreClient(Optional.empty());
}

public ThriftMetastoreClient createMetastoreClient(Optional<ConnectorIdentity> identity)
public ThriftMetastoreClient createMetastoreClientFor(Optional<ConnectorIdentity> identity)
throws TException
{
if (!impersonationEnabled) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,21 +41,21 @@
import static java.util.concurrent.TimeUnit.NANOSECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;

public class StaticMetastoreLocator
implements MetastoreLocator
public class StaticTokenAwareMetastoreClientFactory
implements TokenAwareMetastoreClientFactory
{
private final List<Backoff> backoffs;
private final ThriftMetastoreClientFactory clientFactory;
private final String metastoreUsername;

@Inject
public StaticMetastoreLocator(StaticMetastoreConfig config, ThriftMetastoreAuthenticationConfig authenticationConfig, ThriftMetastoreClientFactory clientFactory)
public StaticTokenAwareMetastoreClientFactory(StaticMetastoreConfig config, ThriftMetastoreAuthenticationConfig authenticationConfig, ThriftMetastoreClientFactory clientFactory)
{
this(config, authenticationConfig, clientFactory, Ticker.systemTicker());
}

@VisibleForTesting
StaticMetastoreLocator(StaticMetastoreConfig config, ThriftMetastoreAuthenticationConfig authenticationConfig, ThriftMetastoreClientFactory clientFactory, Ticker ticker)
StaticTokenAwareMetastoreClientFactory(StaticMetastoreConfig config, ThriftMetastoreAuthenticationConfig authenticationConfig, ThriftMetastoreClientFactory clientFactory, Ticker ticker)
{
this(config.getMetastoreUris(), config.getMetastoreUsername(), clientFactory, ticker);

Expand All @@ -66,17 +66,17 @@ public StaticMetastoreLocator(StaticMetastoreConfig config, ThriftMetastoreAuthe
authenticationConfig.getAuthenticationType());
}

public StaticMetastoreLocator(List<URI> metastoreUris, @Nullable String metastoreUsername, ThriftMetastoreClientFactory clientFactory)
public StaticTokenAwareMetastoreClientFactory(List<URI> metastoreUris, @Nullable String metastoreUsername, ThriftMetastoreClientFactory clientFactory)
{
this(metastoreUris, metastoreUsername, clientFactory, Ticker.systemTicker());
}

private StaticMetastoreLocator(List<URI> metastoreUris, @Nullable String metastoreUsername, ThriftMetastoreClientFactory clientFactory, Ticker ticker)
private StaticTokenAwareMetastoreClientFactory(List<URI> metastoreUris, @Nullable String metastoreUsername, ThriftMetastoreClientFactory clientFactory, Ticker ticker)
{
requireNonNull(metastoreUris, "metastoreUris is null");
checkArgument(!metastoreUris.isEmpty(), "metastoreUris must specify at least one URI");
this.backoffs = metastoreUris.stream()
.map(StaticMetastoreLocator::checkMetastoreUri)
.map(StaticTokenAwareMetastoreClientFactory::checkMetastoreUri)
.map(uri -> HostAndPort.fromParts(uri.getHost(), uri.getPort()))
.map(address -> new Backoff(address, ticker))
.collect(toImmutableList());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,7 @@ public class ThriftHiveMetastore

private final Optional<ConnectorIdentity> identity;
private final HdfsEnvironment hdfsEnvironment;
private final TokenDelegationThriftMetastoreFactory metastoreFactory;
private final IdentityAwareMetastoreClientFactory metastoreClientFactory;
private final double backoffScaleFactor;
private final Duration minBackoffDelay;
private final Duration maxBackoffDelay;
Expand All @@ -158,7 +158,7 @@ public class ThriftHiveMetastore
public ThriftHiveMetastore(
Optional<ConnectorIdentity> identity,
HdfsEnvironment hdfsEnvironment,
TokenDelegationThriftMetastoreFactory metastoreFactory,
IdentityAwareMetastoreClientFactory metastoreClientFactory,
double backoffScaleFactor,
Duration minBackoffDelay,
Duration maxBackoffDelay,
Expand All @@ -172,7 +172,7 @@ public ThriftHiveMetastore(
{
this.identity = requireNonNull(identity, "identity is null");
this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null");
this.metastoreFactory = requireNonNull(metastoreFactory, "metastoreFactory is null");
this.metastoreClientFactory = requireNonNull(metastoreClientFactory, "metastoreClientFactory is null");
this.backoffScaleFactor = backoffScaleFactor;
this.minBackoffDelay = requireNonNull(minBackoffDelay, "minBackoffDelay is null");
this.maxBackoffDelay = requireNonNull(maxBackoffDelay, "maxBackoffDelay is null");
Expand Down Expand Up @@ -1848,7 +1848,7 @@ private static boolean containsAllPrivilege(Set<PrivilegeGrantInfo> requestedPri
private ThriftMetastoreClient createMetastoreClient()
throws TException
{
return metastoreFactory.createMetastoreClient(identity);
return metastoreClientFactory.createMetastoreClientFor(identity);
}

private RetryDriver retry()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ public class ThriftHiveMetastoreFactory
implements ThriftMetastoreFactory
{
private final HdfsEnvironment hdfsEnvironment;
private final TokenDelegationThriftMetastoreFactory metastoreFactory;
private final IdentityAwareMetastoreClientFactory metastoreClientFactory;
private final double backoffScaleFactor;
private final Duration minBackoffDelay;
private final Duration maxBackoffDelay;
Expand All @@ -46,13 +46,13 @@ public class ThriftHiveMetastoreFactory

@Inject
public ThriftHiveMetastoreFactory(
TokenDelegationThriftMetastoreFactory metastoreFactory,
IdentityAwareMetastoreClientFactory metastoreClientFactory,
@HideDeltaLakeTables boolean hideDeltaLakeTables,
@TranslateHiveViews boolean translateHiveViews,
ThriftMetastoreConfig thriftConfig,
HdfsEnvironment hdfsEnvironment)
{
this.metastoreFactory = requireNonNull(metastoreFactory, "metastoreFactory is null");
this.metastoreClientFactory = requireNonNull(metastoreClientFactory, "metastoreClientFactory is null");
this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null");
this.backoffScaleFactor = thriftConfig.getBackoffScaleFactor();
this.minBackoffDelay = thriftConfig.getMinBackoffDelay();
Expand Down Expand Up @@ -87,7 +87,7 @@ public ThriftMetastore createMetastore(Optional<ConnectorIdentity> identity)
return new ThriftHiveMetastore(
identity,
hdfsEnvironment,
metastoreFactory,
metastoreClientFactory,
backoffScaleFactor,
minBackoffDelay,
maxBackoffDelay,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,8 +31,8 @@ protected void setup(Binder binder)
{
OptionalBinder.newOptionalBinder(binder, ThriftMetastoreClientFactory.class)
.setDefault().to(DefaultThriftMetastoreClientFactory.class).in(Scopes.SINGLETON);
binder.bind(MetastoreLocator.class).to(StaticMetastoreLocator.class).in(Scopes.SINGLETON);
binder.bind(TokenDelegationThriftMetastoreFactory.class);
binder.bind(TokenAwareMetastoreClientFactory.class).to(StaticTokenAwareMetastoreClientFactory.class).in(Scopes.SINGLETON);
binder.bind(IdentityAwareMetastoreClientFactory.class);
configBinder(binder).bindConfig(StaticMetastoreConfig.class);
configBinder(binder).bindConfig(ThriftMetastoreConfig.class);

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

import java.util.Optional;

public interface MetastoreLocator
public interface TokenAwareMetastoreClientFactory
{
/**
* Create a connected {@link ThriftMetastoreClient}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,14 +26,14 @@
import io.trino.plugin.hive.gcs.GoogleGcsConfigurationInitializer;
import io.trino.plugin.hive.gcs.HiveGcsConfig;
import io.trino.plugin.hive.metastore.HiveMetastoreConfig;
import io.trino.plugin.hive.metastore.thrift.MetastoreLocator;
import io.trino.plugin.hive.metastore.thrift.TestingMetastoreLocator;
import io.trino.plugin.hive.metastore.thrift.IdentityAwareMetastoreClientFactory;
import io.trino.plugin.hive.metastore.thrift.TestingTokenAwareMetastoreClientFactory;
import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastoreFactory;
import io.trino.plugin.hive.metastore.thrift.ThriftMetastore;
import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreAuthenticationConfig;
import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreClient;
import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig;
import io.trino.plugin.hive.metastore.thrift.TokenDelegationThriftMetastoreFactory;
import io.trino.plugin.hive.metastore.thrift.TokenAwareMetastoreClientFactory;
import io.trino.plugin.hive.s3.HiveS3Config;
import io.trino.plugin.hive.s3.TrinoS3ConfigurationInitializer;

Expand All @@ -57,7 +57,7 @@ public final class TestingThriftHiveMetastoreBuilder
new HdfsConfig(),
new NoHdfsAuthentication());

private MetastoreLocator metastoreLocator;
private TokenAwareMetastoreClientFactory tokenAwareMetastoreClientFactory;
private HiveConfig hiveConfig = new HiveConfig();
private ThriftMetastoreConfig thriftMetastoreConfig = new ThriftMetastoreConfig();
private HdfsEnvironment hdfsEnvironment = HDFS_ENVIRONMENT;
Expand All @@ -73,24 +73,24 @@ public TestingThriftHiveMetastoreBuilder metastoreClient(HostAndPort address, Du
{
requireNonNull(address, "address is null");
requireNonNull(timeout, "timeout is null");
checkState(metastoreLocator == null, "Metastore client already set");
metastoreLocator = new TestingMetastoreLocator(HiveTestUtils.SOCKS_PROXY, address, timeout);
checkState(tokenAwareMetastoreClientFactory == null, "Metastore client already set");
tokenAwareMetastoreClientFactory = new TestingTokenAwareMetastoreClientFactory(HiveTestUtils.SOCKS_PROXY, address, timeout);
return this;
}

public TestingThriftHiveMetastoreBuilder metastoreClient(HostAndPort address)
{
requireNonNull(address, "address is null");
checkState(metastoreLocator == null, "Metastore client already set");
metastoreLocator = new TestingMetastoreLocator(HiveTestUtils.SOCKS_PROXY, address);
checkState(tokenAwareMetastoreClientFactory == null, "Metastore client already set");
tokenAwareMetastoreClientFactory = new TestingTokenAwareMetastoreClientFactory(HiveTestUtils.SOCKS_PROXY, address);
return this;
}

public TestingThriftHiveMetastoreBuilder metastoreClient(ThriftMetastoreClient client)
{
requireNonNull(client, "client is null");
checkState(metastoreLocator == null, "Metastore client already set");
metastoreLocator = token -> client;
checkState(tokenAwareMetastoreClientFactory == null, "Metastore client already set");
tokenAwareMetastoreClientFactory = token -> client;
return this;
}

Expand All @@ -114,10 +114,10 @@ public TestingThriftHiveMetastoreBuilder hdfsEnvironment(HdfsEnvironment hdfsEnv

public ThriftMetastore build()
{
checkState(metastoreLocator != null, "metastore client not set");
checkState(tokenAwareMetastoreClientFactory != null, "metastore client not set");
ThriftHiveMetastoreFactory metastoreFactory = new ThriftHiveMetastoreFactory(
new TokenDelegationThriftMetastoreFactory(
metastoreLocator,
new IdentityAwareMetastoreClientFactory(
tokenAwareMetastoreClientFactory,
thriftMetastoreConfig,
new ThriftMetastoreAuthenticationConfig()),
new HiveMetastoreConfig().isHideDeltaLakeTables(),
Expand Down
Loading

0 comments on commit fc11270

Please sign in to comment.