Skip to content

Commit

Permalink
Remove usages of MethodHandleUtil.methodHandle()
Browse files Browse the repository at this point in the history
  • Loading branch information
electrum committed Aug 9, 2022
1 parent 874c042 commit e4e0d19
Show file tree
Hide file tree
Showing 15 changed files with 138 additions and 101 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -49,14 +49,14 @@

import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static io.trino.SessionTestUtils.TEST_SESSION;
import static io.trino.spi.block.MethodHandleUtil.methodHandle;
import static io.trino.sql.planner.TestingPlannerContext.plannerContextBuilder;
import static io.trino.testing.TestingAccessControlManager.TestingPrivilegeType.INSERT_TABLE;
import static io.trino.testing.TestingAccessControlManager.privilege;
import static io.trino.testing.TestingEventListenerManager.emptyEventListenerManager;
import static io.trino.testing.TestingHandles.TEST_CATALOG_HANDLE;
import static io.trino.testing.TestingHandles.TEST_CATALOG_NAME;
import static io.trino.testing.TestingSession.testSessionBuilder;
import static io.trino.util.Reflection.methodHandle;
import static java.util.concurrent.Executors.newCachedThreadPool;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,15 +24,22 @@
import java.lang.invoke.MethodHandle;
import java.util.Optional;

import static io.trino.spi.block.MethodHandleUtil.methodHandle;
import static java.lang.invoke.MethodHandles.lookup;
import static java.util.Objects.requireNonNull;

