From 496e98ca25fb243ab25731d188c2fd56eba3b3e9 Mon Sep 17 00:00:00 2001 From: Nikhil Collooru Date: Wed, 10 Jan 2024 12:18:20 -0800 Subject: [PATCH] Add WarningCollector to MetastoreContext --- .../facebook/presto/delta/DeltaMetadata.java | 3 +- .../hive/metastore/CachingHiveMetastore.java | 2 +- .../hive/metastore/MetastoreContext.java | 45 +++++++++++++--- .../SemiTransactionalHiveMetastore.java | 14 +++-- .../thrift/MockHiveMetastoreClient.java | 3 +- .../hive/CreateEmptyPartitionProcedure.java | 2 +- .../facebook/presto/hive/HiveMetadata.java | 2 +- .../presto/hive/HivePageSinkProvider.java | 2 +- .../presto/hive/HivePartitionManager.java | 8 +-- .../presto/hive/HiveSplitManager.java | 4 +- .../facebook/presto/hive/HiveWriteUtils.java | 2 +- .../hive/SyncPartitionMetadataProcedure.java | 3 +- .../presto/hive/rule/HiveFilterPushdown.java | 2 +- .../hive/security/LegacyAccessControl.java | 2 +- .../security/SqlStandardAccessControl.java | 52 +++++++++---------- .../MetastoreHiveStatisticsProvider.java | 2 +- .../presto/hive/AbstractTestHiveClient.java | 50 +++++++++--------- .../facebook/presto/hive/HiveQueryRunner.java | 3 +- .../hive/TestHiveCommitHandleOutput.java | 3 +- .../presto/hive/TestHiveLogicalPlanner.java | 6 +-- ...estHiveMaterializedViewLogicalPlanner.java | 2 +- .../hive/TestHiveMaterializedViewUtils.java | 3 +- .../hive/geospatial/TestSpatialJoins.java | 3 +- .../hive/hudi/HudiTestingDataGenerator.java | 3 +- .../glue/TestHiveClientGlueMetastore.java | 3 +- .../ranger/TestRangerBasedAccessControl.java | 3 +- .../facebook/presto/hudi/HudiMetadata.java | 2 +- .../presto/iceberg/IcebergHiveMetadata.java | 2 +- .../facebook/presto/iceberg/IcebergUtil.java | 2 +- .../iceberg/TestIcebergRegisterProcedure.java | 2 +- .../java/com/facebook/presto/Session.java | 2 +- .../presto/security/AccessControlUtils.java | 7 ++- .../security/TestAccessControlManager.java | 17 +++--- .../TestFileBasedSystemAccessControl.java | 3 +- .../security/TestFileBasedAccessControl.java | 3 +- .../presto/spark/PrestoSparkQueryRunner.java | 3 +- .../spi/security/AccessControlContext.java | 10 +++- 37 files changed, 171 insertions(+), 109 deletions(-) diff --git a/presto-delta/src/main/java/com/facebook/presto/delta/DeltaMetadata.java b/presto-delta/src/main/java/com/facebook/presto/delta/DeltaMetadata.java index bde01147f0c16..043b0a4657a63 100644 --- a/presto-delta/src/main/java/com/facebook/presto/delta/DeltaMetadata.java +++ b/presto-delta/src/main/java/com/facebook/presto/delta/DeltaMetadata.java @@ -370,7 +370,8 @@ private MetastoreContext metastoreContext(ConnectorSession session) session.getSource(), Optional.empty(), false, - DEFAULT_COLUMN_CONVERTER_PROVIDER); + DEFAULT_COLUMN_CONVERTER_PROVIDER, + session.getWarningCollector()); } private void checkConnectorId(DeltaTableHandle tableHandle) diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/CachingHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/CachingHiveMetastore.java index d6fe6f5f2cf1a..518336ef447a7 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/CachingHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/CachingHiveMetastore.java @@ -1096,7 +1096,7 @@ public String toString() private KeyAndContext getCachingKey(MetastoreContext context, T key) { - MetastoreContext metastoreContext = metastoreImpersonationEnabled ? new MetastoreContext(context.getUsername(), context.getQueryId(), context.getClientInfo(), context.getSource(), true, context.getMetastoreHeaders(), context.isUserDefinedTypeEncodingEnabled(), context.getColumnConverterProvider()) : context; + MetastoreContext metastoreContext = metastoreImpersonationEnabled ? new MetastoreContext(context.getUsername(), context.getQueryId(), context.getClientInfo(), context.getSource(), true, context.getMetastoreHeaders(), context.isUserDefinedTypeEncodingEnabled(), context.getColumnConverterProvider(), context.getWarningCollector()) : context; return new KeyAndContext<>(metastoreContext, key); } diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/MetastoreContext.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/MetastoreContext.java index 7c33ac6556677..508359853d0e9 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/MetastoreContext.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/MetastoreContext.java @@ -16,6 +16,7 @@ import com.facebook.presto.hive.ColumnConverter; import com.facebook.presto.hive.ColumnConverterProvider; import com.facebook.presto.hive.HiveColumnConverterProvider; +import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.security.ConnectorIdentity; import java.util.Objects; @@ -38,18 +39,44 @@ public class MetastoreContext // a new MetastoreContext from either an existing MetastoreContext or callers // that only have a handle to the provider (e.g. SemiTransactionalHiveMetastore) private final ColumnConverterProvider columnConverterProvider; - - public MetastoreContext(ConnectorIdentity identity, String queryId, Optional clientInfo, Optional source, Optional metastoreHeaders, boolean userDefinedTypeEncodingEnabled, ColumnConverterProvider columnConverterProvider) + private final WarningCollector warningCollector; + + public MetastoreContext( + ConnectorIdentity identity, + String queryId, + Optional clientInfo, + Optional source, + Optional metastoreHeaders, + boolean userDefinedTypeEncodingEnabled, + ColumnConverterProvider columnConverterProvider, + WarningCollector warningCollector) { - this(requireNonNull(identity, "identity is null").getUser(), queryId, clientInfo, source, metastoreHeaders, userDefinedTypeEncodingEnabled, columnConverterProvider); + this(requireNonNull(identity, "identity is null").getUser(), queryId, clientInfo, source, metastoreHeaders, userDefinedTypeEncodingEnabled, columnConverterProvider, warningCollector); } - public MetastoreContext(String username, String queryId, Optional clientInfo, Optional source, Optional metastoreHeaders, boolean userDefinedTypeEncodingEnabled, ColumnConverterProvider columnConverterProvider) + public MetastoreContext( + String username, + String queryId, + Optional clientInfo, + Optional source, + Optional metastoreHeaders, + boolean userDefinedTypeEncodingEnabled, + ColumnConverterProvider columnConverterProvider, + WarningCollector warningCollector) { - this(username, queryId, clientInfo, source, false, metastoreHeaders, userDefinedTypeEncodingEnabled, columnConverterProvider); + this(username, queryId, clientInfo, source, false, metastoreHeaders, userDefinedTypeEncodingEnabled, columnConverterProvider, warningCollector); } - public MetastoreContext(String username, String queryId, Optional clientInfo, Optional source, boolean impersonationEnabled, Optional metastoreHeaders, boolean userDefinedTypeEncodingEnabled, ColumnConverterProvider columnConverterProvider) + public MetastoreContext( + String username, + String queryId, + Optional clientInfo, + Optional source, + boolean impersonationEnabled, + Optional metastoreHeaders, + boolean userDefinedTypeEncodingEnabled, + ColumnConverterProvider columnConverterProvider, + WarningCollector warningCollector) { this.username = requireNonNull(username, "username is null"); this.queryId = requireNonNull(queryId, "queryId is null"); @@ -65,6 +92,7 @@ public MetastoreContext(String username, String queryId, Optional client else { this.columnConverter = requireNonNull(HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER, "columnConverter is null"); } + this.warningCollector = requireNonNull(warningCollector, "warningCollector is null"); } public ColumnConverterProvider getColumnConverterProvider() @@ -112,6 +140,11 @@ public Optional getMetastoreHeaders() return metastoreHeaders; } + public WarningCollector getWarningCollector() + { + return warningCollector; + } + @Override public String toString() { diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java index d54a8ad36cbd4..6ff9bd210c09e 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java @@ -97,6 +97,7 @@ import static com.facebook.presto.spi.StandardErrorCode.ALREADY_EXISTS; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static com.facebook.presto.spi.StandardErrorCode.TRANSACTION_CONFLICT; +import static com.facebook.presto.spi.WarningCollector.NOOP; import static com.facebook.presto.spi.security.PrincipalType.USER; import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkArgument; @@ -512,7 +513,7 @@ public synchronized void finishInsertIntoExistingTable( setShared(); SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); Action oldTableAction = tableActions.get(schemaTableName); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), columnConverterProvider); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), columnConverterProvider, session.getWarningCollector()); if (oldTableAction == null || oldTableAction.getData().getTable().getTableType().equals(TEMPORARY_TABLE)) { Table table = getTable(metastoreContext, databaseName, tableName) .orElseThrow(() -> new TableNotFoundException(schemaTableName)); @@ -558,7 +559,8 @@ public synchronized void truncateUnpartitionedTable(ConnectorSession session, St session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), - columnConverterProvider), + columnConverterProvider, + session.getWarningCollector()), databaseName, tableName); SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); @@ -902,7 +904,7 @@ public synchronized void finishInsertIntoExistingPartition( SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); Map, Action> partitionActionsOfTable = partitionActions.computeIfAbsent(schemaTableName, k -> new HashMap<>()); Action oldPartitionAction = partitionActionsOfTable.get(partitionValues); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), columnConverterProvider); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), columnConverterProvider, session.getWarningCollector()); if (oldPartitionAction == null) { Partition partition = delegate.getPartition(metastoreContext, databaseName, tableName, partitionValues) @@ -1129,7 +1131,8 @@ private ConnectorCommitHandle commitShared() hdfsContext.getSource(), hdfsContext.getSession().flatMap(MetastoreUtil::getMetastoreHeaders), hdfsContext.getSession().map(MetastoreUtil::isUserDefinedTypeEncodingEnabled).orElse(false), - columnConverterProvider); + columnConverterProvider, + hdfsContext.getSession().map(ConnectorSession::getWarningCollector).orElse(NOOP)); switch (action.getType()) { case DROP: committer.prepareDropTable(metastoreContext, schemaTableName); @@ -1160,7 +1163,8 @@ private ConnectorCommitHandle commitShared() hdfsContext.getSource(), hdfsContext.getSession().flatMap(MetastoreUtil::getMetastoreHeaders), hdfsContext.getSession().map(MetastoreUtil::isUserDefinedTypeEncodingEnabled).orElse(false), - columnConverterProvider); + columnConverterProvider, + hdfsContext.getSession().map(ConnectorSession::getWarningCollector).orElse(NOOP)); switch (action.getType()) { case DROP: committer.prepareDropPartition(metastoreContext, schemaTableName, partitionValues); diff --git a/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/MockHiveMetastoreClient.java b/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/MockHiveMetastoreClient.java index a3ebae95fa71a..cafcd6463f1db 100644 --- a/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/MockHiveMetastoreClient.java +++ b/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/MockHiveMetastoreClient.java @@ -18,6 +18,7 @@ import com.facebook.presto.hive.metastore.Column; import com.facebook.presto.hive.metastore.MetastoreContext; import com.facebook.presto.hive.metastore.PartitionNameWithVersion; +import com.facebook.presto.spi.WarningCollector; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -70,7 +71,7 @@ public class MockHiveMetastoreClient public static final List TEST_PRIMARY_KEY = ImmutableList.of(new SQLPrimaryKey(TEST_DATABASE, TEST_TABLE_WITH_CONSTRAINTS, "c1", 0, "", true, false, true)); public static final List TEST_UNIQUE_CONSTRAINT = ImmutableList.of(new SQLUniqueConstraint("", TEST_DATABASE, TEST_TABLE_WITH_CONSTRAINTS, "c2", 1, "", true, false, true)); public static final String TEST_TOKEN = "token"; - public static final MetastoreContext TEST_METASTORE_CONTEXT = new MetastoreContext("test_user", "test_queryId", Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + public static final MetastoreContext TEST_METASTORE_CONTEXT = new MetastoreContext("test_user", "test_queryId", Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, WarningCollector.NOOP); public static final String TEST_PARTITION1 = "key=testpartition1"; public static final String TEST_PARTITION2 = "key=testpartition2"; public static final List TEST_PARTITION_VALUES1 = ImmutableList.of("testpartition1"); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/CreateEmptyPartitionProcedure.java b/presto-hive/src/main/java/com/facebook/presto/hive/CreateEmptyPartitionProcedure.java index d81a39b2c43d9..a41605b41e82f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/CreateEmptyPartitionProcedure.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/CreateEmptyPartitionProcedure.java @@ -119,7 +119,7 @@ private void doCreateEmptyPartition(ConnectorSession session, String schema, Str .map(String.class::cast) .collect(toImmutableList()); - if (metastore.getPartition(new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER), schema, table, partitionStringValues).isPresent()) { + if (metastore.getPartition(new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()), schema, table, partitionStringValues).isPresent()) { throw new PrestoException(ALREADY_EXISTS, "Partition already exists"); } String partitionName = FileUtils.makePartName(actualPartitionColumnNames, partitionStringValues); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java index b0f9ad0342414..b6994e59d3091 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java @@ -1753,7 +1753,7 @@ public static boolean shouldCreateFilesForMissingBuckets(Table table, ConnectorS private MetastoreContext getMetastoreContext(ConnectorSession session) { - return new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider()); + return new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider(), session.getWarningCollector()); } private Column columnHandleToColumn(ConnectorSession session, HiveColumnHandle handle) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java index 6ca4d57cf03f8..7a818d3a6b6ff 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java @@ -179,7 +179,7 @@ private ConnectorPageSink createPageSink(HiveWritableTableHandle handle, boolean new HivePageSinkMetadataProvider( handle.getPageSinkMetadata(), memoizeMetastore(metastore, metastoreImpersonationEnabled, perTransactionMetastoreCacheMaximumSize, metastorePartitionCacheMaxColumnCount), - new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), columnConverterProvider)), + new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), columnConverterProvider, session.getWarningCollector())), typeManager, hdfsEnvironment, pageSorter, diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java index a20c019357f0c..e0f642ebabff8 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java @@ -219,7 +219,7 @@ private Map createPartitionPredicates( if (domains.isPresent()) { Map columnHandleDomainMap = domains.get(); ImmutableMap.Builder partitionPredicateBuilder = ImmutableMap.builder(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider()); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider(), session.getWarningCollector()); for (HiveColumnHandle partitionColumn : partitionColumns) { Column key = new Column( partitionColumn.getName(), @@ -436,7 +436,7 @@ private Optional parseValuesAndFilterPartition( private Table getTable(ConnectorSession session, SemiTransactionalHiveMetastore metastore, HiveTableHandle hiveTableHandle, boolean offlineDataDebugModeEnabled) { - MetastoreContext context = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider()); + MetastoreContext context = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider(), session.getWarningCollector()); Optional target = metastore.getTable(context, hiveTableHandle); if (!target.isPresent()) { throw new TableNotFoundException(hiveTableHandle.getSchemaTableName()); @@ -456,7 +456,7 @@ private List getFilteredPartitionNames(ConnectorSession session, SemiTra return ImmutableList.of(); } - MetastoreContext context = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider()); + MetastoreContext context = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider(), session.getWarningCollector()); // fetch the partition names return metastore.getPartitionNamesByFilter(context, hiveTableHandle, partitionPredicates) .orElseThrow(() -> new TableNotFoundException(hiveTableHandle.getSchemaTableName())); @@ -468,7 +468,7 @@ private List getAllPartitionNames(ConnectorSession session, SemiTransact return ImmutableList.of(); } // fetch the partition names - MetastoreContext context = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider()); + MetastoreContext context = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider(), session.getWarningCollector()); return metastore.getPartitionNames(context, hiveTableHandle) .orElseThrow(() -> new TableNotFoundException(hiveTableHandle.getSchemaTableName())); } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java index 8c309c642c849..e6132e57b8fbb 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java @@ -229,7 +229,7 @@ public ConnectorSplitSource getSplits( throw new PrestoException(HIVE_TRANSACTION_NOT_FOUND, format("Transaction not found: %s", transaction)); } SemiTransactionalHiveMetastore metastore = metadata.getMetastore(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider()); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider(), session.getWarningCollector()); Table table = layout.getTable(metastore, metastoreContext); if (!isOfflineDataDebugModeEnabled(session)) { @@ -671,7 +671,7 @@ private Map getPartitionSplitInfo( Map predicateColumns, Optional> domains) { - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider()); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider(), session.getWarningCollector()); Map> partitions = metastore.getPartitionsByNames( metastoreContext, tableName.getSchemaName(), diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java index d57d6baee11ae..29cb05a8892bf 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java @@ -356,7 +356,7 @@ private static void checkWritable( public static Path getTableDefaultLocation(ConnectorSession session, SemiTransactionalHiveMetastore metastore, HdfsEnvironment hdfsEnvironment, String schemaName, String tableName) { - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider()); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider(), session.getWarningCollector()); Optional location = getDatabase(session.getIdentity(), metastoreContext, metastore, schemaName).getLocation(); if (!location.isPresent() || location.get().isEmpty()) { throw new PrestoException(HIVE_DATABASE_LOCATION_ERROR, format("Database '%s' location is not set", schemaName)); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/SyncPartitionMetadataProcedure.java b/presto-hive/src/main/java/com/facebook/presto/hive/SyncPartitionMetadataProcedure.java index ee99489d370a7..0a4c07fc5d3e7 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/SyncPartitionMetadataProcedure.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/SyncPartitionMetadataProcedure.java @@ -126,7 +126,8 @@ private void doSyncPartitionMetadata(ConnectorSession session, String schemaName session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), - metastore.getColumnConverterProvider()); + metastore.getColumnConverterProvider(), + session.getWarningCollector()); Table table = metastore.getTable(metastoreContext, schemaName, tableName) .orElseThrow(() -> new TableNotFoundException(schemaTableName)); if (table.getPartitionColumns().isEmpty()) { diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveFilterPushdown.java b/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveFilterPushdown.java index c76492a490456..789914fb5f833 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveFilterPushdown.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveFilterPushdown.java @@ -156,7 +156,7 @@ public ConnectorPushdownFilterResult getConnectorPushdownFilterResult( SemiTransactionalHiveMetastore metastore = ((HiveMetadata) metadata).getMetastore(); - MetastoreContext context = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider()); + MetastoreContext context = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider(), session.getWarningCollector()); Table table = metastore.getTable(context, hiveTableHandle) .orElseThrow(() -> new TableNotFoundException(tableName)); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacyAccessControl.java b/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacyAccessControl.java index 0c5a3afcde723..1bfe353385034 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacyAccessControl.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/security/LegacyAccessControl.java @@ -104,7 +104,7 @@ public void checkCanDropTable(ConnectorTransactionHandle transaction, ConnectorI TransactionalMetadata metadata = hiveTransactionManager.get(transaction); // TODO: Refactor code to inject metastore headers using AccessControlContext instead of empty() - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); Optional
target = metadata.getMetastore().getTable(metastoreContext, tableName.getSchemaName(), tableName.getTableName()); if (!target.isPresent()) { diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/security/SqlStandardAccessControl.java b/presto-hive/src/main/java/com/facebook/presto/hive/security/SqlStandardAccessControl.java index 94e9eb12652ea..25de385a3b4fc 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/security/SqlStandardAccessControl.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/security/SqlStandardAccessControl.java @@ -104,7 +104,7 @@ public SqlStandardAccessControl( public void checkCanCreateSchema(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, String schemaName) { // TODO: Refactor code to inject metastore headers using AccessControlContext instead of empty() - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isAdmin(transaction, identity, metastoreContext)) { denyCreateSchema(schemaName); } @@ -113,7 +113,7 @@ public void checkCanCreateSchema(ConnectorTransactionHandle transaction, Connect @Override public void checkCanDropSchema(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, String schemaName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isDatabaseOwner(transaction, identity, metastoreContext, schemaName)) { denyDropSchema(schemaName); } @@ -122,7 +122,7 @@ public void checkCanDropSchema(ConnectorTransactionHandle transaction, Connector @Override public void checkCanRenameSchema(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, String schemaName, String newSchemaName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isDatabaseOwner(transaction, identity, metastoreContext, schemaName)) { denyRenameSchema(schemaName, newSchemaName); } @@ -142,7 +142,7 @@ public Set filterSchemas(ConnectorTransactionHandle transactionHandle, C @Override public void checkCanCreateTable(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isDatabaseOwner(transaction, identity, metastoreContext, tableName.getSchemaName())) { denyCreateTable(tableName.toString()); } @@ -151,7 +151,7 @@ public void checkCanCreateTable(ConnectorTransactionHandle transaction, Connecto @Override public void checkCanDropTable(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isTableOwner(transaction, identity, metastoreContext, tableName)) { denyDropTable(tableName.toString()); } @@ -160,7 +160,7 @@ public void checkCanDropTable(ConnectorTransactionHandle transaction, ConnectorI @Override public void checkCanRenameTable(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName, SchemaTableName newTableName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isTableOwner(transaction, identity, metastoreContext, tableName)) { denyRenameTable(tableName.toString(), newTableName.toString()); } @@ -180,7 +180,7 @@ public Set filterTables(ConnectorTransactionHandle transactionH @Override public void checkCanAddColumn(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isTableOwner(transaction, identity, metastoreContext, tableName)) { denyAddColumn(tableName.toString()); } @@ -189,7 +189,7 @@ public void checkCanAddColumn(ConnectorTransactionHandle transaction, ConnectorI @Override public void checkCanDropColumn(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isTableOwner(transaction, identity, metastoreContext, tableName)) { denyDropColumn(tableName.toString()); } @@ -198,7 +198,7 @@ public void checkCanDropColumn(ConnectorTransactionHandle transaction, Connector @Override public void checkCanRenameColumn(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isTableOwner(transaction, identity, metastoreContext, tableName)) { denyRenameColumn(tableName.toString()); } @@ -207,7 +207,7 @@ public void checkCanRenameColumn(ConnectorTransactionHandle transaction, Connect @Override public void checkCanSelectFromColumns(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName, Set columnOrSubfieldNames) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); // TODO: Implement column level access control if (!checkTablePermission(transaction, identity, metastoreContext, tableName, SELECT, false)) { denySelectTable(tableName.toString()); @@ -217,7 +217,7 @@ public void checkCanSelectFromColumns(ConnectorTransactionHandle transaction, Co @Override public void checkCanInsertIntoTable(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!checkTablePermission(transaction, identity, metastoreContext, tableName, INSERT, false)) { denyInsertTable(tableName.toString()); } @@ -226,7 +226,7 @@ public void checkCanInsertIntoTable(ConnectorTransactionHandle transaction, Conn @Override public void checkCanDeleteFromTable(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!checkTablePermission(transaction, identity, metastoreContext, tableName, DELETE, false)) { denyDeleteTable(tableName.toString()); } @@ -235,7 +235,7 @@ public void checkCanDeleteFromTable(ConnectorTransactionHandle transaction, Conn @Override public void checkCanTruncateTable(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!checkTablePermission(transaction, identity, metastoreContext, tableName, DELETE, false)) { denyTruncateTable(tableName.toString()); } @@ -244,7 +244,7 @@ public void checkCanTruncateTable(ConnectorTransactionHandle transaction, Connec @Override public void checkCanUpdateTableColumns(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName, Set updatedColumns) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!checkTablePermission(transaction, identity, metastoreContext, tableName, UPDATE, false)) { denyUpdateTableColumns(tableName.toString(), updatedColumns); } @@ -253,7 +253,7 @@ public void checkCanUpdateTableColumns(ConnectorTransactionHandle transaction, C @Override public void checkCanCreateView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isDatabaseOwner(transaction, identity, metastoreContext, viewName.getSchemaName())) { denyCreateView(viewName.toString()); } @@ -262,7 +262,7 @@ public void checkCanCreateView(ConnectorTransactionHandle transaction, Connector @Override public void checkCanDropView(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName viewName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isTableOwner(transaction, identity, metastoreContext, viewName)) { denyDropView(viewName.toString()); } @@ -272,7 +272,7 @@ public void checkCanDropView(ConnectorTransactionHandle transaction, ConnectorId public void checkCanCreateViewWithSelectFromColumns(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, SchemaTableName tableName, Set columnNames) { checkCanSelectFromColumns(transaction, identity, context, tableName, columnNames.stream().map(column -> new Subfield(column)).collect(toImmutableSet())); - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); // TODO implement column level access control if (!checkTablePermission(transaction, identity, metastoreContext, tableName, SELECT, true)) { denyCreateViewWithSelect(tableName.toString(), identity); @@ -282,7 +282,7 @@ public void checkCanCreateViewWithSelectFromColumns(ConnectorTransactionHandle t @Override public void checkCanSetCatalogSessionProperty(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, String propertyName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isAdmin(transaction, identity, metastoreContext)) { denySetCatalogSessionProperty(connectorId, propertyName); } @@ -291,7 +291,7 @@ public void checkCanSetCatalogSessionProperty(ConnectorTransactionHandle transac @Override public void checkCanGrantTablePrivilege(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, Privilege privilege, SchemaTableName tableName, PrestoPrincipal grantee, boolean withGrantOption) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (isTableOwner(transaction, identity, metastoreContext, tableName)) { return; } @@ -304,7 +304,7 @@ public void checkCanGrantTablePrivilege(ConnectorTransactionHandle transaction, @Override public void checkCanRevokeTablePrivilege(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, Privilege privilege, SchemaTableName tableName, PrestoPrincipal revokee, boolean grantOptionFor) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (isTableOwner(transaction, identity, metastoreContext, tableName)) { return; } @@ -321,7 +321,7 @@ public void checkCanCreateRole(ConnectorTransactionHandle transactionHandle, Con if (grantor.isPresent()) { throw new AccessDeniedException("Hive Connector does not support WITH ADMIN statement"); } - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isAdmin(transactionHandle, identity, metastoreContext)) { denyCreateRole(role); } @@ -330,7 +330,7 @@ public void checkCanCreateRole(ConnectorTransactionHandle transactionHandle, Con @Override public void checkCanDropRole(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, String role) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isAdmin(transactionHandle, identity, metastoreContext)) { denyDropRole(role); } @@ -343,7 +343,7 @@ public void checkCanGrantRoles(ConnectorTransactionHandle transactionHandle, Con if (grantor.isPresent()) { throw new AccessDeniedException("Hive Connector does not support GRANTED BY statement"); } - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!hasAdminOptionForRoles(transactionHandle, identity, metastoreContext, roles)) { denyGrantRoles(roles, grantees); } @@ -356,7 +356,7 @@ public void checkCanRevokeRoles(ConnectorTransactionHandle transactionHandle, Co if (grantor.isPresent()) { throw new AccessDeniedException("Hive Connector does not support GRANTED BY statement"); } - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!hasAdminOptionForRoles(transactionHandle, identity, metastoreContext, roles)) { denyRevokeRoles(roles, grantees); } @@ -366,7 +366,7 @@ public void checkCanRevokeRoles(ConnectorTransactionHandle transactionHandle, Co public void checkCanSetRole(ConnectorTransactionHandle transaction, ConnectorIdentity identity, AccessControlContext context, String role, String catalogName) { SemiTransactionalHiveMetastore metastore = getMetastore(transaction); - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isRoleApplicable(metastore, identity, new PrestoPrincipal(USER, identity.getUser()), metastoreContext, role)) { denySetRole(role); } @@ -375,7 +375,7 @@ public void checkCanSetRole(ConnectorTransactionHandle transaction, ConnectorIde @Override public void checkCanShowRoles(ConnectorTransactionHandle transactionHandle, ConnectorIdentity identity, AccessControlContext context, String catalogName) { - MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(identity, context.getQueryId().getId(), context.getClientInfo(), context.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, context.getWarningCollector()); if (!isAdmin(transactionHandle, identity, metastoreContext)) { denyShowRoles(catalogName); } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/statistics/MetastoreHiveStatisticsProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/statistics/MetastoreHiveStatisticsProvider.java index 1a248924a3b9b..461d96efcb0a7 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/statistics/MetastoreHiveStatisticsProvider.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/statistics/MetastoreHiveStatisticsProvider.java @@ -117,7 +117,7 @@ private static Map getPartitionsStatistics(Connecto return ImmutableMap.of(); } boolean unpartitioned = hivePartitions.stream().anyMatch(partition -> partition.getPartitionId().equals(UNPARTITIONED_ID)); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider()); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), metastore.getColumnConverterProvider(), session.getWarningCollector()); if (unpartitioned) { checkArgument(hivePartitions.size() == 1, "expected only one hive partition"); return ImmutableMap.of(UNPARTITIONED_ID, metastore.getTableStatistics(metastoreContext, table.getSchemaName(), table.getTableName())); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java index 5b3bfd30071c4..cf41cb0f19eb1 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java @@ -1484,7 +1484,7 @@ protected void doTestMismatchSchemaTable( // alter the table schema try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(session); Table oldTable = transaction.getMetastore().getTable(metastoreContext, schemaName, tableName).get(); HiveTypeTranslator hiveTypeTranslator = new HiveTypeTranslator(); @@ -2739,7 +2739,7 @@ private void assertEmptyFile(HiveStorageFormat format) ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); Table table = transaction.getMetastore() .getTable(metastoreContext, tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(AssertionError::new); @@ -2913,7 +2913,7 @@ public void testTableCreationIgnoreExisting() Table table = createSimpleTable(schemaTableName, columns, session, targetPath, "q1"); transaction.getMetastore() .createTable(session, table, privileges, Optional.empty(), false, EMPTY_TABLE_STATISTICS); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); Optional
tableHandle = transaction.getMetastore().getTable(metastoreContext, schemaName, tableName); assertTrue(tableHandle.isPresent()); transaction.commit(); @@ -3845,7 +3845,7 @@ protected String partitionTargetPath(SchemaTableName schemaTableName, String par ConnectorSession session = newSession(); SemiTransactionalHiveMetastore metastore = transaction.getMetastore(); LocationService locationService = getLocationService(); - Table table = metastore.getTable(new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER), schemaTableName.getSchemaName(), schemaTableName.getTableName()).get(); + Table table = metastore.getTable(new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()), schemaTableName.getSchemaName(), schemaTableName.getTableName()).get(); LocationHandle handle = locationService.forExistingTable(metastore, session, table, false); return locationService.getPartitionWriteInfo(handle, Optional.empty(), partitionName).getTargetPath().toString(); } @@ -4042,7 +4042,7 @@ protected void doCreateTable(SchemaTableName tableName, HiveStorageFormat storag try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); ConnectorMetadata metadata = transaction.getMetadata(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); // load the new table ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); @@ -4098,7 +4098,7 @@ protected void doCreateEmptyTable(SchemaTableName tableName, HiveStorageFormat s try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); ConnectorMetadata metadata = transaction.getMetadata(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); // load the new table ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); @@ -4158,7 +4158,7 @@ private void doInsert(HiveStorageFormat storageFormat, SchemaTableName tableName try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); ConnectorMetadata metadata = transaction.getMetadata(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); // load the new table ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); @@ -4185,7 +4185,7 @@ private void doInsert(HiveStorageFormat storageFormat, SchemaTableName tableName Set existingFiles; try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); existingFiles = listAllDataFiles(metastoreContext, transaction, tableName.getSchemaName(), tableName.getTableName()); assertFalse(existingFiles.isEmpty()); } @@ -4197,7 +4197,7 @@ private void doInsert(HiveStorageFormat storageFormat, SchemaTableName tableName ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); // "stage" insert data HiveInsertTableHandle insertTableHandle = (HiveInsertTableHandle) metadata.beginInsert(session, tableHandle); @@ -4251,7 +4251,7 @@ private void doInsert(HiveStorageFormat storageFormat, SchemaTableName tableName try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); ConnectorMetadata metadata = transaction.getMetadata(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); @@ -4265,7 +4265,7 @@ private void doInsert(HiveStorageFormat storageFormat, SchemaTableName tableName // verify statistics unchanged try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); HiveBasicStatistics statistics = getBasicStatisticsForTable(metastoreContext, transaction, tableName); assertEquals(statistics.getRowCount().getAsLong(), CREATE_TABLE_DATA.getRowCount() * 3L); assertEquals(statistics.getFileCount().getAsLong(), 3L); @@ -4375,7 +4375,7 @@ private void doInsertIntoNewPartition(HiveStorageFormat storageFormat, SchemaTab Set existingFiles; try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); // verify partitions were created List partitionNames = transaction.getMetastore().getPartitionNames(metastoreContext, tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); @@ -4461,7 +4461,7 @@ private void doInsertIntoNewPartition(HiveStorageFormat storageFormat, SchemaTab assertEqualsIgnoreOrder(result.getMaterializedRows(), CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows()); // verify we did not modify the table directory - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); assertEquals(listAllDataFiles(metastoreContext, transaction, tableName.getSchemaName(), tableName.getTableName()), existingFiles); // verify temp directory is empty @@ -4506,7 +4506,7 @@ private void doInsertIntoExistingPartition(HiveStorageFormat storageFormat, Sche ConnectorSession session = newSession(); ConnectorMetadata metadata = transaction.getMetadata(); ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); // verify partitions were created List partitionNames = transaction.getMetastore().getPartitionNames(metastoreContext, tableName.getSchemaName(), tableName.getTableName()) @@ -4541,7 +4541,7 @@ private void doInsertIntoExistingPartition(HiveStorageFormat storageFormat, Sche ConnectorMetadata metadata = transaction.getMetadata(); ConnectorSession session = newSession(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); existingFiles = listAllDataFiles(metastoreContext, transaction, tableName.getSchemaName(), tableName.getTableName()); assertFalse(existingFiles.isEmpty()); @@ -4591,7 +4591,7 @@ private void doInsertIntoExistingPartition(HiveStorageFormat storageFormat, Sche ConnectorSession session = newSession(); ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); // verify the data is unchanged MaterializedResult result = readTable(transaction, tableHandle, columnHandles, newSession(), TupleDomain.all(), OptionalInt.empty(), Optional.empty()); @@ -4635,11 +4635,11 @@ private void doInsertIntoExistingPartitionEmptyStatistics(HiveStorageFormat stor insertData(tableName, CREATE_TABLE_PARTITIONED_DATA); ConnectorSession session = newSession(); try (Transaction transaction = newTransaction()) { - List partitionNames = transaction.getMetastore().getPartitionNames(new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER), tableName.getSchemaName(), tableName.getTableName()) + List partitionNames = transaction.getMetastore().getPartitionNames(new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); for (String partitionName : partitionNames) { - HiveBasicStatistics statistics = getBasicStatisticsForPartition(new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER), transaction, tableName, partitionName); + HiveBasicStatistics statistics = getBasicStatisticsForPartition(new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()), transaction, tableName, partitionName); assertThat(statistics.getRowCount()).isNotPresent(); assertThat(statistics.getInMemoryDataSizeInBytes()).isNotPresent(); // fileCount and rawSize statistics are computed on the fly by the metastore, thus cannot be erased @@ -4758,7 +4758,7 @@ private void doTestMetadataDelete(HiveStorageFormat storageFormat, SchemaTableNa try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); ConnectorMetadata metadata = transaction.getMetadata(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); // verify partitions were created List partitionNames = transaction.getMetastore().getPartitionNames(metastoreContext, tableName.getSchemaName(), tableName.getTableName()) @@ -4842,7 +4842,7 @@ private void doTestMetadataDelete(HiveStorageFormat storageFormat, SchemaTableNa assertEqualsIgnoreOrder(actualAfterDelete2.getMaterializedRows(), ImmutableList.of()); // verify table directory is empty - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); Set filesAfterDelete = listAllDataFiles(metastoreContext, transaction, tableName.getSchemaName(), tableName.getTableName()); assertTrue(filesAfterDelete.isEmpty()); } @@ -5460,7 +5460,7 @@ protected Table createEmptyTable(SchemaTableName schemaTableName, HiveStorageFor try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); return transaction.getMetastore().getTable(metastoreContext, schemaTableName.getSchemaName(), schemaTableName.getTableName()).get(); } } @@ -5473,7 +5473,7 @@ private void alterBucketProperty(SchemaTableName schemaTableName, Optional table = transaction.getMetastore().getTable(metastoreContext, schemaName, tableName); Table.Builder tableBuilder = Table.builder(table.get()); tableBuilder.getStorageBuilder().setBucketProperty(bucketProperty); @@ -5694,7 +5694,7 @@ private void doTestTransactionDeleteInsert( try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); // verify partitions List partitionNames = transaction.getMetastore() .getPartitionNames(metastoreContext, tableName.getSchemaName(), tableName.getTableName()) @@ -5739,7 +5739,7 @@ protected Set getSupportedCreateTableHiveStorageFormats() private List> getTableConstraints(SchemaTableName tableName) { ConnectorSession session = newSession(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); return metastoreClient.getTableConstraints(metastoreContext, tableName.getSchemaName(), tableName.getTableName()); } @@ -5854,7 +5854,7 @@ public void triggerConflict(ConnectorSession session, SchemaTableName tableName, // This method bypasses transaction interface because this method is inherently hacky and doesn't work well with the transaction abstraction. // Additionally, this method is not part of a test. Its purpose is to set up an environment for another test. ExtendedHiveMetastore metastoreClient = getMetastoreClient(); - MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), false, DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); Optional partition = metastoreClient.getPartition(metastoreContext, tableName.getSchemaName(), tableName.getTableName(), copyPartitionFrom); conflictPartition = Partition.builder(partition.get()) .setValues(toPartitionValues(partitionNameToConflict)) diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/HiveQueryRunner.java b/presto-hive/src/test/java/com/facebook/presto/hive/HiveQueryRunner.java index 1541c76fa02bd..291ac3ebfc31e 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/HiveQueryRunner.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/HiveQueryRunner.java @@ -25,6 +25,7 @@ import com.facebook.presto.hive.metastore.MetastoreContext; import com.facebook.presto.hive.metastore.file.FileHiveMetastore; import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.security.Identity; import com.facebook.presto.spi.security.PrincipalType; import com.facebook.presto.spi.security.SelectedRole; @@ -77,7 +78,7 @@ private HiveQueryRunner() public static final String TPCH_BUCKETED_SCHEMA = "tpch_bucketed"; public static final String TPCDS_SCHEMA = "tpcds"; public static final String TPCDS_BUCKETED_SCHEMA = "tpcds_bucketed"; - public static final MetastoreContext METASTORE_CONTEXT = new MetastoreContext("test_user", "test_queryId", Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + public static final MetastoreContext METASTORE_CONTEXT = new MetastoreContext("test_user", "test_queryId", Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, WarningCollector.NOOP); private static final String TEMPORARY_TABLE_SCHEMA = "__temporary_tables__"; private static final DateTimeZone TIME_ZONE = DateTimeZone.forID("America/Bahia_Banderas"); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveCommitHandleOutput.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveCommitHandleOutput.java index 7737bdaf9766f..019efebcf3ea5 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveCommitHandleOutput.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveCommitHandleOutput.java @@ -195,7 +195,8 @@ public void testCommitOutputForPartitions() Optional.empty(), Optional.empty(), false, - HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER), + HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, + connectorSession.getWarningCollector()), TEST_SCHEMA, TEST_TABLE, ImmutableList.of(partitionName)); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveLogicalPlanner.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveLogicalPlanner.java index 098371a81f4c9..af48b7110e218 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveLogicalPlanner.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveLogicalPlanner.java @@ -548,7 +548,7 @@ public void testMetadataAggregationFoldingWithEmptyPartitions() "CREATE TABLE test_metadata_aggregation_folding_with_empty_partitions WITH (partitioned_by = ARRAY['ds']) AS " + "SELECT orderkey, CAST(to_iso8601(date_add('DAY', orderkey % 2, date('2020-07-01'))) AS VARCHAR) AS ds FROM orders WHERE orderkey < 1000"); ExtendedHiveMetastore metastore = replicateHiveMetastore((DistributedQueryRunner) queryRunner); - MetastoreContext metastoreContext = new MetastoreContext(getSession().getUser(), getSession().getQueryId().getId(), Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(getSession().getUser(), getSession().getQueryId().getId(), Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, getSession().getWarningCollector()); Table table = metastore.getTable(metastoreContext, getSession().getSchema().get(), "test_metadata_aggregation_folding_with_empty_partitions").get(); // Add one partition with no statistics. @@ -692,7 +692,7 @@ public void testMetadataAggregationFoldingWithEmptyPartitionsAndMetastoreThresho "CREATE TABLE test_metadata_aggregation_folding_with_empty_partitions_with_threshold WITH (partitioned_by = ARRAY['ds']) AS " + "SELECT orderkey, CAST(to_iso8601(date_add('DAY', orderkey % 2, date('2020-07-01'))) AS VARCHAR) AS ds FROM orders WHERE orderkey < 1000"); ExtendedHiveMetastore metastore = replicateHiveMetastore((DistributedQueryRunner) queryRunner); - MetastoreContext metastoreContext = new MetastoreContext(getSession().getUser(), getSession().getQueryId().getId(), Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(getSession().getUser(), getSession().getQueryId().getId(), Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, getSession().getWarningCollector()); Table table = metastore.getTable(metastoreContext, getSession().getSchema().get(), "test_metadata_aggregation_folding_with_empty_partitions_with_threshold").get(); // Add one partition with no statistics. @@ -766,7 +766,7 @@ public void testMetadataAggregationFoldingWithTwoPartitionColumns() "SELECT orderkey, CAST(to_iso8601(date_add('DAY', orderkey % 2, date('2020-07-01'))) AS VARCHAR) AS ds, IF(orderkey % 2 = 1, 'A', 'B') status " + "FROM orders WHERE orderkey < 1000"); ExtendedHiveMetastore metastore = replicateHiveMetastore((DistributedQueryRunner) queryRunner); - MetastoreContext metastoreContext = new MetastoreContext(getSession().getUser(), getSession().getQueryId().getId(), Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(getSession().getUser(), getSession().getQueryId().getId(), Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, getSession().getWarningCollector()); Table table = metastore.getTable(metastoreContext, getSession().getSchema().get(), "test_metadata_aggregation_folding_with_two_partitions_columns").get(); // Add one partition with no statistics. diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveMaterializedViewLogicalPlanner.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveMaterializedViewLogicalPlanner.java index b0a80e6ecb8a3..7ab87ed3dcd33 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveMaterializedViewLogicalPlanner.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveMaterializedViewLogicalPlanner.java @@ -2717,7 +2717,7 @@ private void setReferencedMaterializedViews(DistributedQueryRunner queryRunner, private void appendTableParameter(ExtendedHiveMetastore metastore, String tableName, String parameterKey, String parameterValue) { - MetastoreContext metastoreContext = new MetastoreContext(getSession().getUser(), getSession().getQueryId().getId(), Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(getSession().getUser(), getSession().getQueryId().getId(), Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, getSession().getWarningCollector()); Optional
table = metastore.getTable(metastoreContext, getSession().getSchema().get(), tableName); if (table.isPresent()) { Table originalTable = table.get(); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveMaterializedViewUtils.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveMaterializedViewUtils.java index 95592678069e5..05e739b262084 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveMaterializedViewUtils.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveMaterializedViewUtils.java @@ -26,6 +26,7 @@ import com.facebook.presto.spi.MaterializedViewDefinition; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.sql.planner.LiteralEncoder; import com.facebook.presto.sql.tree.Expression; import com.google.common.collect.ImmutableList; @@ -62,7 +63,7 @@ public class TestHiveMaterializedViewUtils private static final String SQL = "sql"; private final LiteralEncoder literalEncoder = new LiteralEncoder(new TestingBlockEncodingSerde()); - private final MetastoreContext metastoreContext = new MetastoreContext(USER_NAME, QUERY_ID, Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + private final MetastoreContext metastoreContext = new MetastoreContext(USER_NAME, QUERY_ID, Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, WarningCollector.NOOP); @Test public void testMaterializedDataPredicates() diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/geospatial/TestSpatialJoins.java b/presto-hive/src/test/java/com/facebook/presto/hive/geospatial/TestSpatialJoins.java index 70684176fb53f..607b04634809d 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/geospatial/TestSpatialJoins.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/geospatial/TestSpatialJoins.java @@ -27,6 +27,7 @@ import com.facebook.presto.hive.metastore.Database; import com.facebook.presto.hive.metastore.MetastoreContext; import com.facebook.presto.hive.metastore.file.FileHiveMetastore; +import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.security.PrincipalType; import com.facebook.presto.testing.QueryRunner; import com.facebook.presto.tests.AbstractTestQueryFramework; @@ -104,7 +105,7 @@ protected QueryRunner createQueryRunner() FileHiveMetastore metastore = new FileHiveMetastore(hdfsEnvironment, dataDirectory.toURI().toString(), "test"); ColumnConverterProvider columnConverterProvider = HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER; - metastore.createDatabase(new MetastoreContext("test_user", "test_queryId", Optional.empty(), Optional.empty(), Optional.empty(), false, columnConverterProvider), Database.builder() + metastore.createDatabase(new MetastoreContext("test_user", "test_queryId", Optional.empty(), Optional.empty(), Optional.empty(), false, columnConverterProvider, WarningCollector.NOOP), Database.builder() .setDatabaseName("default") .setOwnerName("public") .setOwnerType(PrincipalType.ROLE) diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/hudi/HudiTestingDataGenerator.java b/presto-hive/src/test/java/com/facebook/presto/hive/hudi/HudiTestingDataGenerator.java index 7e7581946a00c..281bd1568b12f 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/hudi/HudiTestingDataGenerator.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/hudi/HudiTestingDataGenerator.java @@ -28,6 +28,7 @@ import com.facebook.presto.hive.metastore.Storage; import com.facebook.presto.hive.metastore.StorageFormat; import com.facebook.presto.hive.metastore.Table; +import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.security.PrincipalType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -67,7 +68,7 @@ public class HudiTestingDataGenerator { private static final String OWNER_PUBLIC = "public"; - private static final MetastoreContext METASTORE_CONTEXT = new MetastoreContext("test_user", "test_queryId", Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + private static final MetastoreContext METASTORE_CONTEXT = new MetastoreContext("test_user", "test_queryId", Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, WarningCollector.NOOP); private static final PrincipalPrivileges PRINCIPAL_PRIVILEGES = new PrincipalPrivileges(ImmutableMultimap.of(), ImmutableMultimap.of()); private static final StorageFormat STORAGE_FORMAT_COPY_ON_WRITE = StorageFormat.create( diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/glue/TestHiveClientGlueMetastore.java b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/glue/TestHiveClientGlueMetastore.java index cab8395e70d7c..bc59ef244162b 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/glue/TestHiveClientGlueMetastore.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/glue/TestHiveClientGlueMetastore.java @@ -273,7 +273,8 @@ public void testTableWithoutStorageDescriptor() session.getSource(), getMetastoreHeaders(session), false, - DEFAULT_COLUMN_CONVERTER_PROVIDER); + DEFAULT_COLUMN_CONVERTER_PROVIDER, + session.getWarningCollector()); TableInput tableInput = new TableInput() .withName(table.getTableName()) .withTableType(EXTERNAL_TABLE.name()); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/security/ranger/TestRangerBasedAccessControl.java b/presto-hive/src/test/java/com/facebook/presto/hive/security/ranger/TestRangerBasedAccessControl.java index 3717f0b2d226a..4c69536d931c1 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/security/ranger/TestRangerBasedAccessControl.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/security/ranger/TestRangerBasedAccessControl.java @@ -20,6 +20,7 @@ import com.facebook.presto.common.Subfield; import com.facebook.presto.spi.QueryId; import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.connector.ConnectorAccessControl; import com.facebook.presto.spi.connector.ConnectorTransactionHandle; import com.facebook.presto.spi.security.AccessControlContext; @@ -52,7 +53,7 @@ public class TestRangerBasedAccessControl { public static final ConnectorTransactionHandle TRANSACTION_HANDLE = new ConnectorTransactionHandle() {}; - public static final AccessControlContext CONTEXT = new AccessControlContext(new QueryId("query_id"), Optional.empty(), Optional.empty()); + public static final AccessControlContext CONTEXT = new AccessControlContext(new QueryId("query_id"), Optional.empty(), Optional.empty(), WarningCollector.NOOP); @Test public void testTablePriviledgesRolesNotAllowed() diff --git a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java index e2ecdde5f6cbc..0aa9b7dca0bd7 100644 --- a/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java +++ b/presto-hudi/src/main/java/com/facebook/presto/hudi/HudiMetadata.java @@ -257,7 +257,7 @@ static List fromDataColumns(List dataColumns) public static MetastoreContext toMetastoreContext(ConnectorSession session) { - return new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + return new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); } private static HudiColumnHandle fromDataColumn(int index, Column column) diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadata.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadata.java index 3b0bb48cc40a4..6e026f990f6af 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadata.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadata.java @@ -442,7 +442,7 @@ public void dropView(ConnectorSession session, SchemaTableName viewName) private MetastoreContext getMetastoreContext(ConnectorSession session) { - return new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + return new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), getMetastoreHeaders(session), isUserDefinedTypeEncodingEnabled(session), HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()); } private List listSchemas(ConnectorSession session, String schemaNameOrNull) diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java index 54e31adf06ea5..989baec687622 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java @@ -189,7 +189,7 @@ public static Table getHiveIcebergTable(ExtendedHiveMetastore metastore, HdfsEnv HdfsContext hdfsContext = new HdfsContext(session, table.getSchemaName(), table.getTableName()); TableOperations operations = new HiveTableOperations( metastore, - new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER), + new MetastoreContext(session.getIdentity(), session.getQueryId(), session.getClientInfo(), session.getSource(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, session.getWarningCollector()), hdfsEnvironment, hdfsContext, table.getSchemaName(), diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergRegisterProcedure.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergRegisterProcedure.java index 9186d43a51ac3..6c825276c908e 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergRegisterProcedure.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergRegisterProcedure.java @@ -508,7 +508,7 @@ protected void dropTableFromMetastore(String schemaName, String tableName) { ExtendedHiveMetastore metastore = getFileHiveMetastore(); ConnectorSession connectorSession = session.toConnectorSession(); - MetastoreContext metastoreContext = new MetastoreContext(connectorSession.getIdentity(), connectorSession.getQueryId(), connectorSession.getClientInfo(), connectorSession.getSource(), getMetastoreHeaders(connectorSession), isUserDefinedTypeEncodingEnabled(connectorSession), HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + MetastoreContext metastoreContext = new MetastoreContext(connectorSession.getIdentity(), connectorSession.getQueryId(), connectorSession.getClientInfo(), connectorSession.getSource(), getMetastoreHeaders(connectorSession), isUserDefinedTypeEncodingEnabled(connectorSession), HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, connectorSession.getWarningCollector()); metastore.dropTableFromMetastore(metastoreContext, schemaName, tableName); } diff --git a/presto-main/src/main/java/com/facebook/presto/Session.java b/presto-main/src/main/java/com/facebook/presto/Session.java index 6fe6655812bab..428a28362cde0 100644 --- a/presto-main/src/main/java/com/facebook/presto/Session.java +++ b/presto-main/src/main/java/com/facebook/presto/Session.java @@ -161,9 +161,9 @@ public Session( checkArgument(!transactionId.isPresent() || unprocessedCatalogProperties.isEmpty(), "Catalog session properties cannot be set if there is an open transaction"); checkArgument(catalog.isPresent() || !schema.isPresent(), "schema is set but catalog is not"); - this.context = new AccessControlContext(queryId, clientInfo, source); this.tracer = requireNonNull(tracer, "tracer is null"); this.warningCollector = requireNonNull(warningCollector, "warningCollector is null"); + this.context = new AccessControlContext(queryId, clientInfo, source, warningCollector); } public QueryId getQueryId() diff --git a/presto-main/src/main/java/com/facebook/presto/security/AccessControlUtils.java b/presto-main/src/main/java/com/facebook/presto/security/AccessControlUtils.java index bf773aebf071c..b15331a8d1a1e 100644 --- a/presto-main/src/main/java/com/facebook/presto/security/AccessControlUtils.java +++ b/presto-main/src/main/java/com/facebook/presto/security/AccessControlUtils.java @@ -16,6 +16,7 @@ import com.facebook.presto.server.SessionContext; import com.facebook.presto.server.security.SecurityConfig; import com.facebook.presto.spi.QueryId; +import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.security.AccessControl; import com.facebook.presto.spi.security.AccessControlContext; import com.facebook.presto.spi.security.AccessDeniedException; @@ -42,7 +43,8 @@ public static void checkPermissions(AccessControl accessControl, SecurityConfig new AccessControlContext( queryId, Optional.ofNullable(sessionContext.getClientInfo()), - Optional.ofNullable(sessionContext.getSource())), + Optional.ofNullable(sessionContext.getSource()), + WarningCollector.NOOP), identity.getPrincipal(), identity.getUser()); } @@ -64,7 +66,8 @@ public static Optional getAuthorizedIdentity(AccessControl a new AccessControlContext( queryId, Optional.ofNullable(sessionContext.getClientInfo()), - Optional.ofNullable(sessionContext.getSource())), + Optional.ofNullable(sessionContext.getSource()), + WarningCollector.NOOP), identity.getUser(), sessionContext.getCertificates()); return Optional.of(authorizedIdentity); diff --git a/presto-main/src/test/java/com/facebook/presto/security/TestAccessControlManager.java b/presto-main/src/test/java/com/facebook/presto/security/TestAccessControlManager.java index 87914996430d5..94791effc9b54 100644 --- a/presto-main/src/test/java/com/facebook/presto/security/TestAccessControlManager.java +++ b/presto-main/src/test/java/com/facebook/presto/security/TestAccessControlManager.java @@ -28,6 +28,7 @@ import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.QueryId; import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.connector.Connector; import com.facebook.presto.spi.connector.ConnectorAccessControl; import com.facebook.presto.spi.connector.ConnectorTransactionHandle; @@ -79,7 +80,7 @@ public void testInitializing() AccessControlManager accessControlManager = new AccessControlManager(createTestTransactionManager()); accessControlManager.checkCanSetUser( new Identity(USER_NAME, Optional.of(PRINCIPAL)), - new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty()), + new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty(), WarningCollector.NOOP), Optional.empty(), "foo"); } @@ -91,7 +92,7 @@ public void testNoneSystemAccessControl() accessControlManager.setSystemAccessControl(AllowAllSystemAccessControl.NAME, ImmutableMap.of()); accessControlManager.checkCanSetUser( new Identity(USER_NAME, Optional.of(PRINCIPAL)), - new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty()), + new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty(), WarningCollector.NOOP), Optional.empty(), USER_NAME); } @@ -103,7 +104,7 @@ public void testReadOnlySystemAccessControl() QualifiedObjectName tableName = new QualifiedObjectName("catalog", "schema", "table"); TransactionManager transactionManager = createTestTransactionManager(); AccessControlManager accessControlManager = new AccessControlManager(transactionManager); - AccessControlContext context = new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty()); + AccessControlContext context = new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty(), WarningCollector.NOOP); accessControlManager.setSystemAccessControl(ReadOnlySystemAccessControl.NAME, ImmutableMap.of()); accessControlManager.checkCanSetUser(identity, context, Optional.of(PRINCIPAL), USER_NAME); @@ -146,7 +147,7 @@ public void testSetAccessControl() accessControlManager.checkCanSetUser( new Identity(USER_NAME, Optional.of(PRINCIPAL)), - new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty()), + new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty(), WarningCollector.NOOP), Optional.of(PRINCIPAL), USER_NAME); assertEquals(accessControlFactory.getCheckedUserName(), USER_NAME); @@ -157,7 +158,7 @@ public void testSetAccessControl() public void testCheckQueryIntegrity() { AccessControlManager accessControlManager = new AccessControlManager(createTestTransactionManager()); - AccessControlContext context = new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty()); + AccessControlContext context = new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty(), WarningCollector.NOOP); TestSystemAccessControlFactory accessControlFactory = new TestSystemAccessControlFactory("test"); accessControlManager.addSystemAccessControlFactory(accessControlFactory); @@ -207,7 +208,7 @@ public void testNoCatalogAccessControl() transaction(transactionManager, accessControlManager) .execute(transactionId -> { accessControlManager.checkCanSelectFromColumns(transactionId, new Identity(USER_NAME, Optional.of(PRINCIPAL)), - new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty()), + new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty(), WarningCollector.NOOP), new QualifiedObjectName("catalog", "schema", "table"), ImmutableSet.of(new Subfield("column"))); }); } @@ -229,7 +230,7 @@ public void testDenyCatalogAccessControl() transaction(transactionManager, accessControlManager) .execute(transactionId -> { accessControlManager.checkCanSelectFromColumns(transactionId, new Identity(USER_NAME, Optional.of(PRINCIPAL)), - new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty()), + new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty(), WarningCollector.NOOP), new QualifiedObjectName("catalog", "schema", "table"), ImmutableSet.of(new Subfield("column"))); }); } @@ -251,7 +252,7 @@ public void testDenySystemAccessControl() transaction(transactionManager, accessControlManager) .execute(transactionId -> { accessControlManager.checkCanSelectFromColumns(transactionId, new Identity(USER_NAME, Optional.of(PRINCIPAL)), - new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty()), + new AccessControlContext(new QueryId(QUERY_ID), Optional.empty(), Optional.empty(), WarningCollector.NOOP), new QualifiedObjectName("secured_catalog", "schema", "table"), ImmutableSet.of(new Subfield("column"))); }); } diff --git a/presto-main/src/test/java/com/facebook/presto/security/TestFileBasedSystemAccessControl.java b/presto-main/src/test/java/com/facebook/presto/security/TestFileBasedSystemAccessControl.java index c7640e7c91c94..09e1ffe2dabbf 100644 --- a/presto-main/src/test/java/com/facebook/presto/security/TestFileBasedSystemAccessControl.java +++ b/presto-main/src/test/java/com/facebook/presto/security/TestFileBasedSystemAccessControl.java @@ -17,6 +17,7 @@ import com.facebook.presto.common.QualifiedObjectName; import com.facebook.presto.spi.QueryId; import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.security.AccessControlContext; import com.facebook.presto.spi.security.AccessDeniedException; import com.facebook.presto.spi.security.Identity; @@ -67,7 +68,7 @@ public class TestFileBasedSystemAccessControl private static final QualifiedObjectName aliceTable = new QualifiedObjectName("alice-catalog", "schema", "table"); private static final QualifiedObjectName aliceView = new QualifiedObjectName("alice-catalog", "schema", "view"); private static final CatalogSchemaName aliceSchema = new CatalogSchemaName("alice-catalog", "schema"); - private static final AccessControlContext context = new AccessControlContext(new QueryId("query_id"), Optional.empty(), Optional.empty()); + private static final AccessControlContext context = new AccessControlContext(new QueryId("query_id"), Optional.empty(), Optional.empty(), WarningCollector.NOOP); @Test public void testCanSetUserOperations() throws IOException { diff --git a/presto-plugin-toolkit/src/test/java/com/facebook/presto/plugin/base/security/TestFileBasedAccessControl.java b/presto-plugin-toolkit/src/test/java/com/facebook/presto/plugin/base/security/TestFileBasedAccessControl.java index 0478e501e0430..f88606ac7a4c5 100644 --- a/presto-plugin-toolkit/src/test/java/com/facebook/presto/plugin/base/security/TestFileBasedAccessControl.java +++ b/presto-plugin-toolkit/src/test/java/com/facebook/presto/plugin/base/security/TestFileBasedAccessControl.java @@ -16,6 +16,7 @@ import com.facebook.presto.common.Subfield; import com.facebook.presto.spi.QueryId; import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.connector.ConnectorAccessControl; import com.facebook.presto.spi.connector.ConnectorTransactionHandle; import com.facebook.presto.spi.security.AccessControlContext; @@ -39,7 +40,7 @@ public class TestFileBasedAccessControl { public static final ConnectorTransactionHandle TRANSACTION_HANDLE = new ConnectorTransactionHandle() {}; - public static final AccessControlContext CONTEXT = new AccessControlContext(new QueryId("query_id"), Optional.empty(), Optional.empty()); + public static final AccessControlContext CONTEXT = new AccessControlContext(new QueryId("query_id"), Optional.empty(), Optional.empty(), WarningCollector.NOOP); @Test public void testSchemaRules() diff --git a/presto-spark-base/src/test/java/com/facebook/presto/spark/PrestoSparkQueryRunner.java b/presto-spark-base/src/test/java/com/facebook/presto/spark/PrestoSparkQueryRunner.java index b303be970fec6..43275f9754da6 100644 --- a/presto-spark-base/src/test/java/com/facebook/presto/spark/PrestoSparkQueryRunner.java +++ b/presto-spark-base/src/test/java/com/facebook/presto/spark/PrestoSparkQueryRunner.java @@ -57,6 +57,7 @@ import com.facebook.presto.spark.execution.AbstractPrestoSparkQueryExecution; import com.facebook.presto.spark.execution.nativeprocess.NativeExecutionModule; import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.eventlistener.EventListener; import com.facebook.presto.spi.function.FunctionImplementationType; import com.facebook.presto.spi.security.PrincipalType; @@ -169,7 +170,7 @@ public class PrestoSparkQueryRunner private final ReadWriteLock lock = new ReentrantReadWriteLock(); - protected static final MetastoreContext METASTORE_CONTEXT = new MetastoreContext("test_user", "test_queryId", Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER); + protected static final MetastoreContext METASTORE_CONTEXT = new MetastoreContext("test_user", "test_queryId", Optional.empty(), Optional.empty(), Optional.empty(), false, HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER, WarningCollector.NOOP); public static PrestoSparkQueryRunner createHivePrestoSparkQueryRunner() { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/security/AccessControlContext.java b/presto-spi/src/main/java/com/facebook/presto/spi/security/AccessControlContext.java index 0e6a406fd42fc..fff34b1f8c06d 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/security/AccessControlContext.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/security/AccessControlContext.java @@ -14,6 +14,7 @@ package com.facebook.presto.spi.security; import com.facebook.presto.spi.QueryId; +import com.facebook.presto.spi.WarningCollector; import java.util.Optional; @@ -24,12 +25,14 @@ public class AccessControlContext private final QueryId queryId; private final Optional clientInfo; private final Optional source; + private final WarningCollector warningCollector; - public AccessControlContext(QueryId queryId, Optional clientInfo, Optional source) + public AccessControlContext(QueryId queryId, Optional clientInfo, Optional source, WarningCollector warningCollector) { this.queryId = requireNonNull(queryId, "queryId is null"); this.clientInfo = requireNonNull(clientInfo, "clientInfo is null"); this.source = requireNonNull(source, "source is null"); + this.warningCollector = requireNonNull(warningCollector, "warningCollector is null"); } public QueryId getQueryId() @@ -46,4 +49,9 @@ public Optional getSource() { return source; } + + public WarningCollector getWarningCollector() + { + return warningCollector; + } }