public class FlushJdbcMetadataCacheProcedure
implements Provider<Procedure>
{
private static final MethodHandle FLUSH_JDBC_METADATA_CACHE = methodHandle(
FlushJdbcMetadataCacheProcedure.class,
"flushMetadataCache");
private static final MethodHandle FLUSH_JDBC_METADATA_CACHE;

static {
try {
FLUSH_JDBC_METADATA_CACHE = lookup().unreflect(FlushJdbcMetadataCacheProcedure.class.getMethod("flushMetadataCache"));
}
catch (ReflectiveOperationException e) {
throw new AssertionError(e);
}
}

private final CachingJdbcClient cachingJdbcClient;
private final Optional<CachingIdentifierMapping> cachingIdentifierMapping;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,22 +31,24 @@

import static io.trino.plugin.deltalake.procedure.Procedures.checkProcedureArgument;
import static io.trino.spi.StandardErrorCode.INVALID_PROCEDURE_ARGUMENT;
import static io.trino.spi.block.MethodHandleUtil.methodHandle;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static java.lang.String.format;
import static java.lang.invoke.MethodHandles.lookup;
import static java.util.Objects.requireNonNull;

public class DropExtendedStatsProcedure
implements Provider<Procedure>
{
private static final MethodHandle PROCEDURE_METHOD = methodHandle(
DropExtendedStatsProcedure.class,
"dropStats",
ConnectorSession.class,
ConnectorAccessControl.class,
// Schema name and table name
String.class,
String.class);
private static final MethodHandle PROCEDURE_METHOD;

static {
try {
PROCEDURE_METHOD = lookup().unreflect(DropExtendedStatsProcedure.class.getMethod("dropStats", ConnectorSession.class, ConnectorAccessControl.class, String.class, String.class));
}
catch (ReflectiveOperationException e) {
throw new AssertionError(e);
}
}

private final DeltaLakeMetadataFactory metadataFactory;
private final ExtendedStatisticsAccess statsAccess;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,9 +59,9 @@
import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.TRANSACTION_LOG_DIRECTORY;
import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.getTransactionLogDir;
import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
import static io.trino.spi.block.MethodHandleUtil.methodHandle;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static java.lang.String.format;
import static java.lang.invoke.MethodHandles.lookup;
import static java.util.Comparator.naturalOrder;
import static java.util.Objects.requireNonNull;

Expand All @@ -70,14 +70,16 @@ public class VacuumProcedure
{
private static final Logger log = Logger.get(VacuumProcedure.class);

private static final MethodHandle VACUUM = methodHandle(
VacuumProcedure.class,
"vacuum",
ConnectorSession.class,
ConnectorAccessControl.class,
String.class,
String.class,
String.class);
private static final MethodHandle VACUUM;

static {
try {
VACUUM = lookup().unreflect(VacuumProcedure.class.getMethod("vacuum", ConnectorSession.class, ConnectorAccessControl.class, String.class, String.class, String.class));
}
catch (ReflectiveOperationException e) {
throw new AssertionError(e);
}
}

private final CatalogName catalogName;
private final HdfsEnvironment hdfsEnvironment;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,9 +29,9 @@
import java.util.Optional;

import static com.google.common.base.Preconditions.checkState;
import static io.trino.spi.block.MethodHandleUtil.methodHandle;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static java.lang.String.format;
import static java.lang.invoke.MethodHandles.lookup;
import static java.util.Locale.ENGLISH;
import static java.util.Objects.requireNonNull;

Expand All @@ -56,14 +56,17 @@ public class FlushHiveMetastoreCacheProcedure
PARAM_PARTITION_COLUMN.toLowerCase(ENGLISH),
PARAM_PARTITION_VALUE.toLowerCase(ENGLISH));

private static final MethodHandle FLUSH_HIVE_METASTORE_CACHE = methodHandle(
FlushHiveMetastoreCacheProcedure.class,
"flushMetadataCache",
String.class,
String.class,
String.class,
List.class,
List.class);
private static final MethodHandle FLUSH_HIVE_METASTORE_CACHE;

static {
try {
FLUSH_HIVE_METASTORE_CACHE = lookup().unreflect(FlushHiveMetastoreCacheProcedure.class.getMethod("flushMetadataCache", String.class, String.class, String.class, List.class, List.class));
}
catch (ReflectiveOperationException e) {
throw new AssertionError(e);
}
}

private static final String FAKE_PARAM_DEFAULT_VALUE = "procedure should only be invoked with named parameters";

private final Optional<CachingHiveMetastore> cachingHiveMetastore;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,15 +23,22 @@
import java.io.IOException;
import java.lang.invoke.MethodHandle;

import static io.trino.spi.block.MethodHandleUtil.methodHandle;
import static java.lang.invoke.MethodHandles.lookup;
import static java.util.Objects.requireNonNull;

public class WriteHiveMetastoreRecordingProcedure
implements Provider<Procedure>
{
private static final MethodHandle WRITE_HIVE_METASTORE_RECORDING = methodHandle(
WriteHiveMetastoreRecordingProcedure.class,
"writeHiveMetastoreRecording");
private static final MethodHandle WRITE_HIVE_METASTORE_RECORDING;

static {
try {
WRITE_HIVE_METASTORE_RECORDING = lookup().unreflect(WriteHiveMetastoreRecordingProcedure.class.getMethod("writeHiveMetastoreRecording"));
}
catch (ReflectiveOperationException e) {
throw new AssertionError(e);
}
}

private final RateLimiter rateLimiter = RateLimiter.create(0.2);
private final HiveMetastoreRecording hiveMetastoreRecording;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,24 +48,25 @@
import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.trino.spi.StandardErrorCode.ALREADY_EXISTS;
import static io.trino.spi.StandardErrorCode.INVALID_PROCEDURE_ARGUMENT;
import static io.trino.spi.block.MethodHandleUtil.methodHandle;
import static io.trino.spi.connector.RetryMode.NO_RETRIES;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static java.lang.String.format;
import static java.lang.invoke.MethodHandles.lookup;
import static java.util.Objects.requireNonNull;

public class CreateEmptyPartitionProcedure
implements Provider<Procedure>
{
private static final MethodHandle CREATE_EMPTY_PARTITION = methodHandle(
CreateEmptyPartitionProcedure.class,
"createEmptyPartition",
ConnectorSession.class,
ConnectorAccessControl.class,
String.class,
String.class,
List.class,
List.class);
private static final MethodHandle CREATE_EMPTY_PARTITION;

static {
try {
CREATE_EMPTY_PARTITION = lookup().unreflect(CreateEmptyPartitionProcedure.class.getMethod("createEmptyPartition", ConnectorSession.class, ConnectorAccessControl.class, String.class, String.class, List.class, List.class));
}
catch (ReflectiveOperationException e) {
throw new AssertionError(e);
}
}

private final TransactionalMetadataFactory hiveMetadataFactory;
private final LocationService locationService;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,9 +41,9 @@
import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.trino.plugin.hive.acid.AcidTransaction.NO_ACID_TRANSACTION;
import static io.trino.spi.StandardErrorCode.INVALID_PROCEDURE_ARGUMENT;
import static io.trino.spi.block.MethodHandleUtil.methodHandle;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static java.lang.String.format;
import static java.lang.invoke.MethodHandles.lookup;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.hive.metastore.utils.FileUtils.makePartName;

Expand All @@ -55,14 +55,16 @@
public class DropStatsProcedure
implements Provider<Procedure>
{
private static final MethodHandle DROP_STATS = methodHandle(
DropStatsProcedure.class,
"dropStats",
ConnectorSession.class,
ConnectorAccessControl.class,
String.class,
String.class,
List.class);
private static final MethodHandle DROP_STATS;

static {
try {
DROP_STATS = lookup().unreflect(DropStatsProcedure.class.getMethod("dropStats", ConnectorSession.class, ConnectorAccessControl.class, String.class, String.class, List.class));
}
catch (ReflectiveOperationException e) {
throw new AssertionError(e);
}
}

private final TransactionalMetadataFactory hiveMetadataFactory;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,24 +48,24 @@
import static io.trino.spi.StandardErrorCode.ALREADY_EXISTS;
import static io.trino.spi.StandardErrorCode.INVALID_PROCEDURE_ARGUMENT;
import static io.trino.spi.StandardErrorCode.PERMISSION_DENIED;
import static io.trino.spi.block.MethodHandleUtil.methodHandle;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static java.lang.String.format;
import static java.lang.invoke.MethodHandles.lookup;
import static java.util.Objects.requireNonNull;

public class RegisterPartitionProcedure
implements Provider<Procedure>
{
private static final MethodHandle REGISTER_PARTITION = methodHandle(
RegisterPartitionProcedure.class,
"registerPartition",
ConnectorSession.class,
ConnectorAccessControl.class,
String.class,
String.class,
List.class,
List.class,
String.class);
private static final MethodHandle REGISTER_PARTITION;

static {
try {
REGISTER_PARTITION = lookup().unreflect(RegisterPartitionProcedure.class.getMethod("registerPartition", ConnectorSession.class, ConnectorAccessControl.class, String.class, String.class, List.class, List.class, String.class));
}
catch (ReflectiveOperationException e) {
throw new AssertionError(e);
}
}

private final boolean allowRegisterPartition;
private final TransactionalMetadataFactory hiveMetadataFactory;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,10 +53,10 @@
import static io.trino.plugin.hive.HiveMetadata.PRESTO_QUERY_ID_NAME;
import static io.trino.plugin.hive.HivePartitionManager.extractPartitionValues;
import static io.trino.spi.StandardErrorCode.INVALID_PROCEDURE_ARGUMENT;
import static io.trino.spi.block.MethodHandleUtil.methodHandle;
import static io.trino.spi.type.BooleanType.BOOLEAN;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static java.lang.Boolean.TRUE;
import static java.lang.invoke.MethodHandles.lookup;
import static java.util.Locale.ENGLISH;
import static java.util.Objects.requireNonNull;

Expand All @@ -70,15 +70,16 @@ public enum SyncMode

private static final int BATCH_GET_PARTITIONS_BY_NAMES_MAX_PAGE_SIZE = 1000;

private static final MethodHandle SYNC_PARTITION_METADATA = methodHandle(
SyncPartitionMetadataProcedure.class,
"syncPartitionMetadata",
ConnectorSession.class,
ConnectorAccessControl.class,
String.class,
String.class,
String.class,
boolean.class);
private static final MethodHandle SYNC_PARTITION_METADATA;

static {
try {
SYNC_PARTITION_METADATA = lookup().unreflect(SyncPartitionMetadataProcedure.class.getMethod("syncPartitionMetadata", ConnectorSession.class, ConnectorAccessControl.class, String.class, String.class, String.class, boolean.class));
}
catch (ReflectiveOperationException e) {
throw new AssertionError(e);
}
}

private final TransactionalMetadataFactory hiveMetadataFactory;
private final HdfsEnvironment hdfsEnvironment;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,23 +37,24 @@
import static io.trino.plugin.hive.procedure.Procedures.checkIsPartitionedTable;
import static io.trino.plugin.hive.procedure.Procedures.checkPartitionColumns;
import static io.trino.spi.StandardErrorCode.NOT_FOUND;
import static io.trino.spi.block.MethodHandleUtil.methodHandle;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static java.lang.String.format;
import static java.lang.invoke.MethodHandles.lookup;
import static java.util.Objects.requireNonNull;

public class UnregisterPartitionProcedure
implements Provider<Procedure>
{
private static final MethodHandle UNREGISTER_PARTITION = methodHandle(
UnregisterPartitionProcedure.class,
"unregisterPartition",
ConnectorSession.class,
ConnectorAccessControl.class,
String.class,
String.class,
List.class,
List.class);
private static final MethodHandle UNREGISTER_PARTITION;

static {
try {
UNREGISTER_PARTITION = lookup().unreflect(UnregisterPartitionProcedure.class.getMethod("unregisterPartition", ConnectorSession.class, ConnectorAccessControl.class, String.class, String.class, List.class, List.class));
}
catch (ReflectiveOperationException e) {
throw new AssertionError(e);
}
}

private final TransactionalMetadataFactory hiveMetadataFactory;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,21 +26,24 @@

import java.lang.invoke.MethodHandle;

import static io.trino.spi.block.MethodHandleUtil.methodHandle;
import static io.trino.spi.type.BigintType.BIGINT;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static java.lang.invoke.MethodHandles.lookup;
import static java.util.Objects.requireNonNull;

public class RollbackToSnapshotProcedure
implements Provider<Procedure>
{
private static final MethodHandle ROLLBACK_TO_SNAPSHOT = methodHandle(
RollbackToSnapshotProcedure.class,
"rollbackToSnapshot",
ConnectorSession.class,
String.class,
String.class,
Long.class);
private static final MethodHandle ROLLBACK_TO_SNAPSHOT;

static {
try {
ROLLBACK_TO_SNAPSHOT = lookup().unreflect(RollbackToSnapshotProcedure.class.getMethod("rollbackToSnapshot", ConnectorSession.class, String.class, String.class, Long.class));
}
catch (ReflectiveOperationException e) {
throw new AssertionError(e);
}
}

private final TrinoCatalogFactory catalogFactory;
private final ClassLoader classLoader;
Expand Down
Loading

0 comments on commit e4e0d19

Please sign in to comment.