diff --git a/core/trino-main/src/main/java/io/trino/connector/ConnectorManager.java b/core/trino-main/src/main/java/io/trino/connector/ConnectorManager.java index d480474486c3..ad59a5293f95 100644 --- a/core/trino-main/src/main/java/io/trino/connector/ConnectorManager.java +++ b/core/trino-main/src/main/java/io/trino/connector/ConnectorManager.java @@ -48,6 +48,7 @@ import io.trino.spi.connector.ConnectorRecordSetProvider; import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.SystemTable; +import io.trino.spi.connector.TableProcedureMetadata; import io.trino.spi.eventlistener.EventListener; import io.trino.spi.procedure.Procedure; import io.trino.spi.session.PropertyMetadata; @@ -299,6 +300,8 @@ private synchronized void addConnectorInternal(MaterializedConnector connector) .ifPresent(partitioningProvider -> nodePartitioningManager.addPartitioningProvider(catalogName, partitioningProvider)); metadataManager.getProcedureRegistry().addProcedures(catalogName, connector.getProcedures()); + Set tableProcedures = connector.getTableProcedures(); + metadataManager.getTableProcedureRegistry().addTableProcedures(catalogName, tableProcedures); connector.getAccessControl() .ifPresent(accessControl -> accessControlManager.addCatalogAccessControl(catalogName, accessControl)); @@ -308,6 +311,9 @@ private synchronized void addConnectorInternal(MaterializedConnector connector) metadataManager.getColumnPropertyManager().addProperties(catalogName, connector.getColumnProperties()); metadataManager.getSchemaPropertyManager().addProperties(catalogName, connector.getSchemaProperties()); metadataManager.getAnalyzePropertyManager().addProperties(catalogName, connector.getAnalyzeProperties()); + for (TableProcedureMetadata tableProcedure : tableProcedures) { + metadataManager.getTableProceduresPropertyManager().addProperties(catalogName, tableProcedure.getName(), tableProcedure.getProperties()); + } metadataManager.getSessionPropertyManager().addConnectorSessionProperties(catalogName, connector.getSessionProperties()); } @@ -332,12 +338,14 @@ private synchronized void removeConnectorInternal(CatalogName catalogName) indexManager.removeIndexProvider(catalogName); nodePartitioningManager.removePartitioningProvider(catalogName); metadataManager.getProcedureRegistry().removeProcedures(catalogName); + metadataManager.getTableProcedureRegistry().removeProcedures(catalogName); accessControlManager.removeCatalogAccessControl(catalogName); metadataManager.getTablePropertyManager().removeProperties(catalogName); metadataManager.getMaterializedViewPropertyManager().removeProperties(catalogName); metadataManager.getColumnPropertyManager().removeProperties(catalogName); metadataManager.getSchemaPropertyManager().removeProperties(catalogName); metadataManager.getAnalyzePropertyManager().removeProperties(catalogName); + metadataManager.getTableProceduresPropertyManager().removeProperties(catalogName); metadataManager.getSessionPropertyManager().removeConnectorSessionProperties(catalogName); MaterializedConnector materializedConnector = connectors.remove(catalogName); @@ -401,6 +409,7 @@ private static class MaterializedConnector private final Connector connector; private final Set systemTables; private final Set procedures; + private final Set tableProcedures; private final Optional splitManager; private final Optional pageSourceProvider; private final Optional pageSinkProvider; @@ -428,6 +437,10 @@ public MaterializedConnector(CatalogName catalogName, Connector connector) requireNonNull(procedures, format("Connector '%s' returned a null procedures set", catalogName)); this.procedures = ImmutableSet.copyOf(procedures); + Set tableProcedures = connector.getTableProcedures(); + requireNonNull(procedures, format("Connector '%s' returned a null table procedures set", catalogName)); + this.tableProcedures = ImmutableSet.copyOf(tableProcedures); + ConnectorSplitManager splitManager = null; try { splitManager = connector.getSplitManager(); @@ -538,6 +551,11 @@ public Set getProcedures() return procedures; } + public Set getTableProcedures() + { + return tableProcedures; + } + public Optional getSplitManager() { return splitManager; diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlQueryExecution.java b/core/trino-main/src/main/java/io/trino/execution/SqlQueryExecution.java index 9f478b2fe4fe..331cdf4c3a59 100644 --- a/core/trino-main/src/main/java/io/trino/execution/SqlQueryExecution.java +++ b/core/trino-main/src/main/java/io/trino/execution/SqlQueryExecution.java @@ -132,6 +132,7 @@ public class SqlQueryExecution private final StatsCalculator statsCalculator; private final CostCalculator costCalculator; private final DynamicFilterService dynamicFilterService; + private final TableExecuteContextManager tableExecuteContextManager; private SqlQueryExecution( PreparedQuery preparedQuery, @@ -159,7 +160,8 @@ private SqlQueryExecution( StatsCalculator statsCalculator, CostCalculator costCalculator, DynamicFilterService dynamicFilterService, - WarningCollector warningCollector) + WarningCollector warningCollector, + TableExecuteContextManager tableExecuteContextManager) { try (SetThreadName ignored = new SetThreadName("Query-%s", stateMachine.getQueryId())) { this.slug = requireNonNull(slug, "slug is null"); @@ -180,6 +182,7 @@ private SqlQueryExecution( this.statsCalculator = requireNonNull(statsCalculator, "statsCalculator is null"); this.costCalculator = requireNonNull(costCalculator, "costCalculator is null"); this.dynamicFilterService = requireNonNull(dynamicFilterService, "dynamicFilterService is null"); + this.tableExecuteContextManager = requireNonNull(tableExecuteContextManager, "tableExecuteContextManager is null"); checkArgument(scheduleSplitBatchSize > 0, "scheduleSplitBatchSize must be greater than 0"); this.scheduleSplitBatchSize = scheduleSplitBatchSize; @@ -195,6 +198,8 @@ private SqlQueryExecution( } unregisterDynamicFilteringQuery( dynamicFilterService.getDynamicFilteringStats(stateMachine.getQueryId(), stateMachine.getSession())); + + tableExecuteContextManager.unregisterTableExecuteContextForQuery(stateMachine.getQueryId()); }); // when the query finishes cache the final query info, and clear the reference to the output stage @@ -423,6 +428,8 @@ public void start() } } + tableExecuteContextManager.registerTableExecuteContextForQuery(getQueryId()); + if (!stateMachine.transitionToStarting()) { // query already started or finished return; @@ -544,7 +551,8 @@ private void planDistribution(PlanRoot plan) nodeTaskMap, executionPolicy, schedulerStats, - dynamicFilterService); + dynamicFilterService, + tableExecuteContextManager); queryScheduler.set(scheduler); @@ -741,6 +749,7 @@ public static class SqlQueryExecutionFactory private final StatsCalculator statsCalculator; private final CostCalculator costCalculator; private final DynamicFilterService dynamicFilterService; + private final TableExecuteContextManager tableExecuteContextManager; @Inject SqlQueryExecutionFactory( @@ -765,7 +774,8 @@ public static class SqlQueryExecutionFactory SplitSchedulerStats schedulerStats, StatsCalculator statsCalculator, CostCalculator costCalculator, - DynamicFilterService dynamicFilterService) + DynamicFilterService dynamicFilterService, + TableExecuteContextManager tableExecuteContextManager) { requireNonNull(config, "config is null"); this.schedulerStats = requireNonNull(schedulerStats, "schedulerStats is null"); @@ -790,6 +800,7 @@ public static class SqlQueryExecutionFactory this.statsCalculator = requireNonNull(statsCalculator, "statsCalculator is null"); this.costCalculator = requireNonNull(costCalculator, "costCalculator is null"); this.dynamicFilterService = requireNonNull(dynamicFilterService, "dynamicFilterService is null"); + this.tableExecuteContextManager = requireNonNull(tableExecuteContextManager, "tableExecuteContextManager is null"); } @Override @@ -829,7 +840,8 @@ public QueryExecution createQueryExecution( statsCalculator, costCalculator, dynamicFilterService, - warningCollector); + warningCollector, + tableExecuteContextManager); } } } diff --git a/core/trino-main/src/main/java/io/trino/execution/TableExecuteContext.java b/core/trino-main/src/main/java/io/trino/execution/TableExecuteContext.java new file mode 100644 index 000000000000..ff0b796c8018 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/execution/TableExecuteContext.java @@ -0,0 +1,45 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.execution; + +import com.google.common.collect.ImmutableList; + +import javax.annotation.concurrent.GuardedBy; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class TableExecuteContext +{ + @GuardedBy("this") + private List splitsInfo; + + public synchronized void setSplitsInfo(List splitsInfo) + { + requireNonNull(splitsInfo, "splitsInfo is null"); + if (this.splitsInfo != null) { + throw new IllegalStateException("splitsInfo already set to " + this.splitsInfo); + } + this.splitsInfo = ImmutableList.copyOf(splitsInfo); + } + + public synchronized List getSplitsInfo() + { + if (splitsInfo == null) { + throw new IllegalStateException("splitsInfo not set yet"); + } + return splitsInfo; + } +} diff --git a/core/trino-main/src/main/java/io/trino/execution/TableExecuteContextManager.java b/core/trino-main/src/main/java/io/trino/execution/TableExecuteContextManager.java new file mode 100644 index 000000000000..aa85c44f52de --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/execution/TableExecuteContextManager.java @@ -0,0 +1,49 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.execution; + +import io.trino.spi.QueryId; + +import javax.annotation.concurrent.ThreadSafe; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +@ThreadSafe +public class TableExecuteContextManager +{ + private final ConcurrentMap contexts = new ConcurrentHashMap<>(); + + public void registerTableExecuteContextForQuery(QueryId queryId) + { + TableExecuteContext newContext = new TableExecuteContext(); + if (contexts.putIfAbsent(queryId, newContext) != null) { + throw new IllegalStateException("TableExecuteContext already registered for query " + queryId); + } + } + + public void unregisterTableExecuteContextForQuery(QueryId queryId) + { + contexts.remove(queryId); + } + + public TableExecuteContext getTableExecuteContextForQuery(QueryId queryId) + { + TableExecuteContext context = contexts.get(queryId); + if (context == null) { + throw new IllegalStateException("TableExecuteContext not registered for query " + queryId); + } + return context; + } +} diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java index 8424f7c9a44d..6cc76e0d83a6 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java @@ -21,6 +21,7 @@ import io.trino.execution.Lifespan; import io.trino.execution.RemoteTask; import io.trino.execution.SqlStageExecution; +import io.trino.execution.TableExecuteContextManager; import io.trino.execution.scheduler.ScheduleResult.BlockedReason; import io.trino.execution.scheduler.group.DynamicLifespanScheduler; import io.trino.execution.scheduler.group.FixedLifespanScheduler; @@ -75,13 +76,15 @@ public FixedSourcePartitionedScheduler( OptionalInt concurrentLifespansPerTask, NodeSelector nodeSelector, List partitionHandles, - DynamicFilterService dynamicFilterService) + DynamicFilterService dynamicFilterService, + TableExecuteContextManager tableExecuteContextManager) { requireNonNull(stage, "stage is null"); requireNonNull(splitSources, "splitSources is null"); requireNonNull(bucketNodeMap, "bucketNodeMap is null"); checkArgument(!requireNonNull(nodes, "nodes is null").isEmpty(), "nodes is empty"); requireNonNull(partitionHandles, "partitionHandles is null"); + requireNonNull(tableExecuteContextManager, "tableExecuteContextManager is null"); this.stage = stage; this.nodes = ImmutableList.copyOf(nodes); @@ -119,6 +122,7 @@ public FixedSourcePartitionedScheduler( Math.max(splitBatchSize / concurrentLifespans, 1), groupedExecutionForScanNode, dynamicFilterService, + tableExecuteContextManager, () -> true); if (stageExecutionDescriptor.isStageGroupedExecution() && !groupedExecutionForScanNode) { diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/SourcePartitionedScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/SourcePartitionedScheduler.java index 1697ade417b4..8f98981569cb 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/SourcePartitionedScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/SourcePartitionedScheduler.java @@ -23,6 +23,8 @@ import io.trino.execution.Lifespan; import io.trino.execution.RemoteTask; import io.trino.execution.SqlStageExecution; +import io.trino.execution.TableExecuteContext; +import io.trino.execution.TableExecuteContextManager; import io.trino.execution.scheduler.FixedSourcePartitionedScheduler.BucketedSplitPlacementPolicy; import io.trino.metadata.InternalNode; import io.trino.metadata.Split; @@ -40,6 +42,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ThreadLocalRandom; import java.util.function.BooleanSupplier; @@ -96,6 +99,7 @@ private enum State private final PlanNodeId partitionedNode; private final boolean groupedExecution; private final DynamicFilterService dynamicFilterService; + private final TableExecuteContextManager tableExecuteContextManager; private final BooleanSupplier anySourceTaskBlocked; private final Map scheduleGroups = new HashMap<>(); @@ -112,6 +116,7 @@ private SourcePartitionedScheduler( int splitBatchSize, boolean groupedExecution, DynamicFilterService dynamicFilterService, + TableExecuteContextManager tableExecuteContextManager, BooleanSupplier anySourceTaskBlocked) { this.stage = requireNonNull(stage, "stage is null"); @@ -119,6 +124,7 @@ private SourcePartitionedScheduler( this.splitSource = requireNonNull(splitSource, "splitSource is null"); this.splitPlacementPolicy = requireNonNull(splitPlacementPolicy, "splitPlacementPolicy is null"); this.dynamicFilterService = requireNonNull(dynamicFilterService, "dynamicFilterService is null"); + this.tableExecuteContextManager = requireNonNull(tableExecuteContextManager, "tableExecuteContextManager is null"); this.anySourceTaskBlocked = requireNonNull(anySourceTaskBlocked, "anySourceTaskBlocked is null"); checkArgument(splitBatchSize > 0, "splitBatchSize must be at least one"); @@ -146,6 +152,7 @@ public static StageScheduler newSourcePartitionedSchedulerAsStageScheduler( SplitPlacementPolicy splitPlacementPolicy, int splitBatchSize, DynamicFilterService dynamicFilterService, + TableExecuteContextManager tableExecuteContextManager, BooleanSupplier anySourceTaskBlocked) { SourcePartitionedScheduler sourcePartitionedScheduler = new SourcePartitionedScheduler( @@ -156,6 +163,7 @@ public static StageScheduler newSourcePartitionedSchedulerAsStageScheduler( splitBatchSize, false, dynamicFilterService, + tableExecuteContextManager, anySourceTaskBlocked); sourcePartitionedScheduler.startLifespan(Lifespan.taskWide(), NOT_PARTITIONED); sourcePartitionedScheduler.noMoreLifespans(); @@ -197,6 +205,7 @@ public static SourceScheduler newSourcePartitionedSchedulerAsSourceScheduler( int splitBatchSize, boolean groupedExecution, DynamicFilterService dynamicFilterService, + TableExecuteContextManager tableExecuteContextManager, BooleanSupplier anySourceTaskBlocked) { return new SourcePartitionedScheduler( @@ -207,6 +216,7 @@ public static SourceScheduler newSourcePartitionedSchedulerAsSourceScheduler( splitBatchSize, groupedExecution, dynamicFilterService, + tableExecuteContextManager, anySourceTaskBlocked); } @@ -357,6 +367,16 @@ else if (pendingSplits.isEmpty()) { throw new IllegalStateException("At least 1 split should have been scheduled for this plan node"); case SPLITS_ADDED: state = State.NO_MORE_SPLITS; + + Optional> tableExecuteSplitsInfo = splitSource.getTableExecuteSplitsInfo(); + // Here we assume that we can get non-empty tableExecuteSplitsInfo only for queries which facilitate single split-source. + // If we had more than one split-source and got multiple tableExecuteSplitsInfos the call to tableExecuteContext.setSplitsInfo will + // throw an error. + tableExecuteSplitsInfo.ifPresent(info -> { + TableExecuteContext tableExecuteContext = tableExecuteContextManager.getTableExecuteContextForQuery(stage.getStageId().getQueryId()); + tableExecuteContext.setSplitsInfo(info); + }); + splitSource.close(); // fall through case NO_MORE_SPLITS: diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/SqlQueryScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/SqlQueryScheduler.java index d20407c34745..e800b98fbe73 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/SqlQueryScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/SqlQueryScheduler.java @@ -36,6 +36,7 @@ import io.trino.execution.StageId; import io.trino.execution.StageInfo; import io.trino.execution.StageState; +import io.trino.execution.TableExecuteContextManager; import io.trino.execution.TaskStatus; import io.trino.execution.buffer.OutputBuffers; import io.trino.execution.buffer.OutputBuffers.OutputBufferId; @@ -121,6 +122,7 @@ public class SqlQueryScheduler private final SplitSchedulerStats schedulerStats; private final boolean summarizeTaskInfo; private final DynamicFilterService dynamicFilterService; + private final TableExecuteContextManager tableExecuteContextManager; private final AtomicBoolean started = new AtomicBoolean(); public static SqlQueryScheduler createSqlQueryScheduler( @@ -139,7 +141,8 @@ public static SqlQueryScheduler createSqlQueryScheduler( NodeTaskMap nodeTaskMap, ExecutionPolicy executionPolicy, SplitSchedulerStats schedulerStats, - DynamicFilterService dynamicFilterService) + DynamicFilterService dynamicFilterService, + TableExecuteContextManager tableExecuteContextManager) { SqlQueryScheduler sqlQueryScheduler = new SqlQueryScheduler( queryStateMachine, @@ -157,7 +160,8 @@ public static SqlQueryScheduler createSqlQueryScheduler( nodeTaskMap, executionPolicy, schedulerStats, - dynamicFilterService); + dynamicFilterService, + tableExecuteContextManager); sqlQueryScheduler.initialize(); return sqlQueryScheduler; } @@ -178,13 +182,15 @@ private SqlQueryScheduler( NodeTaskMap nodeTaskMap, ExecutionPolicy executionPolicy, SplitSchedulerStats schedulerStats, - DynamicFilterService dynamicFilterService) + DynamicFilterService dynamicFilterService, + TableExecuteContextManager tableExecuteContextManager) { this.queryStateMachine = requireNonNull(queryStateMachine, "queryStateMachine is null"); this.executionPolicy = requireNonNull(executionPolicy, "executionPolicy is null"); this.schedulerStats = requireNonNull(schedulerStats, "schedulerStats is null"); this.summarizeTaskInfo = summarizeTaskInfo; this.dynamicFilterService = requireNonNull(dynamicFilterService, "dynamicFilterService is null"); + this.tableExecuteContextManager = requireNonNull(tableExecuteContextManager, "tableExecuteContextManager is null"); // todo come up with a better way to build this, or eliminate this map ImmutableMap.Builder stageSchedulers = ImmutableMap.builder(); @@ -363,6 +369,7 @@ private List createStages( placementPolicy, splitBatchSize, dynamicFilterService, + tableExecuteContextManager, () -> childStages.stream().anyMatch(SqlStageExecution::isAnyTaskBlocked))); } else if (partitioningHandle.equals(SCALED_WRITER_DISTRIBUTION)) { @@ -441,7 +448,8 @@ else if (partitioningHandle.equals(SCALED_WRITER_DISTRIBUTION)) { getConcurrentLifespansPerNode(session), nodeScheduler.createNodeSelector(session, catalogName), connectorPartitionHandles, - dynamicFilterService)); + dynamicFilterService, + tableExecuteContextManager)); } else { // all sources are remote diff --git a/core/trino-main/src/main/java/io/trino/metadata/AbstractCatalogPropertyManager.java b/core/trino-main/src/main/java/io/trino/metadata/AbstractCatalogPropertyManager.java new file mode 100644 index 000000000000..0c8432950838 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/metadata/AbstractCatalogPropertyManager.java @@ -0,0 +1,75 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.metadata; + +import io.trino.Session; +import io.trino.connector.CatalogName; +import io.trino.security.AccessControl; +import io.trino.spi.ErrorCodeSupplier; +import io.trino.spi.session.PropertyMetadata; +import io.trino.sql.tree.Expression; +import io.trino.sql.tree.NodeRef; +import io.trino.sql.tree.Parameter; + +import java.util.List; +import java.util.Map; + +abstract class AbstractCatalogPropertyManager + extends AbstractPropertyManager +{ + protected AbstractCatalogPropertyManager(String propertyType, ErrorCodeSupplier propertyError) + { + super(propertyType, propertyError); + } + + public final void addProperties(CatalogName catalogName, List> properties) + { + innerAddProperties(catalogName, properties); + } + + public final void removeProperties(CatalogName catalogName) + { + innerRemoveProperties(catalogName); + } + + public final Map getProperties( + CatalogName catalog, + String catalogName, // only use this for error messages + Map sqlPropertyValues, + Session session, + Metadata metadata, + AccessControl accessControl, + Map, Expression> parameters) + { + return innerGetProperties( + catalog, + catalogName, + sqlPropertyValues, + session, + metadata, + accessControl, + parameters); + } + + public Map>> getAllProperties() + { + return innerGetAllProperties(); + } + + @Override + protected String formatPropertiesKeyForMessage(String catalogName, CatalogName ignored) + { + return "Catalog '" + catalogName + "'"; + } +} diff --git a/core/trino-main/src/main/java/io/trino/metadata/AbstractPropertyManager.java b/core/trino-main/src/main/java/io/trino/metadata/AbstractPropertyManager.java index 5acbd549a2b7..f782d0ac19f7 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/AbstractPropertyManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/AbstractPropertyManager.java @@ -16,7 +16,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import io.trino.Session; -import io.trino.connector.CatalogName; import io.trino.security.AccessControl; import io.trino.spi.ErrorCodeSupplier; import io.trino.spi.TrinoException; @@ -42,9 +41,9 @@ import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; -abstract class AbstractPropertyManager +abstract class AbstractPropertyManager { - private final ConcurrentMap>> connectorProperties = new ConcurrentHashMap<>(); + protected final ConcurrentMap>> connectorProperties = new ConcurrentHashMap<>(); private final String propertyType; private final ErrorCodeSupplier propertyError; @@ -55,33 +54,33 @@ protected AbstractPropertyManager(String propertyType, ErrorCodeSupplier propert this.propertyError = requireNonNull(propertyError, "propertyError is null"); } - public final void addProperties(CatalogName catalogName, List> properties) + protected final void innerAddProperties(K propertiesKey, List> properties) { - requireNonNull(catalogName, "catalogName is null"); + requireNonNull(propertiesKey, "propertiesKey is null"); requireNonNull(properties, "properties is null"); Map> propertiesByName = Maps.uniqueIndex(properties, PropertyMetadata::getName); - checkState(connectorProperties.putIfAbsent(catalogName, propertiesByName) == null, "Properties for connector '%s' are already registered", catalogName); + checkState(connectorProperties.putIfAbsent(propertiesKey, propertiesByName) == null, "Properties for key %s are already registered", propertiesKey); } - public final void removeProperties(CatalogName catalogName) + protected final void innerRemoveProperties(K propertiesKey) { - connectorProperties.remove(catalogName); + connectorProperties.remove(propertiesKey); } - public final Map getProperties( - CatalogName catalogName, - String catalog, // only use this for error messages + protected final Map innerGetProperties( + K propertiesKey, + String catalogName, // only use this for error messages Map sqlPropertyValues, Session session, Metadata metadata, AccessControl accessControl, Map, Expression> parameters) { - Map> supportedProperties = connectorProperties.get(catalogName); + Map> supportedProperties = connectorProperties.get(propertiesKey); if (supportedProperties == null) { - throw new TrinoException(NOT_FOUND, "Catalog not found: " + catalog); + throw new TrinoException(NOT_FOUND, formatPropertiesKeyForMessage(catalogName, propertiesKey) + " not found"); } ImmutableMap.Builder properties = ImmutableMap.builder(); @@ -93,8 +92,8 @@ public final Map getProperties( if (property == null) { throw new TrinoException( propertyError, - format("Catalog '%s' does not support %s property '%s'", - catalog, + format("%s does not support %s property '%s'", + formatPropertiesKeyForMessage(catalogName, propertiesKey), propertyType, propertyName)); } @@ -146,7 +145,7 @@ public final Map getProperties( return properties.build(); } - public Map>> getAllProperties() + protected Map>> innerGetAllProperties() { return ImmutableMap.copyOf(connectorProperties); } @@ -172,4 +171,6 @@ private Object evaluatePropertyValue( } return objectValue; } + + protected abstract String formatPropertiesKeyForMessage(String catalogName, K propertiesKey); } diff --git a/core/trino-main/src/main/java/io/trino/metadata/AnalyzePropertyManager.java b/core/trino-main/src/main/java/io/trino/metadata/AnalyzePropertyManager.java index f31d733310d4..98cf098160b8 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/AnalyzePropertyManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/AnalyzePropertyManager.java @@ -16,7 +16,7 @@ import static io.trino.spi.StandardErrorCode.INVALID_ANALYZE_PROPERTY; public class AnalyzePropertyManager - extends AbstractPropertyManager + extends AbstractCatalogPropertyManager { public AnalyzePropertyManager() { diff --git a/core/trino-main/src/main/java/io/trino/metadata/ColumnPropertyManager.java b/core/trino-main/src/main/java/io/trino/metadata/ColumnPropertyManager.java index 949d4e44834b..6db2ef32ede2 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/ColumnPropertyManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/ColumnPropertyManager.java @@ -16,7 +16,7 @@ import static io.trino.spi.StandardErrorCode.INVALID_COLUMN_PROPERTY; public class ColumnPropertyManager - extends AbstractPropertyManager + extends AbstractCatalogPropertyManager { public ColumnPropertyManager() { diff --git a/core/trino-main/src/main/java/io/trino/metadata/HandleJsonModule.java b/core/trino-main/src/main/java/io/trino/metadata/HandleJsonModule.java index 8ffd3f94a8e8..8e8c3b405b76 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/HandleJsonModule.java +++ b/core/trino-main/src/main/java/io/trino/metadata/HandleJsonModule.java @@ -23,6 +23,7 @@ import io.trino.spi.connector.ConnectorOutputTableHandle; import io.trino.spi.connector.ConnectorPartitioningHandle; import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorTableExecuteHandle; import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.ConnectorTableLayoutHandle; import io.trino.spi.connector.ConnectorTransactionHandle; @@ -72,6 +73,12 @@ public static com.fasterxml.jackson.databind.Module insertTableHandleModule(Hand return new AbstractTypedJacksonModule<>(ConnectorInsertTableHandle.class, resolver::getId, resolver::getInsertTableHandleClass) {}; } + @ProvidesIntoSet + public static com.fasterxml.jackson.databind.Module tableExecuteHandleModule(HandleResolver resolver) + { + return new AbstractTypedJacksonModule<>(ConnectorTableExecuteHandle.class, resolver::getId, resolver::getTableExecuteHandleClass) {}; + } + @ProvidesIntoSet public static com.fasterxml.jackson.databind.Module indexHandleModule(HandleResolver resolver) { diff --git a/core/trino-main/src/main/java/io/trino/metadata/HandleResolver.java b/core/trino-main/src/main/java/io/trino/metadata/HandleResolver.java index b2b61a372192..bd8e834f2a27 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/HandleResolver.java +++ b/core/trino-main/src/main/java/io/trino/metadata/HandleResolver.java @@ -22,6 +22,7 @@ import io.trino.spi.connector.ConnectorOutputTableHandle; import io.trino.spi.connector.ConnectorPartitioningHandle; import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorTableExecuteHandle; import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.ConnectorTableLayoutHandle; import io.trino.spi.connector.ConnectorTransactionHandle; @@ -103,6 +104,11 @@ public String getId(ConnectorInsertTableHandle insertHandle) return getId(insertHandle, MaterializedHandleResolver::getInsertTableHandleClass); } + public String getId(ConnectorTableExecuteHandle tableExecuteHandle) + { + return getId(tableExecuteHandle, MaterializedHandleResolver::getTableExecuteHandleClass); + } + public String getId(ConnectorPartitioningHandle partitioningHandle) { return getId(partitioningHandle, MaterializedHandleResolver::getPartitioningHandleClass); @@ -148,6 +154,11 @@ public Class getInsertTableHandleClass(Str return resolverFor(id).getInsertTableHandleClass().orElseThrow(() -> new IllegalArgumentException("No resolver for " + id)); } + public Class getTableExecuteHandleClass(String id) + { + return resolverFor(id).getTableExecuteHandleClass().orElseThrow(() -> new IllegalArgumentException("No resolver for " + id)); + } + public Class getPartitioningHandleClass(String id) { return resolverFor(id).getPartitioningHandleClass().orElseThrow(() -> new IllegalArgumentException("No resolver for " + id)); @@ -188,6 +199,7 @@ private static class MaterializedHandleResolver private final Optional> indexHandle; private final Optional> outputTableHandle; private final Optional> insertTableHandle; + private final Optional> tableExecuteHandle; private final Optional> partitioningHandle; private final Optional> transactionHandle; @@ -200,6 +212,7 @@ public MaterializedHandleResolver(ConnectorHandleResolver resolver) indexHandle = getHandleClass(resolver::getIndexHandleClass); outputTableHandle = getHandleClass(resolver::getOutputTableHandleClass); insertTableHandle = getHandleClass(resolver::getInsertTableHandleClass); + tableExecuteHandle = getHandleClass(resolver::getTableExecuteHandleClass); partitioningHandle = getHandleClass(resolver::getPartitioningHandleClass); transactionHandle = getHandleClass(resolver::getTransactionHandleClass); } @@ -249,6 +262,11 @@ public Optional> getInsertTableHandl return insertTableHandle; } + public Optional> getTableExecuteHandleClass() + { + return tableExecuteHandle; + } + public Optional> getPartitioningHandleClass() { return partitioningHandle; @@ -276,6 +294,7 @@ public boolean equals(Object o) Objects.equals(indexHandle, that.indexHandle) && Objects.equals(outputTableHandle, that.outputTableHandle) && Objects.equals(insertTableHandle, that.insertTableHandle) && + Objects.equals(tableExecuteHandle, that.tableExecuteHandle) && Objects.equals(partitioningHandle, that.partitioningHandle) && Objects.equals(transactionHandle, that.transactionHandle); } @@ -283,7 +302,7 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(tableHandle, layoutHandle, columnHandle, split, indexHandle, outputTableHandle, insertTableHandle, partitioningHandle, transactionHandle); + return Objects.hash(tableHandle, layoutHandle, columnHandle, split, indexHandle, outputTableHandle, insertTableHandle, tableExecuteHandle, partitioningHandle, transactionHandle); } } } diff --git a/core/trino-main/src/main/java/io/trino/metadata/MaterializedViewPropertyManager.java b/core/trino-main/src/main/java/io/trino/metadata/MaterializedViewPropertyManager.java index d1b0179f923f..374008668892 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/MaterializedViewPropertyManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/MaterializedViewPropertyManager.java @@ -16,7 +16,7 @@ import static io.trino.spi.StandardErrorCode.INVALID_MATERIALIZED_VIEW_PROPERTY; public class MaterializedViewPropertyManager - extends AbstractPropertyManager + extends AbstractCatalogPropertyManager { public MaterializedViewPropertyManager() { diff --git a/core/trino-main/src/main/java/io/trino/metadata/Metadata.java b/core/trino-main/src/main/java/io/trino/metadata/Metadata.java index 01efbbc8757a..603616f423f7 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/Metadata.java +++ b/core/trino-main/src/main/java/io/trino/metadata/Metadata.java @@ -97,6 +97,19 @@ public interface Metadata Optional getTableHandleForStatisticsCollection(Session session, QualifiedObjectName tableName, Map analyzeProperties); + Optional getTableHandleForExecute( + Session session, + TableHandle tableHandle, + String procedureName, + Map executeProperties, + Constraint constraint); + + Optional getLayoutForTableExecute(Session session, TableExecuteHandle tableExecuteHandle); + + TableExecuteHandle beginTableExecute(Session session, TableExecuteHandle handle); + + void finishTableExecute(Session session, TableExecuteHandle handle, Collection fragments, List tableExecuteState); + @Deprecated Optional getLayout(Session session, TableHandle tableHandle, Constraint constraint, Optional> desiredColumns); @@ -638,6 +651,8 @@ default ResolvedFunction getCoercion(Type fromType, Type toType) ProcedureRegistry getProcedureRegistry(); + TableProceduresRegistry getTableProcedureRegistry(); + // // Blocks // @@ -660,6 +675,8 @@ default ResolvedFunction getCoercion(Type fromType, Type toType) AnalyzePropertyManager getAnalyzePropertyManager(); + TableProceduresPropertyManager getTableProceduresPropertyManager(); + /** * Creates the specified materialized view with the specified view definition. */ diff --git a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java index 894677c4eb12..bc26fc1856e0 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java @@ -71,6 +71,7 @@ import io.trino.spi.connector.ConnectorPartitioningHandle; import io.trino.spi.connector.ConnectorResolvedIndex; import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTableExecuteHandle; import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.ConnectorTableLayout; import io.trino.spi.connector.ConnectorTableLayoutHandle; @@ -210,12 +211,14 @@ public final class MetadataManager private final TypeOperators typeOperators; private final FunctionResolver functionResolver; private final ProcedureRegistry procedures; + private final TableProceduresRegistry tableProcedures; private final SessionPropertyManager sessionPropertyManager; private final SchemaPropertyManager schemaPropertyManager; private final TablePropertyManager tablePropertyManager; private final MaterializedViewPropertyManager materializedViewPropertyManager; private final ColumnPropertyManager columnPropertyManager; private final AnalyzePropertyManager analyzePropertyManager; + private final TableProceduresPropertyManager tableProceduresPropertyManager; private final SystemSecurityMetadata systemSecurityMetadata; private final TransactionManager transactionManager; private final TypeRegistry typeRegistry; @@ -237,6 +240,7 @@ public MetadataManager( MaterializedViewPropertyManager materializedViewPropertyManager, ColumnPropertyManager columnPropertyManager, AnalyzePropertyManager analyzePropertyManager, + TableProceduresPropertyManager tableProceduresPropertyManager, SystemSecurityMetadata systemSecurityMetadata, TransactionManager transactionManager, TypeOperators typeOperators, @@ -249,12 +253,14 @@ public MetadataManager( functionResolver = new FunctionResolver(this); this.procedures = new ProcedureRegistry(); + this.tableProcedures = new TableProceduresRegistry(); this.sessionPropertyManager = requireNonNull(sessionPropertyManager, "sessionPropertyManager is null"); this.schemaPropertyManager = requireNonNull(schemaPropertyManager, "schemaPropertyManager is null"); this.tablePropertyManager = requireNonNull(tablePropertyManager, "tablePropertyManager is null"); this.materializedViewPropertyManager = requireNonNull(materializedViewPropertyManager, "materializedViewPropertyManager is null"); this.columnPropertyManager = requireNonNull(columnPropertyManager, "columnPropertyManager is null"); this.analyzePropertyManager = requireNonNull(analyzePropertyManager, "analyzePropertyManager is null"); + this.tableProceduresPropertyManager = requireNonNull(tableProceduresPropertyManager, "tableProceduresPropertyManager is null"); this.systemSecurityMetadata = requireNonNull(systemSecurityMetadata, "systemSecurityMetadata is null"); this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); this.typeOperators = requireNonNull(typeOperators, "typeOperators is null"); @@ -329,6 +335,7 @@ public static MetadataManager createTestMetadataManager(TransactionManager trans new MaterializedViewPropertyManager(), new ColumnPropertyManager(), new AnalyzePropertyManager(), + new TableProceduresPropertyManager(), new DisabledSystemSecurityMetadata(), transactionManager, typeOperators, @@ -429,6 +436,61 @@ public Optional getTableHandleForStatisticsCollection(Session sessi return Optional.empty(); } + @Override + public Optional getTableHandleForExecute(Session session, TableHandle tableHandle, String procedure, Map executeProperties, Constraint constraint) + { + requireNonNull(session, "session is null"); + requireNonNull(tableHandle, "tableHandle is null"); + requireNonNull(procedure, "procedure is null"); + requireNonNull(executeProperties, "executeProperties is null"); + requireNonNull(constraint, "constraint is null"); + + CatalogName catalogName = tableHandle.getCatalogName(); + CatalogMetadata catalogMetadata = getCatalogMetadata(session, catalogName); + ConnectorMetadata metadata = catalogMetadata.getMetadataFor(catalogName); + + Optional executeHandle = metadata.getTableHandleForExecute( + session.toConnectorSession(catalogName), + tableHandle.getConnectorHandle(), + procedure, + executeProperties, + constraint); + + return executeHandle.map(handle -> new TableExecuteHandle( + catalogName, + tableHandle.getTransaction(), + handle)); + } + + @Override + public Optional getLayoutForTableExecute(Session session, TableExecuteHandle tableExecuteHandle) + { + CatalogName catalogName = tableExecuteHandle.getCatalogName(); + CatalogMetadata catalogMetadata = getCatalogMetadataForWrite(session, catalogName); + ConnectorMetadata metadata = catalogMetadata.getMetadata(); + + return metadata.getLayoutForTableExecute(session.toConnectorSession(catalogName), tableExecuteHandle.getConnectorHandle()) + .map(layout -> new NewTableLayout(catalogName, catalogMetadata.getTransactionHandleFor(catalogName), layout)); + } + + @Override + public TableExecuteHandle beginTableExecute(Session session, TableExecuteHandle tableExecuteHandle) + { + CatalogName catalogName = tableExecuteHandle.getCatalogName(); + CatalogMetadata catalogMetadata = getCatalogMetadataForWrite(session, catalogName); + ConnectorMetadata metadata = catalogMetadata.getMetadata(); + ConnectorTableExecuteHandle newConnectorHandle = metadata.beginTableExecute(session.toConnectorSession(), tableExecuteHandle.getConnectorHandle()); + return tableExecuteHandle.withConnectorHandle(newConnectorHandle); + } + + @Override + public void finishTableExecute(Session session, TableExecuteHandle tableExecuteHandle, Collection fragments, List tableExecuteState) + { + CatalogName catalogName = tableExecuteHandle.getCatalogName(); + ConnectorMetadata metadata = getMetadata(session, catalogName); + metadata.finishTableExecute(session.toConnectorSession(catalogName), tableExecuteHandle.getConnectorHandle(), fragments, tableExecuteState); + } + @Override public Optional getSystemTable(Session session, QualifiedObjectName tableName) { @@ -2540,6 +2602,12 @@ public ProcedureRegistry getProcedureRegistry() return procedures; } + @Override + public TableProceduresRegistry getTableProcedureRegistry() + { + return tableProcedures; + } + // // Blocks // @@ -2604,6 +2672,12 @@ public AnalyzePropertyManager getAnalyzePropertyManager() return analyzePropertyManager; } + @Override + public TableProceduresPropertyManager getTableProceduresPropertyManager() + { + return tableProceduresPropertyManager; + } + // // Helpers // diff --git a/core/trino-main/src/main/java/io/trino/metadata/SchemaPropertyManager.java b/core/trino-main/src/main/java/io/trino/metadata/SchemaPropertyManager.java index fffe8ca55a36..961e95a20b43 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/SchemaPropertyManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/SchemaPropertyManager.java @@ -16,7 +16,7 @@ import static io.trino.spi.StandardErrorCode.INVALID_SCHEMA_PROPERTY; public class SchemaPropertyManager - extends AbstractPropertyManager + extends AbstractCatalogPropertyManager { public SchemaPropertyManager() { diff --git a/core/trino-main/src/main/java/io/trino/metadata/TableExecuteHandle.java b/core/trino-main/src/main/java/io/trino/metadata/TableExecuteHandle.java new file mode 100644 index 000000000000..fe07530ddbf5 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/metadata/TableExecuteHandle.java @@ -0,0 +1,103 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.metadata; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.connector.CatalogName; +import io.trino.spi.connector.ConnectorTableExecuteHandle; +import io.trino.spi.connector.ConnectorTransactionHandle; + +import java.util.Objects; +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public final class TableExecuteHandle +{ + private final CatalogName catalogName; + private final ConnectorTransactionHandle transactionHandle; + private final ConnectorTableExecuteHandle connectorHandle; + private final TableHandle sourceTableHandle; + + @JsonCreator + public TableExecuteHandle( + @JsonProperty("catalogName") CatalogName catalogName, + @JsonProperty("transactionHandle") ConnectorTransactionHandle transactionHandle, + @JsonProperty("connectorHandle") ConnectorTableExecuteHandle connectorHandle) + { + this.catalogName = requireNonNull(catalogName, "catalogName is null"); + this.transactionHandle = requireNonNull(transactionHandle, "transactionHandle is null"); + this.connectorHandle = requireNonNull(connectorHandle, "connectorHandle is null"); + this.sourceTableHandle = new TableHandle(catalogName, connectorHandle.getSourceTableHandle(), transactionHandle, Optional.empty()); + } + + @JsonProperty + public CatalogName getCatalogName() + { + return catalogName; + } + + @JsonProperty + public ConnectorTransactionHandle getTransactionHandle() + { + return transactionHandle; + } + + @JsonProperty + public ConnectorTableExecuteHandle getConnectorHandle() + { + return connectorHandle; + } + + @JsonIgnore + public TableHandle getSourceTableHandle() + { + return sourceTableHandle; + } + + public TableExecuteHandle withConnectorHandle(ConnectorTableExecuteHandle connectorHandle) + { + return new TableExecuteHandle(catalogName, transactionHandle, connectorHandle); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + TableExecuteHandle o = (TableExecuteHandle) obj; + return Objects.equals(this.catalogName, o.catalogName) && + Objects.equals(this.transactionHandle, o.transactionHandle) && + Objects.equals(this.connectorHandle, o.connectorHandle) && + Objects.equals(this.sourceTableHandle, o.sourceTableHandle); + } + + @Override + public int hashCode() + { + return Objects.hash(catalogName, transactionHandle, connectorHandle, sourceTableHandle); + } + + @Override + public String toString() + { + return "Execute[" + catalogName + ":" + connectorHandle + "]"; + } +} diff --git a/core/trino-main/src/main/java/io/trino/metadata/TableProceduresPropertyManager.java b/core/trino-main/src/main/java/io/trino/metadata/TableProceduresPropertyManager.java new file mode 100644 index 000000000000..4dd99298fc2c --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/metadata/TableProceduresPropertyManager.java @@ -0,0 +1,133 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.metadata; + +import io.trino.Session; +import io.trino.connector.CatalogName; +import io.trino.security.AccessControl; +import io.trino.spi.session.PropertyMetadata; +import io.trino.sql.tree.Expression; +import io.trino.sql.tree.NodeRef; +import io.trino.sql.tree.Parameter; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static io.trino.spi.StandardErrorCode.INVALID_PROCEDURE_ARGUMENT; +import static java.util.Objects.requireNonNull; + +public class TableProceduresPropertyManager + extends AbstractPropertyManager +{ + public TableProceduresPropertyManager() + { + super("procedure", INVALID_PROCEDURE_ARGUMENT); + } + + public final void addProperties(CatalogName catalogName, String procedureName, List> properties) + { + innerAddProperties(new Key(catalogName, procedureName), properties); + } + + public final void removeProperties(CatalogName catalogName) + { + Set keysToRemove = connectorProperties.keySet().stream() + .filter(key -> catalogName.equals(key.getCatalogName())) + .collect(toImmutableSet()); + for (Key key : keysToRemove) { + innerRemoveProperties(key); + } + } + + public final Map getProperties( + CatalogName catalog, + String procedureName, + String catalogName, // only use this for error messages + Map sqlPropertyValues, + Session session, + Metadata metadata, + AccessControl accessControl, + Map, Expression> parameters) + { + return innerGetProperties( + new Key(catalog, procedureName), + catalogName, + sqlPropertyValues, + session, + metadata, + accessControl, + parameters); + } + + public Map>> getAllProperties() + { + return innerGetAllProperties(); + } + + @Override + protected String formatPropertiesKeyForMessage(String catalogName, Key propertiesKey) + { + return String.format("Procedure '%s.%s'", catalogName, propertiesKey.procedureName); + } + + public static Key tableProcedureKey(CatalogName catalogName, String procedureName) + { + return new Key(catalogName, procedureName); + } + + public static final class Key + { + private final CatalogName catalogName; + private final String procedureName; + + private Key(CatalogName catalogName, String procedureName) + { + this.catalogName = requireNonNull(catalogName, "catalogName is null"); + this.procedureName = requireNonNull(procedureName, "procedureName is null"); + } + + public CatalogName getCatalogName() + { + return catalogName; + } + + public String getProcedureName() + { + return procedureName; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Key key = (Key) o; + return Objects.equals(catalogName, key.catalogName) + && Objects.equals(procedureName, key.procedureName); + } + + @Override + public int hashCode() + { + return Objects.hash(catalogName, procedureName); + } + } +} diff --git a/core/trino-main/src/main/java/io/trino/metadata/TableProceduresRegistry.java b/core/trino-main/src/main/java/io/trino/metadata/TableProceduresRegistry.java new file mode 100644 index 000000000000..62122a722184 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/metadata/TableProceduresRegistry.java @@ -0,0 +1,64 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.metadata; + +import com.google.common.collect.Maps; +import io.trino.connector.CatalogName; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.TableProcedureMetadata; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import static com.google.common.base.Preconditions.checkState; +import static io.trino.spi.StandardErrorCode.PROCEDURE_NOT_FOUND; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class TableProceduresRegistry +{ + private final Map> tableProcedures = new ConcurrentHashMap<>(); + + public TableProceduresRegistry() + { + } + + public void addTableProcedures(CatalogName catalogName, Collection procedures) + { + requireNonNull(catalogName, "catalogName is null"); + requireNonNull(procedures, "procedures is null"); + + Map proceduresByName = Maps.uniqueIndex(procedures, TableProcedureMetadata::getName); + + checkState(tableProcedures.putIfAbsent(catalogName, proceduresByName) == null, "Table procedures already registered for connector: %s", catalogName); + } + + public void removeProcedures(CatalogName catalogName) + { + tableProcedures.remove(catalogName); + } + + public TableProcedureMetadata resolve(CatalogName catalogName, String name) + { + Map procedures = tableProcedures.get(catalogName); + if (procedures != null) { + TableProcedureMetadata procedure = procedures.get(name); + if (procedure != null) { + return procedure; + } + } + throw new TrinoException(PROCEDURE_NOT_FOUND, format("Procedure %s not registered for catalog %s", name, catalogName)); + } +} diff --git a/core/trino-main/src/main/java/io/trino/metadata/TablePropertyManager.java b/core/trino-main/src/main/java/io/trino/metadata/TablePropertyManager.java index ec4fae8aa3c0..e1774436472a 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/TablePropertyManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/TablePropertyManager.java @@ -16,7 +16,7 @@ import static io.trino.spi.StandardErrorCode.INVALID_TABLE_PROPERTY; public class TablePropertyManager - extends AbstractPropertyManager + extends AbstractCatalogPropertyManager { public TablePropertyManager() { diff --git a/core/trino-main/src/main/java/io/trino/operator/TableFinishOperator.java b/core/trino-main/src/main/java/io/trino/operator/TableFinishOperator.java index 91fc5b539f8d..ab4a8a232670 100644 --- a/core/trino-main/src/main/java/io/trino/operator/TableFinishOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/TableFinishOperator.java @@ -18,9 +18,12 @@ import io.airlift.slice.Slice; import io.airlift.units.Duration; import io.trino.Session; +import io.trino.execution.TableExecuteContext; +import io.trino.execution.TableExecuteContextManager; import io.trino.operator.OperationTimer.OperationTiming; import io.trino.spi.Page; import io.trino.spi.PageBuilder; +import io.trino.spi.QueryId; import io.trino.spi.block.Block; import io.trino.spi.connector.ConnectorOutputMetadata; import io.trino.spi.statistics.ComputedStatistics; @@ -57,6 +60,8 @@ public static class TableFinishOperatorFactory private final TableFinisher tableFinisher; private final OperatorFactory statisticsAggregationOperatorFactory; private final StatisticAggregationsDescriptor descriptor; + private final TableExecuteContextManager tableExecuteContextManager; + private final boolean outputRowCount; private final Session session; private boolean closed; @@ -66,6 +71,8 @@ public TableFinishOperatorFactory( TableFinisher tableFinisher, OperatorFactory statisticsAggregationOperatorFactory, StatisticAggregationsDescriptor descriptor, + TableExecuteContextManager tableExecuteContextManager, + boolean outputRowCount, Session session) { this.operatorId = operatorId; @@ -74,6 +81,8 @@ public TableFinishOperatorFactory( this.statisticsAggregationOperatorFactory = requireNonNull(statisticsAggregationOperatorFactory, "statisticsAggregationOperatorFactory is null"); this.descriptor = requireNonNull(descriptor, "descriptor is null"); this.session = requireNonNull(session, "session is null"); + this.tableExecuteContextManager = requireNonNull(tableExecuteContextManager, "tableExecuteContextManager is null"); + this.outputRowCount = outputRowCount; } @Override @@ -83,7 +92,9 @@ public Operator createOperator(DriverContext driverContext) OperatorContext context = driverContext.addOperatorContext(operatorId, planNodeId, TableFinishOperator.class.getSimpleName()); Operator statisticsAggregationOperator = statisticsAggregationOperatorFactory.createOperator(driverContext); boolean statisticsCpuTimerEnabled = !(statisticsAggregationOperator instanceof DevNullOperator) && isStatisticsCpuTimerEnabled(session); - return new TableFinishOperator(context, tableFinisher, statisticsAggregationOperator, descriptor, statisticsCpuTimerEnabled); + QueryId queryId = driverContext.getPipelineContext().getTaskContext().getQueryContext().getQueryId(); + TableExecuteContext tableExecuteContext = tableExecuteContextManager.getTableExecuteContextForQuery(queryId); + return new TableFinishOperator(context, tableFinisher, statisticsAggregationOperator, descriptor, statisticsCpuTimerEnabled, tableExecuteContext, outputRowCount); } @Override @@ -95,7 +106,7 @@ public void noMoreOperators() @Override public OperatorFactory duplicate() { - return new TableFinishOperatorFactory(operatorId, planNodeId, tableFinisher, statisticsAggregationOperatorFactory, descriptor, session); + return new TableFinishOperatorFactory(operatorId, planNodeId, tableFinisher, statisticsAggregationOperatorFactory, descriptor, tableExecuteContextManager, outputRowCount, session); } } @@ -118,6 +129,9 @@ private enum State private final OperationTiming statisticsTiming = new OperationTiming(); private final boolean statisticsCpuTimerEnabled; + private final TableExecuteContext tableExecuteContext; + private final boolean outputRowCount; + private final Supplier tableFinishInfoSupplier; public TableFinishOperator( @@ -125,14 +139,18 @@ public TableFinishOperator( TableFinisher tableFinisher, Operator statisticsAggregationOperator, StatisticAggregationsDescriptor descriptor, - boolean statisticsCpuTimerEnabled) + boolean statisticsCpuTimerEnabled, + TableExecuteContext tableExecuteContext, + boolean outputRowCount) { this.operatorContext = requireNonNull(operatorContext, "operatorContext is null"); this.tableFinisher = requireNonNull(tableFinisher, "tableFinisher is null"); this.statisticsAggregationOperator = requireNonNull(statisticsAggregationOperator, "statisticsAggregationOperator is null"); this.descriptor = requireNonNull(descriptor, "descriptor is null"); this.statisticsCpuTimerEnabled = statisticsCpuTimerEnabled; + this.tableExecuteContext = requireNonNull(tableExecuteContext, "tableExecuteContext is null"); this.tableFinishInfoSupplier = createTableFinishInfoSupplier(outputMetadata, statisticsTiming); + this.outputRowCount = outputRowCount; operatorContext.setInfoSupplier(tableFinishInfoSupplier); } @@ -297,13 +315,15 @@ public Page getOutput() } state = State.FINISHED; - this.outputMetadata.set(tableFinisher.finishTable(fragmentBuilder.build(), computedStatisticsBuilder.build())); + this.outputMetadata.set(tableFinisher.finishTable(fragmentBuilder.build(), computedStatisticsBuilder.build(), tableExecuteContext)); // output page will only be constructed once, // so a new PageBuilder is constructed (instead of using PageBuilder.reset) PageBuilder page = new PageBuilder(1, TYPES); - page.declarePosition(); - BIGINT.writeLong(page.getBlockBuilder(0), rowCount); + if (outputRowCount) { + page.declarePosition(); + BIGINT.writeLong(page.getBlockBuilder(0), rowCount); + } return page.build(); } @@ -345,6 +365,9 @@ public void close() public interface TableFinisher { - Optional finishTable(Collection fragments, Collection computedStatistics); + Optional finishTable( + Collection fragments, + Collection computedStatistics, + TableExecuteContext tableExecuteContext); } } diff --git a/core/trino-main/src/main/java/io/trino/operator/TableWriterOperator.java b/core/trino-main/src/main/java/io/trino/operator/TableWriterOperator.java index b1034d8d6c2f..ce9cd904d1e7 100644 --- a/core/trino-main/src/main/java/io/trino/operator/TableWriterOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/TableWriterOperator.java @@ -98,8 +98,12 @@ public TableWriterOperatorFactory( this.columnChannels = requireNonNull(columnChannels, "columnChannels is null"); this.notNullChannelColumnNames = requireNonNull(notNullChannelColumnNames, "notNullChannelColumnNames is null"); this.pageSinkManager = requireNonNull(pageSinkManager, "pageSinkManager is null"); - checkArgument(writerTarget instanceof CreateTarget || writerTarget instanceof InsertTarget || writerTarget instanceof TableWriterNode.RefreshMaterializedViewTarget, - "writerTarget must be CreateTarget, InsertTarget or RefreshMaterializedViewTarget"); + checkArgument( + writerTarget instanceof CreateTarget + || writerTarget instanceof InsertTarget + || writerTarget instanceof TableWriterNode.RefreshMaterializedViewTarget + || writerTarget instanceof TableWriterNode.TableExecuteTarget, + "writerTarget must be CreateTarget, InsertTarget, RefreshMaterializedViewTarget or TableExecuteTarget"); this.target = requireNonNull(writerTarget, "writerTarget is null"); this.session = session; this.statisticsAggregationOperatorFactory = requireNonNull(statisticsAggregationOperatorFactory, "statisticsAggregationOperatorFactory is null"); @@ -127,6 +131,9 @@ private ConnectorPageSink createPageSink() if (target instanceof TableWriterNode.RefreshMaterializedViewTarget) { return pageSinkManager.createPageSink(session, ((TableWriterNode.RefreshMaterializedViewTarget) target).getInsertHandle()); } + if (target instanceof TableWriterNode.TableExecuteTarget) { + return pageSinkManager.createPageSink(session, ((TableWriterNode.TableExecuteTarget) target).getHandle()); + } throw new UnsupportedOperationException("Unhandled target type: " + target.getClass().getName()); } diff --git a/core/trino-main/src/main/java/io/trino/security/AccessControl.java b/core/trino-main/src/main/java/io/trino/security/AccessControl.java index 673aa5c6b575..6c11685772b4 100644 --- a/core/trino-main/src/main/java/io/trino/security/AccessControl.java +++ b/core/trino-main/src/main/java/io/trino/security/AccessControl.java @@ -488,6 +488,13 @@ void checkCanRevokeRoles(SecurityContext context, */ void checkCanExecuteFunction(SecurityContext context, String functionName); + /** + * Check if identity is allowed to execute given table procedure on given table + * + * @throws AccessDeniedException if not allowed + */ + void checkCanExecuteTableProcedure(SecurityContext context, QualifiedObjectName tableName, String procedureName); + default List getRowFilters(SecurityContext context, QualifiedObjectName tableName) { return ImmutableList.of(); diff --git a/core/trino-main/src/main/java/io/trino/security/AccessControlManager.java b/core/trino-main/src/main/java/io/trino/security/AccessControlManager.java index 65552f986e46..707a8963b973 100644 --- a/core/trino-main/src/main/java/io/trino/security/AccessControlManager.java +++ b/core/trino-main/src/main/java/io/trino/security/AccessControlManager.java @@ -1051,6 +1051,27 @@ public void checkCanExecuteFunction(SecurityContext context, String functionName systemAuthorizationCheck(control -> control.checkCanExecuteFunction(context.toSystemSecurityContext(), functionName)); } + @Override + public void checkCanExecuteTableProcedure(SecurityContext securityContext, QualifiedObjectName tableName, String procedureName) + { + requireNonNull(securityContext, "securityContext is null"); + requireNonNull(procedureName, "procedureName is null"); + requireNonNull(tableName, "tableName is null"); + + systemAuthorizationCheck(control -> control.checkCanExecuteTableProcedure( + securityContext.toSystemSecurityContext(), + tableName.asCatalogSchemaTableName(), + procedureName)); + + catalogAuthorizationCheck( + tableName.getCatalogName(), + securityContext, + (control, context) -> control.checkCanExecuteTableProcedure( + context, + tableName.asSchemaTableName(), + procedureName)); + } + @Override public List getRowFilters(SecurityContext context, QualifiedObjectName tableName) { diff --git a/core/trino-main/src/main/java/io/trino/security/AllowAllAccessControl.java b/core/trino-main/src/main/java/io/trino/security/AllowAllAccessControl.java index 745c861e4e9b..d0f63e20b113 100644 --- a/core/trino-main/src/main/java/io/trino/security/AllowAllAccessControl.java +++ b/core/trino-main/src/main/java/io/trino/security/AllowAllAccessControl.java @@ -337,4 +337,9 @@ public void checkCanExecuteProcedure(SecurityContext context, QualifiedObjectNam public void checkCanExecuteFunction(SecurityContext context, String functionName) { } + + @Override + public void checkCanExecuteTableProcedure(SecurityContext context, QualifiedObjectName tableName, String procedureName) + { + } } diff --git a/core/trino-main/src/main/java/io/trino/security/DenyAllAccessControl.java b/core/trino-main/src/main/java/io/trino/security/DenyAllAccessControl.java index 0803060c89fd..1bdb295b9c7d 100644 --- a/core/trino-main/src/main/java/io/trino/security/DenyAllAccessControl.java +++ b/core/trino-main/src/main/java/io/trino/security/DenyAllAccessControl.java @@ -45,6 +45,7 @@ import static io.trino.spi.security.AccessDeniedException.denyExecuteFunction; import static io.trino.spi.security.AccessDeniedException.denyExecuteProcedure; import static io.trino.spi.security.AccessDeniedException.denyExecuteQuery; +import static io.trino.spi.security.AccessDeniedException.denyExecuteTableProcedure; import static io.trino.spi.security.AccessDeniedException.denyGrantExecuteFunctionPrivilege; import static io.trino.spi.security.AccessDeniedException.denyGrantRoles; import static io.trino.spi.security.AccessDeniedException.denyGrantSchemaPrivilege; @@ -451,4 +452,10 @@ public void checkCanExecuteFunction(SecurityContext context, String functionName { denyExecuteFunction(functionName); } + + @Override + public void checkCanExecuteTableProcedure(SecurityContext context, QualifiedObjectName tableName, String procedureName) + { + denyExecuteTableProcedure(tableName.toString(), procedureName.toString()); + } } diff --git a/core/trino-main/src/main/java/io/trino/security/ForwardingAccessControl.java b/core/trino-main/src/main/java/io/trino/security/ForwardingAccessControl.java index f1421d9be99f..65b06608aed0 100644 --- a/core/trino-main/src/main/java/io/trino/security/ForwardingAccessControl.java +++ b/core/trino-main/src/main/java/io/trino/security/ForwardingAccessControl.java @@ -416,6 +416,12 @@ public void checkCanExecuteFunction(SecurityContext context, String functionName delegate().checkCanExecuteFunction(context, functionName); } + @Override + public void checkCanExecuteTableProcedure(SecurityContext context, QualifiedObjectName tableName, String procedureName) + { + delegate().checkCanExecuteTableProcedure(context, tableName, procedureName); + } + @Override public List getRowFilters(SecurityContext context, QualifiedObjectName tableName) { diff --git a/core/trino-main/src/main/java/io/trino/security/InjectedConnectorAccessControl.java b/core/trino-main/src/main/java/io/trino/security/InjectedConnectorAccessControl.java index bc6cf1e9db54..ae4dce4b8932 100644 --- a/core/trino-main/src/main/java/io/trino/security/InjectedConnectorAccessControl.java +++ b/core/trino-main/src/main/java/io/trino/security/InjectedConnectorAccessControl.java @@ -398,6 +398,16 @@ public void checkCanExecuteProcedure(ConnectorSecurityContext context, SchemaRou accessControl.checkCanExecuteProcedure(securityContext, new QualifiedObjectName(catalogName, procedure.getSchemaName(), procedure.getRoutineName())); } + @Override + public void checkCanExecuteTableProcedure(ConnectorSecurityContext context, SchemaTableName tableName, String procedure) + { + checkArgument(context == null, "context must be null"); + accessControl.checkCanExecuteTableProcedure( + securityContext, + getQualifiedObjectName(tableName), + procedure); + } + @Override public Optional getRowFilter(ConnectorSecurityContext context, SchemaTableName tableName) { diff --git a/core/trino-main/src/main/java/io/trino/server/QueryExecutionFactoryModule.java b/core/trino-main/src/main/java/io/trino/server/QueryExecutionFactoryModule.java index b0241a9b9bca..2814c4302e27 100644 --- a/core/trino-main/src/main/java/io/trino/server/QueryExecutionFactoryModule.java +++ b/core/trino-main/src/main/java/io/trino/server/QueryExecutionFactoryModule.java @@ -96,6 +96,7 @@ import io.trino.sql.tree.SetViewAuthorization; import io.trino.sql.tree.StartTransaction; import io.trino.sql.tree.Statement; +import io.trino.sql.tree.TableExecute; import io.trino.sql.tree.Use; import static com.google.common.base.Preconditions.checkArgument; @@ -116,6 +117,9 @@ public void configure(Binder binder) executionBinder.addBinding(statement).to(SqlQueryExecutionFactory.class).in(Scopes.SINGLETON); } + // we want to execute TableExecute in distributed manner so we are binding it to SqlQueryExecutionFactory. + executionBinder.addBinding(TableExecute.class).to(SqlQueryExecutionFactory.class).in(Scopes.SINGLETON); + binder.bind(DataDefinitionExecutionFactory.class).in(Scopes.SINGLETON); bindDataDefinitionTask(binder, executionBinder, AddColumn.class, AddColumnTask.class); bindDataDefinitionTask(binder, executionBinder, Call.class, CallTask.class); diff --git a/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java b/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java index fd7e52580e81..b768c9c62e15 100644 --- a/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java +++ b/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java @@ -44,6 +44,7 @@ import io.trino.execution.NodeTaskMap; import io.trino.execution.QueryManagerConfig; import io.trino.execution.SqlTaskManager; +import io.trino.execution.TableExecuteContextManager; import io.trino.execution.TaskManagementExecutor; import io.trino.execution.TaskManager; import io.trino.execution.TaskManagerConfig; @@ -77,6 +78,7 @@ import io.trino.metadata.StaticCatalogStore; import io.trino.metadata.StaticCatalogStoreConfig; import io.trino.metadata.SystemSecurityMetadata; +import io.trino.metadata.TableProceduresPropertyManager; import io.trino.metadata.TablePropertyManager; import io.trino.operator.ExchangeClientConfig; import io.trino.operator.ExchangeClientFactory; @@ -233,6 +235,9 @@ protected void setup(Binder binder) // analyze properties binder.bind(AnalyzePropertyManager.class).in(Scopes.SINGLETON); + // table procedures properties + binder.bind(TableProceduresPropertyManager.class).in(Scopes.SINGLETON); + // node manager discoveryBinder(binder).bindSelector("trino"); binder.bind(DiscoveryNodeManager.class).in(Scopes.SINGLETON); @@ -271,6 +276,7 @@ protected void setup(Binder binder) binder.bind(TaskManagementExecutor.class).in(Scopes.SINGLETON); binder.bind(SqlTaskManager.class).in(Scopes.SINGLETON); binder.bind(TaskManager.class).to(Key.get(SqlTaskManager.class)); + binder.bind(TableExecuteContextManager.class).in(Scopes.SINGLETON); // memory revoking scheduler binder.bind(MemoryRevokingScheduler.class).in(Scopes.SINGLETON); diff --git a/core/trino-main/src/main/java/io/trino/split/BufferingSplitSource.java b/core/trino-main/src/main/java/io/trino/split/BufferingSplitSource.java index d1fa23836286..8bd175b23116 100644 --- a/core/trino-main/src/main/java/io/trino/split/BufferingSplitSource.java +++ b/core/trino-main/src/main/java/io/trino/split/BufferingSplitSource.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Optional; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.util.concurrent.Futures.immediateVoidFuture; @@ -65,6 +66,12 @@ public boolean isFinished() return source.isFinished(); } + @Override + public Optional> getTableExecuteSplitsInfo() + { + return source.getTableExecuteSplitsInfo(); + } + private static class GetNextBatch { private final SplitSource splitSource; diff --git a/core/trino-main/src/main/java/io/trino/split/ConnectorAwareSplitSource.java b/core/trino-main/src/main/java/io/trino/split/ConnectorAwareSplitSource.java index 4bce504db55b..83792cbf5674 100644 --- a/core/trino-main/src/main/java/io/trino/split/ConnectorAwareSplitSource.java +++ b/core/trino-main/src/main/java/io/trino/split/ConnectorAwareSplitSource.java @@ -24,6 +24,9 @@ import io.trino.spi.connector.ConnectorSplitSource; import io.trino.spi.connector.ConnectorSplitSource.ConnectorSplitBatch; +import java.util.List; +import java.util.Optional; + import static com.google.common.util.concurrent.MoreExecutors.directExecutor; import static io.airlift.concurrent.MoreFutures.toListenableFuture; import static java.util.Objects.requireNonNull; @@ -71,6 +74,12 @@ public boolean isFinished() return source.isFinished(); } + @Override + public Optional> getTableExecuteSplitsInfo() + { + return source.getTableExecuteSplitsInfo(); + } + @Override public String toString() { diff --git a/core/trino-main/src/main/java/io/trino/split/PageSinkManager.java b/core/trino-main/src/main/java/io/trino/split/PageSinkManager.java index 1c9f7d62f4e8..0e54c712f6de 100644 --- a/core/trino-main/src/main/java/io/trino/split/PageSinkManager.java +++ b/core/trino-main/src/main/java/io/trino/split/PageSinkManager.java @@ -17,6 +17,7 @@ import io.trino.connector.CatalogName; import io.trino.metadata.InsertTableHandle; import io.trino.metadata.OutputTableHandle; +import io.trino.metadata.TableExecuteHandle; import io.trino.spi.connector.ConnectorPageSink; import io.trino.spi.connector.ConnectorPageSinkProvider; import io.trino.spi.connector.ConnectorSession; @@ -61,6 +62,14 @@ public ConnectorPageSink createPageSink(Session session, InsertTableHandle table return providerFor(tableHandle.getCatalogName()).createPageSink(tableHandle.getTransactionHandle(), connectorSession, tableHandle.getConnectorHandle()); } + @Override + public ConnectorPageSink createPageSink(Session session, TableExecuteHandle tableHandle) + { + // assumes connectorId and catalog are the same + ConnectorSession connectorSession = session.toConnectorSession(tableHandle.getCatalogName()); + return providerFor(tableHandle.getCatalogName()).createPageSink(tableHandle.getTransactionHandle(), connectorSession, tableHandle.getConnectorHandle()); + } + private ConnectorPageSinkProvider providerFor(CatalogName catalogName) { ConnectorPageSinkProvider provider = pageSinkProviders.get(catalogName); diff --git a/core/trino-main/src/main/java/io/trino/split/PageSinkProvider.java b/core/trino-main/src/main/java/io/trino/split/PageSinkProvider.java index 8cf84625635a..effcfaef23f3 100644 --- a/core/trino-main/src/main/java/io/trino/split/PageSinkProvider.java +++ b/core/trino-main/src/main/java/io/trino/split/PageSinkProvider.java @@ -16,6 +16,7 @@ import io.trino.Session; import io.trino.metadata.InsertTableHandle; import io.trino.metadata.OutputTableHandle; +import io.trino.metadata.TableExecuteHandle; import io.trino.spi.connector.ConnectorPageSink; public interface PageSinkProvider @@ -23,4 +24,6 @@ public interface PageSinkProvider ConnectorPageSink createPageSink(Session session, OutputTableHandle tableHandle); ConnectorPageSink createPageSink(Session session, InsertTableHandle tableHandle); + + ConnectorPageSink createPageSink(Session session, TableExecuteHandle tableHandle); } diff --git a/core/trino-main/src/main/java/io/trino/split/SampledSplitSource.java b/core/trino-main/src/main/java/io/trino/split/SampledSplitSource.java index 3130ebead480..120ff8d6a079 100644 --- a/core/trino-main/src/main/java/io/trino/split/SampledSplitSource.java +++ b/core/trino-main/src/main/java/io/trino/split/SampledSplitSource.java @@ -21,6 +21,8 @@ import javax.annotation.Nullable; +import java.util.List; +import java.util.Optional; import java.util.concurrent.ThreadLocalRandom; import static com.google.common.collect.ImmutableList.toImmutableList; @@ -68,4 +70,13 @@ public boolean isFinished() { return splitSource.isFinished(); } + + @Override + public Optional> getTableExecuteSplitsInfo() + { + splitSource.getTableExecuteSplitsInfo().ifPresent(splitInfo -> { + throw new IllegalStateException("Cannot use SampledSplitSource with SplitSource which returns non-empty TableExecuteSplitsInfo=" + splitInfo); + }); + return Optional.empty(); + } } diff --git a/core/trino-main/src/main/java/io/trino/split/SplitSource.java b/core/trino-main/src/main/java/io/trino/split/SplitSource.java index fff277bec426..b2247b4f9440 100644 --- a/core/trino-main/src/main/java/io/trino/split/SplitSource.java +++ b/core/trino-main/src/main/java/io/trino/split/SplitSource.java @@ -21,6 +21,7 @@ import java.io.Closeable; import java.util.List; +import java.util.Optional; import static java.util.Objects.requireNonNull; @@ -36,6 +37,8 @@ public interface SplitSource boolean isFinished(); + Optional> getTableExecuteSplitsInfo(); + class SplitBatch { private final List splits; diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java index d49c33d6097a..b79cbbd5b6f9 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java @@ -215,6 +215,8 @@ public class Analysis private final Multimap originColumnDetails = ArrayListMultimap.create(); private final Multimap, Field> fieldLineage = ArrayListMultimap.create(); + private Map tableExecuteProperties; + public Analysis(@Nullable Statement root, Map, Expression> parameters, QueryType queryType) { this.root = root; @@ -1114,6 +1116,19 @@ public PredicateCoercions getPredicateCoercions(Expression expression) return predicateCoercions.get(NodeRef.of(expression)); } + public void setTableExecuteProperties(Map tableExecuteProperties) + { + requireNonNull(tableExecuteProperties, "tableExecuteProperties is null"); + checkState(this.tableExecuteProperties == null, "tableExecuteProperties already set"); + this.tableExecuteProperties = ImmutableMap.copyOf(tableExecuteProperties); + } + + public Map getTableExecuteProperties() + { + checkState(this.tableExecuteProperties != null, "tableExecuteProperties not set"); + return tableExecuteProperties; + } + @Immutable public static final class SelectExpression { diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java index 638aaad2739a..8b4b5513013d 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java @@ -51,6 +51,7 @@ import io.trino.spi.connector.ConnectorViewDefinition; import io.trino.spi.connector.ConnectorViewDefinition.ViewColumn; import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.TableProcedureMetadata; import io.trino.spi.function.OperatorType; import io.trino.spi.security.AccessDeniedException; import io.trino.spi.security.GroupProvider; @@ -170,6 +171,7 @@ import io.trino.sql.tree.SubqueryExpression; import io.trino.sql.tree.SubscriptExpression; import io.trino.sql.tree.Table; +import io.trino.sql.tree.TableExecute; import io.trino.sql.tree.TableSubquery; import io.trino.sql.tree.Union; import io.trino.sql.tree.Unnest; @@ -975,6 +977,71 @@ protected Scope visitSetTableAuthorization(SetTableAuthorization node, Optional< return createAndAssignScope(node, scope); } + @Override + protected Scope visitTableExecute(TableExecute node, Optional scope) + { + Table table = node.getTable(); + QualifiedObjectName originalName = createQualifiedObjectName(session, table, table.getName()); + String procedureName = node.getProcedureName().getCanonicalValue(); + + if (metadata.getMaterializedView(session, originalName).isPresent()) { + throw semanticException(NOT_SUPPORTED, node, "ALTER TABLE EXECUTE is not supported for materialized views"); + } + + if (metadata.getView(session, originalName).isPresent()) { + throw semanticException(NOT_SUPPORTED, node, "ALTER TABLE EXECUTE is not supported for views"); + } + + RedirectionAwareTableHandle redirection = metadata.getRedirectionAwareTableHandle(session, originalName); + QualifiedObjectName tableName = redirection.getRedirectedTableName().orElse(originalName); + TableHandle tableHandle = redirection.getTableHandle() + .orElseThrow(() -> semanticException(TABLE_NOT_FOUND, table, "Table '%s' does not exist", tableName)); + + accessControl.checkCanExecuteTableProcedure( + session.toSecurityContext(), + tableName, + procedureName); + + if (!accessControl.getRowFilters(session.toSecurityContext(), tableName).isEmpty()) { + throw semanticException(NOT_SUPPORTED, node, "ALTER TABLE EXECUTE is not supported for table with row filter"); + } + + TableMetadata tableMetadata = metadata.getTableMetadata(session, tableHandle); + for (ColumnMetadata tableColumn : tableMetadata.getColumns()) { + if (!accessControl.getColumnMasks(session.toSecurityContext(), tableName, tableColumn.getName(), tableColumn.getType()).isEmpty()) { + throw semanticException(NOT_SUPPORTED, node, "ALTER TABLE EXECUTE is not supported for table with column masks"); + } + } + + Scope tableScope = analyze(table, scope); + + CatalogName catalogName = getRequiredCatalogHandle(metadata, session, node, tableName.getCatalogName()); + TableProcedureMetadata procedureMetadata = metadata.getTableProcedureRegistry().resolve(catalogName, procedureName); + + // analyze WHERE + if (!procedureMetadata.getExecutionMode().supportsFilter() && node.getWhere().isPresent()) { + throw semanticException(NOT_SUPPORTED, node, "WHERE not supported for procedure " + procedureName); + } + node.getWhere().ifPresent(where -> analyzeWhere(node, tableScope, where)); + + // analyze WITH + validateProperties(node.getProperties(), scope); + Map tableProperties = metadata.getTableProceduresPropertyManager().getProperties( + catalogName, + procedureName, + catalogName.getCatalogName(), + mapFromProperties(node.getProperties()), + session, + metadata, + accessControl, + analysis.getParameters()); + analysis.setTableExecuteProperties(tableProperties); + + analysis.setUpdateType("EXECUTE"); + + return createAndAssignScope(node, scope, Field.newUnqualified("rows", BIGINT)); + } + @Override protected Scope visitRenameView(RenameView node, Optional scope) { diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java index 2beab68cda60..bd689cf169ef 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java @@ -34,6 +34,7 @@ import io.trino.execution.DynamicFilterConfig; import io.trino.execution.ExplainAnalyzeContext; import io.trino.execution.StageId; +import io.trino.execution.TableExecuteContextManager; import io.trino.execution.TaskId; import io.trino.execution.TaskManagerConfig; import io.trino.execution.buffer.OutputBuffer; @@ -198,6 +199,7 @@ import io.trino.sql.planner.plan.TableScanNode; import io.trino.sql.planner.plan.TableWriterNode; import io.trino.sql.planner.plan.TableWriterNode.DeleteTarget; +import io.trino.sql.planner.plan.TableWriterNode.TableExecuteTarget; import io.trino.sql.planner.plan.TableWriterNode.UpdateTarget; import io.trino.sql.planner.plan.TopNNode; import io.trino.sql.planner.plan.TopNRankingNode; @@ -362,6 +364,7 @@ public class LocalExecutionPlanner private final DynamicFilterConfig dynamicFilterConfig; private final TypeOperators typeOperators; private final BlockTypeOperators blockTypeOperators; + private final TableExecuteContextManager tableExecuteContextManager; @Inject public LocalExecutionPlanner( @@ -387,7 +390,8 @@ public LocalExecutionPlanner( OrderingCompiler orderingCompiler, DynamicFilterConfig dynamicFilterConfig, TypeOperators typeOperators, - BlockTypeOperators blockTypeOperators) + BlockTypeOperators blockTypeOperators, + TableExecuteContextManager tableExecuteContextManager) { this.explainAnalyzeContext = requireNonNull(explainAnalyzeContext, "explainAnalyzeContext is null"); this.pageSourceProvider = requireNonNull(pageSourceProvider, "pageSourceProvider is null"); @@ -415,6 +419,7 @@ public LocalExecutionPlanner( this.dynamicFilterConfig = requireNonNull(dynamicFilterConfig, "dynamicFilterConfig is null"); this.typeOperators = requireNonNull(typeOperators, "typeOperators is null"); this.blockTypeOperators = requireNonNull(blockTypeOperators, "blockTypeOperators is null"); + this.tableExecuteContextManager = requireNonNull(tableExecuteContextManager, "tableExecuteContextManager is null"); } public LocalExecutionPlan plan( @@ -3023,6 +3028,8 @@ public PhysicalOperation visitTableFinish(TableFinishNode node, LocalExecutionPl createTableFinisher(session, node, metadata), statisticsAggregation, descriptor, + tableExecuteContextManager, + shouldOutputRowCount(node), session); Map layout = ImmutableMap.of(node.getOutputSymbols().get(0), 0); @@ -3590,7 +3597,7 @@ private static List getTypes(List expressions, Map { + return (fragments, statistics, tableExecuteContext) -> { if (target instanceof CreateTarget) { return metadata.finishCreateTable(session, ((CreateTarget) target).getHandle(), fragments, statistics); } @@ -3615,12 +3622,23 @@ else if (target instanceof UpdateTarget) { metadata.finishUpdate(session, ((UpdateTarget) target).getHandleOrElseThrow(), fragments); return Optional.empty(); } + else if (target instanceof TableExecuteTarget) { + List splitsInfo = tableExecuteContext.getSplitsInfo(); + metadata.finishTableExecute(session, ((TableExecuteTarget) target).getHandle(), fragments, splitsInfo); + return Optional.empty(); + } else { throw new AssertionError("Unhandled target type: " + target.getClass().getName()); } }; } + private static boolean shouldOutputRowCount(TableFinishNode node) + { + WriterTarget target = node.getTarget(); + return !(target instanceof TableExecuteTarget); + } + private static Function enforceLoadedLayoutProcessor(List expectedLayout, Map inputLayout) { int[] channels = expectedLayout.stream() diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/LogicalPlanner.java b/core/trino-main/src/main/java/io/trino/sql/planner/LogicalPlanner.java index aae76ef26354..16a2899588a6 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/LogicalPlanner.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/LogicalPlanner.java @@ -29,12 +29,14 @@ import io.trino.metadata.NewTableLayout; import io.trino.metadata.QualifiedObjectName; import io.trino.metadata.ResolvedFunction; +import io.trino.metadata.TableExecuteHandle; import io.trino.metadata.TableHandle; import io.trino.metadata.TableMetadata; import io.trino.spi.TrinoException; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.ConnectorTableMetadata; +import io.trino.spi.connector.Constraint; import io.trino.spi.statistics.TableStatisticsMetadata; import io.trino.spi.type.CharType; import io.trino.spi.type.Type; @@ -56,6 +58,7 @@ import io.trino.sql.planner.plan.PlanNode; import io.trino.sql.planner.plan.ProjectNode; import io.trino.sql.planner.plan.RefreshMaterializedViewNode; +import io.trino.sql.planner.plan.SimplePlanRewriter; import io.trino.sql.planner.plan.StatisticAggregations; import io.trino.sql.planner.plan.StatisticsWriterNode; import io.trino.sql.planner.plan.TableFinishNode; @@ -86,6 +89,8 @@ import io.trino.sql.tree.Row; import io.trino.sql.tree.Statement; import io.trino.sql.tree.StringLiteral; +import io.trino.sql.tree.Table; +import io.trino.sql.tree.TableExecute; import io.trino.sql.tree.Update; import io.trino.type.TypeCoercion; import io.trino.type.UnknownType; @@ -101,6 +106,7 @@ import java.util.Optional; import java.util.Set; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Verify.verify; import static com.google.common.collect.ImmutableList.toImmutableList; @@ -108,15 +114,21 @@ import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.collect.Streams.zip; import static io.trino.SystemSessionProperties.isCollectPlanStatisticsForAllQueries; +import static io.trino.metadata.MetadataUtil.createQualifiedObjectName; +import static io.trino.spi.StandardErrorCode.NOT_FOUND; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.trino.spi.connector.Constraint.alwaysTrue; import static io.trino.spi.statistics.TableStatisticType.ROW_COUNT; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.VarbinaryType.VARBINARY; import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.sql.ExpressionUtils.filterNonDeterministicConjuncts; +import static io.trino.sql.analyzer.SemanticExceptions.semanticException; import static io.trino.sql.analyzer.TypeSignatureProvider.fromTypes; import static io.trino.sql.analyzer.TypeSignatureTranslator.toSqlType; import static io.trino.sql.planner.LogicalPlanner.Stage.OPTIMIZED; import static io.trino.sql.planner.LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED; +import static io.trino.sql.planner.PlanBuilder.newPlanBuilder; import static io.trino.sql.planner.QueryPlanner.visibleFields; import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_HASH_DISTRIBUTION; import static io.trino.sql.planner.plan.AggregationNode.singleGroupingSet; @@ -124,6 +136,7 @@ import static io.trino.sql.planner.plan.TableWriterNode.InsertReference; import static io.trino.sql.planner.plan.TableWriterNode.WriterTarget; import static io.trino.sql.planner.sanity.PlanSanityChecker.DISTRIBUTED_PLAN_SANITY_CHECKER; +import static io.trino.sql.tree.BooleanLiteral.TRUE_LITERAL; import static io.trino.sql.tree.ComparisonExpression.Operator.GREATER_THAN_OR_EQUAL; import static java.lang.String.format; import static java.util.Objects.requireNonNull; @@ -280,6 +293,9 @@ private RelationPlan planStatementWithoutOutput(Analysis analysis, Statement sta if (statement instanceof ExplainAnalyze) { return createExplainAnalyzePlan(analysis, (ExplainAnalyze) statement); } + if (statement instanceof TableExecute) { + return createTableExecutePlan(analysis, (TableExecute) statement); + } throw new TrinoException(NOT_SUPPORTED, "Unsupported statement type " + statement.getClass().getSimpleName()); } @@ -718,8 +734,17 @@ private PlanNode createOutputPlan(RelationPlan plan, Analysis analysis) private RelationPlan createRelationPlan(Analysis analysis, Query query) { - return new RelationPlanner(analysis, symbolAllocator, idAllocator, buildLambdaDeclarationToSymbolMap(analysis, symbolAllocator), metadata, Optional.empty(), session, ImmutableMap.of()) - .process(query, null); + return getRelationPlanner(analysis).process(query, null); + } + + private RelationPlan createRelationPlan(Analysis analysis, Table table) + { + return getRelationPlanner(analysis).process(table, null); + } + + private RelationPlanner getRelationPlanner(Analysis analysis) + { + return new RelationPlanner(analysis, symbolAllocator, idAllocator, buildLambdaDeclarationToSymbolMap(analysis, symbolAllocator), metadata, Optional.empty(), session, ImmutableMap.of()); } private static Map, Symbol> buildLambdaDeclarationToSymbolMap(Analysis analysis, SymbolAllocator symbolAllocator) @@ -750,6 +775,77 @@ private static Map, Symbol> buildLambdaDeclar return result; } + private RelationPlan createTableExecutePlan(Analysis analysis, TableExecute statement) + { + Table table = statement.getTable(); + TableHandle tableHandle = analysis.getTableHandle(table); + QualifiedObjectName tableName = createQualifiedObjectName(session, statement, table.getName()); + String procedureName = statement.getProcedureName().getCanonicalValue(); + + RelationPlan tableScanPlan = createRelationPlan(analysis, table); + Optional constraint = getConstraintForTableExecute(analysis, statement, tableScanPlan); + TableExecuteHandle executeHandle = + metadata.getTableHandleForExecute( + session, + tableHandle, + procedureName, + analysis.getTableExecuteProperties(), + constraint.orElse(alwaysTrue())) + .orElseThrow(() -> semanticException(NOT_FOUND, statement, "Table '%s' does not exist", tableName)); + + // replace TableHandle in TableScanNode to point to executeHandle.getSourceTableHandle() + TableHandleUpdater tableHandleUpdater = new TableHandleUpdater(executeHandle.getSourceTableHandle()); + PlanNode executeTableScanRoot = SimplePlanRewriter.rewriteWith(tableHandleUpdater, tableScanPlan.getRoot()); + checkState(tableHandleUpdater.wasHandleUpdated(), "TableScanNode was not found in plan"); + + TableMetadata tableMetadata = metadata.getTableMetadata(session, tableHandle); + List columnNames = tableMetadata.getColumns().stream() + .filter(column -> !column.isHidden()) // todo this filter is redundant + .map(ColumnMetadata::getName) + .collect(toImmutableList()); + + Optional layout = metadata.getLayoutForTableExecute(session, executeHandle); + + return createTableWriterPlan( + analysis, + executeTableScanRoot, + visibleFields(tableScanPlan), + new TableWriterNode.TableExecuteTarget(executeHandle, tableName.asSchemaTableName()), + columnNames, + tableMetadata.getColumns(), + layout, + TableStatisticsMetadata.empty()); + } + + private Optional getConstraintForTableExecute(Analysis analysis, TableExecute statement, RelationPlan tableScanPlan) + { + return statement.getWhere().map(predicate -> { + PlanBuilder planBuilder = newPlanBuilder(tableScanPlan, analysis, ImmutableMap.of(), ImmutableMap.of()); + + // don't include non-deterministic predicates + if (!TRUE_LITERAL.equals(filterNonDeterministicConjuncts(metadata, predicate))) { + throw new TrinoException(NOT_SUPPORTED, "cannot use non-deterministic predicate with ALTER TABLE ... EXECUTE"); + } + + Expression rewrittenPredicate = planBuilder.rewrite(predicate); + + DomainTranslator.ExtractionResult decomposedPredicate = DomainTranslator.fromPredicate( + metadata, + typeOperators, + session, + rewrittenPredicate, + symbolAllocator.getTypes()); + + if (!TRUE_LITERAL.equals(decomposedPredicate.getRemainingExpression())) { + // TODO support broader range. + throw new TrinoException(NOT_SUPPORTED, "only predicates expressible as TupleDomain can be used with ALTER TABLE ... EXECUTE"); + } + + TableScanNode tableScanNode = (TableScanNode) tableScanPlan.getRoot(); + return new Constraint(decomposedPredicate.getTupleDomain().transformKeys(tableScanNode.getAssignments()::get)); + }); + } + private static class Key { private final LambdaArgumentDeclaration argument; @@ -781,4 +877,29 @@ public int hashCode() return Objects.hash(argument, type); } } + + private static class TableHandleUpdater + extends SimplePlanRewriter + { + private final TableHandle newTableHandle; + private boolean handleUpdated; + + public TableHandleUpdater(TableHandle newTableHandle) + { + this.newTableHandle = requireNonNull(newTableHandle, "newTableHandle is null"); + } + + public boolean wasHandleUpdated() + { + return handleUpdated; + } + + @Override + public PlanNode visitTableScan(TableScanNode node, RewriteContext context) + { + checkArgument(!handleUpdated, "More than one TableScanNode found in plan"); + handleUpdated = true; + return node.withTableHandle(newTableHandle); + } + } } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/BeginTableWrite.java b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/BeginTableWrite.java index 4bb84e8d2e41..3f4c4bfbb00b 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/BeginTableWrite.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/BeginTableWrite.java @@ -18,6 +18,7 @@ import io.trino.cost.StatsAndCosts; import io.trino.execution.warnings.WarningCollector; import io.trino.metadata.Metadata; +import io.trino.metadata.TableExecuteHandle; import io.trino.metadata.TableHandle; import io.trino.sql.planner.PlanNodeIdAllocator; import io.trino.sql.planner.SymbolAllocator; @@ -42,6 +43,7 @@ import io.trino.sql.planner.plan.TableWriterNode.DeleteTarget; import io.trino.sql.planner.plan.TableWriterNode.InsertReference; import io.trino.sql.planner.plan.TableWriterNode.InsertTarget; +import io.trino.sql.planner.plan.TableWriterNode.TableExecuteTarget; import io.trino.sql.planner.plan.TableWriterNode.UpdateTarget; import io.trino.sql.planner.plan.TableWriterNode.WriterTarget; import io.trino.sql.planner.plan.UnionNode; @@ -250,6 +252,11 @@ private WriterTarget createWriterTarget(WriterTarget target) metadata.getTableMetadata(session, refreshMV.getStorageTableHandle()).getTable(), refreshMV.getSourceTableHandles()); } + if (target instanceof TableExecuteTarget) { + TableExecuteTarget tableExecute = (TableExecuteTarget) target; + TableExecuteHandle newHandle = metadata.beginTableExecute(session, tableExecute.getHandle()); + return new TableExecuteTarget(newHandle, tableExecute.getSchemaTableName()); + } throw new IllegalArgumentException("Unhandled target type: " + target.getClass().getSimpleName()); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/plan/TableScanNode.java b/core/trino-main/src/main/java/io/trino/sql/planner/plan/TableScanNode.java index 3717e8599e4c..cc6436e69140 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/plan/TableScanNode.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/plan/TableScanNode.java @@ -253,4 +253,17 @@ public TableScanNode withUseConnectorNodePartitioning(boolean useConnectorNodePa updateTarget, Optional.of(useConnectorNodePartitioning)); } + + public TableScanNode withTableHandle(TableHandle table) + { + return new TableScanNode( + getId(), + table, + outputSymbols, + assignments, + enforcedConstraint, + statistics, + updateTarget, + useConnectorNodePartitioning); + } } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/plan/TableWriterNode.java b/core/trino-main/src/main/java/io/trino/sql/planner/plan/TableWriterNode.java index 5c3d0255d6c2..cd71d1854159 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/plan/TableWriterNode.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/plan/TableWriterNode.java @@ -25,6 +25,7 @@ import io.trino.metadata.NewTableLayout; import io.trino.metadata.OutputTableHandle; import io.trino.metadata.QualifiedObjectName; +import io.trino.metadata.TableExecuteHandle; import io.trino.metadata.TableHandle; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorTableMetadata; @@ -199,7 +200,9 @@ public PlanNode replaceChildren(List newChildren) @JsonSubTypes.Type(value = InsertTarget.class, name = "InsertTarget"), @JsonSubTypes.Type(value = DeleteTarget.class, name = "DeleteTarget"), @JsonSubTypes.Type(value = UpdateTarget.class, name = "UpdateTarget"), - @JsonSubTypes.Type(value = RefreshMaterializedViewTarget.class, name = "RefreshMaterializedViewTarget")}) + @JsonSubTypes.Type(value = RefreshMaterializedViewTarget.class, name = "RefreshMaterializedViewTarget"), + @JsonSubTypes.Type(value = TableExecuteTarget.class, name = "TableExecuteTarget") + }) @SuppressWarnings({"EmptyClass", "ClassMayBeInterface"}) public abstract static class WriterTarget { @@ -528,4 +531,38 @@ public String toString() return handle.map(Object::toString).orElse("[]"); } } + + public static class TableExecuteTarget + extends WriterTarget + { + private final TableExecuteHandle handle; + private final SchemaTableName schemaTableName; + + @JsonCreator + public TableExecuteTarget( + @JsonProperty("handle") TableExecuteHandle handle, + @JsonProperty("schemaTableName") SchemaTableName schemaTableName) + { + this.handle = requireNonNull(handle, "handle is null"); + this.schemaTableName = requireNonNull(schemaTableName, "schemaTableName is null"); + } + + @JsonProperty + public TableExecuteHandle getHandle() + { + return handle; + } + + @JsonProperty + public SchemaTableName getSchemaTableName() + { + return schemaTableName; + } + + @Override + public String toString() + { + return handle.toString(); + } + } } diff --git a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java index 795e6912a12a..6a680b716f59 100644 --- a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java @@ -77,6 +77,7 @@ import io.trino.execution.SetSessionTask; import io.trino.execution.SetTimeZoneTask; import io.trino.execution.StartTransactionTask; +import io.trino.execution.TableExecuteContextManager; import io.trino.execution.TaskManagerConfig; import io.trino.execution.TaskSource; import io.trino.execution.resourcegroups.NoOpResourceGroupManager; @@ -104,6 +105,7 @@ import io.trino.metadata.Split; import io.trino.metadata.SqlFunction; import io.trino.metadata.TableHandle; +import io.trino.metadata.TableProceduresPropertyManager; import io.trino.metadata.TablePropertyManager; import io.trino.operator.Driver; import io.trino.operator.DriverContext; @@ -348,6 +350,7 @@ private LocalQueryRunner( new MaterializedViewPropertyManager(), new ColumnPropertyManager(), new AnalyzePropertyManager(), + new TableProceduresPropertyManager(), new DisabledSystemSecurityMetadata(), transactionManager, typeOperators, @@ -801,6 +804,8 @@ private List createDrivers(Session session, Plan plan, OutputFactory out throw new AssertionError("Expected subplan to have no children"); } + TableExecuteContextManager tableExecuteContextManager = new TableExecuteContextManager(); + tableExecuteContextManager.registerTableExecuteContextForQuery(taskContext.getQueryContext().getQueryId()); LocalExecutionPlanner executionPlanner = new LocalExecutionPlanner( metadata, new TypeAnalyzer(sqlParser, metadata), @@ -824,7 +829,8 @@ private List createDrivers(Session session, Plan plan, OutputFactory out new OrderingCompiler(typeOperators), new DynamicFilterConfig(), typeOperators, - blockTypeOperators); + blockTypeOperators, + tableExecuteContextManager); // plan query StageExecutionDescriptor stageExecutionDescriptor = subplan.getFragment().getStageExecutionDescriptor(); diff --git a/core/trino-main/src/main/java/io/trino/util/StatementUtils.java b/core/trino-main/src/main/java/io/trino/util/StatementUtils.java index 8f7301db7d18..efa24a7d0734 100644 --- a/core/trino-main/src/main/java/io/trino/util/StatementUtils.java +++ b/core/trino-main/src/main/java/io/trino/util/StatementUtils.java @@ -73,6 +73,7 @@ import io.trino.sql.tree.ShowTables; import io.trino.sql.tree.StartTransaction; import io.trino.sql.tree.Statement; +import io.trino.sql.tree.TableExecute; import io.trino.sql.tree.Update; import io.trino.sql.tree.Use; @@ -158,6 +159,7 @@ private StatementUtils() {} .put(SetTimeZone.class, DATA_DEFINITION) .put(SetViewAuthorization.class, DATA_DEFINITION) .put(StartTransaction.class, DATA_DEFINITION) + .put(TableExecute.class, DATA_DEFINITION) // TODO: should we define it as DATA_DEFINITION if explictily use SqlQueryExecutionFactory for it? .put(Use.class, DATA_DEFINITION) .build(); diff --git a/core/trino-main/src/test/java/io/trino/execution/TaskTestUtils.java b/core/trino-main/src/test/java/io/trino/execution/TaskTestUtils.java index cb2512656de7..5c698ee583cd 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TaskTestUtils.java +++ b/core/trino-main/src/test/java/io/trino/execution/TaskTestUtils.java @@ -150,7 +150,8 @@ public static LocalExecutionPlanner createTestingPlanner() new OrderingCompiler(typeOperators), new DynamicFilterConfig(), typeOperators, - blockTypeOperators); + blockTypeOperators, + new TableExecuteContextManager()); } public static TaskInfo updateTask(SqlTask sqlTask, List taskSources, OutputBuffers outputBuffers) diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSourcePartitionedScheduler.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSourcePartitionedScheduler.java index d4f0c4a5e86b..7f0e0e28d82a 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSourcePartitionedScheduler.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSourcePartitionedScheduler.java @@ -29,6 +29,7 @@ import io.trino.execution.RemoteTask; import io.trino.execution.SqlStageExecution; import io.trino.execution.StageId; +import io.trino.execution.TableExecuteContextManager; import io.trino.execution.TableInfo; import io.trino.execution.buffer.OutputBuffers.OutputBufferId; import io.trino.failuredetector.NoOpFailureDetector; @@ -340,6 +341,7 @@ public void testNoNodes() new DynamicSplitPlacementPolicy(nodeScheduler.createNodeSelector(session, Optional.of(CONNECTOR_ID)), stage::getAllTasks), 2, new DynamicFilterService(metadata, typeOperators, new DynamicFilterConfig()), + new TableExecuteContextManager(), () -> false); scheduler.schedule(); }).hasErrorCode(NO_NODES_AVAILABLE); @@ -414,6 +416,7 @@ public void testNewTaskScheduledWhenChildStageBufferIsUnderutilized() new DynamicSplitPlacementPolicy(nodeScheduler.createNodeSelector(session, Optional.of(CONNECTOR_ID)), stage::getAllTasks), 500, new DynamicFilterService(metadata, typeOperators, new DynamicFilterConfig()), + new TableExecuteContextManager(), () -> false); // the queues of 3 running nodes should be full @@ -453,6 +456,7 @@ public void testNoNewTaskScheduledWhenChildStageBufferIsOverutilized() new DynamicSplitPlacementPolicy(nodeScheduler.createNodeSelector(session, Optional.of(CONNECTOR_ID)), stage::getAllTasks), 400, new DynamicFilterService(metadata, typeOperators, new DynamicFilterConfig()), + new TableExecuteContextManager(), () -> true); // the queues of 3 running nodes should be full @@ -490,6 +494,7 @@ public void testDynamicFiltersUnblockedOnBlockedBuildSource() new DynamicSplitPlacementPolicy(nodeScheduler.createNodeSelector(session, Optional.of(CONNECTOR_ID)), stage::getAllTasks), 2, dynamicFilterService, + new TableExecuteContextManager(), () -> true); SymbolAllocator symbolAllocator = new SymbolAllocator(); @@ -555,6 +560,7 @@ private StageScheduler getSourcePartitionedScheduler( placementPolicy, splitBatchSize, new DynamicFilterService(metadata, typeOperators, new DynamicFilterConfig()), + new TableExecuteContextManager(), () -> false); } diff --git a/core/trino-main/src/test/java/io/trino/metadata/AbstractMockMetadata.java b/core/trino-main/src/test/java/io/trino/metadata/AbstractMockMetadata.java index cc48b4900f38..248592c64c07 100644 --- a/core/trino-main/src/test/java/io/trino/metadata/AbstractMockMetadata.java +++ b/core/trino-main/src/test/java/io/trino/metadata/AbstractMockMetadata.java @@ -132,6 +132,30 @@ public Optional getTableHandleForStatisticsCollection(Session sessi throw new UnsupportedOperationException(); } + @Override + public Optional getTableHandleForExecute(Session session, TableHandle tableHandle, String procedureName, Map executeProperties, Constraint constraint) + { + throw new UnsupportedOperationException(); + } + + @Override + public Optional getLayoutForTableExecute(Session session, TableExecuteHandle tableExecuteHandle) + { + throw new UnsupportedOperationException(); + } + + @Override + public TableExecuteHandle beginTableExecute(Session session, TableExecuteHandle tableExecuteHandle) + { + throw new UnsupportedOperationException(); + } + + @Override + public void finishTableExecute(Session session, TableExecuteHandle handle, Collection fragments, List tableExecuteState) + { + throw new UnsupportedOperationException(); + } + @Override public Optional getSystemTable(Session session, QualifiedObjectName tableName) { @@ -810,6 +834,12 @@ public ProcedureRegistry getProcedureRegistry() throw new UnsupportedOperationException(); } + @Override + public TableProceduresRegistry getTableProcedureRegistry() + { + throw new UnsupportedOperationException(); + } + // // Blocks // @@ -860,6 +890,12 @@ public AnalyzePropertyManager getAnalyzePropertyManager() throw new UnsupportedOperationException(); } + @Override + public TableProceduresPropertyManager getTableProceduresPropertyManager() + { + throw new UnsupportedOperationException(); + } + @Override public Optional> applyProjection(Session session, TableHandle table, List projections, Map assignments) { diff --git a/core/trino-main/src/test/java/io/trino/operator/TestTableFinishOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestTableFinishOperator.java index 408ea0b259c2..b8379f03a659 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestTableFinishOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestTableFinishOperator.java @@ -18,6 +18,8 @@ import io.airlift.slice.Slice; import io.airlift.slice.Slices; import io.trino.Session; +import io.trino.execution.TableExecuteContext; +import io.trino.execution.TableExecuteContextManager; import io.trino.metadata.Metadata; import io.trino.operator.TableFinishOperator.TableFinishOperatorFactory; import io.trino.operator.TableFinishOperator.TableFinisher; @@ -95,6 +97,7 @@ public void testStatisticsAggregation() Session session = testSessionBuilder() .setSystemProperty("statistics_cpu_timer_enabled", "true") .build(); + TableExecuteContextManager tableExecuteContextManager = new TableExecuteContextManager(); TableFinishOperatorFactory operatorFactory = new TableFinishOperatorFactory( 0, new PlanNodeId("node"), @@ -106,10 +109,13 @@ public void testStatisticsAggregation() ImmutableList.of(LONG_MAX.bind(ImmutableList.of(2), Optional.empty())), true), descriptor, + tableExecuteContextManager, + true, session); DriverContext driverContext = createTaskContext(scheduledExecutor, scheduledExecutor, session) .addPipelineContext(0, true, true, false) .addDriverContext(); + tableExecuteContextManager.registerTableExecuteContextForQuery(driverContext.getPipelineContext().getTaskContext().getQueryContext().getQueryId()); TableFinishOperator operator = (TableFinishOperator) operatorFactory.createOperator(driverContext); List inputTypes = ImmutableList.of(BIGINT, VARBINARY, BIGINT); @@ -159,14 +165,16 @@ private static class TestTableFinisher private boolean finished; private Collection fragments; private Collection computedStatistics; + private TableExecuteContext tableExecuteContext; @Override - public Optional finishTable(Collection fragments, Collection computedStatistics) + public Optional finishTable(Collection fragments, Collection computedStatistics, TableExecuteContext tableExecuteContext) { checkState(!finished, "already finished"); finished = true; this.fragments = fragments; this.computedStatistics = computedStatistics; + this.tableExecuteContext = tableExecuteContext; return Optional.empty(); } @@ -179,5 +187,10 @@ public Collection getComputedStatistics() { return computedStatistics; } + + public TableExecuteContext getTableExecuteContext() + { + return tableExecuteContext; + } } } diff --git a/core/trino-main/src/test/java/io/trino/split/MockSplitSource.java b/core/trino-main/src/test/java/io/trino/split/MockSplitSource.java index 83fb7587886f..b1424c612b84 100644 --- a/core/trino-main/src/test/java/io/trino/split/MockSplitSource.java +++ b/core/trino-main/src/test/java/io/trino/split/MockSplitSource.java @@ -28,6 +28,7 @@ import java.util.Collections; import java.util.List; +import java.util.Optional; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; @@ -141,6 +142,12 @@ public boolean isFinished() return splitsProduced == totalSplits && atSplitDepletion == FINISH; } + @Override + public Optional> getTableExecuteSplitsInfo() + { + return Optional.empty(); + } + public int getNextBatchInvocationCount() { return nextBatchInvocationCount; diff --git a/core/trino-parser/src/main/antlr4/io/trino/sql/parser/SqlBase.g4 b/core/trino-parser/src/main/antlr4/io/trino/sql/parser/SqlBase.g4 index 0167fa00907c..2503f77534cd 100644 --- a/core/trino-parser/src/main/antlr4/io/trino/sql/parser/SqlBase.g4 +++ b/core/trino-parser/src/main/antlr4/io/trino/sql/parser/SqlBase.g4 @@ -70,6 +70,10 @@ statement | ALTER TABLE (IF EXISTS)? tableName=qualifiedName DROP COLUMN (IF EXISTS)? column=qualifiedName #dropColumn | ALTER TABLE tableName=qualifiedName SET AUTHORIZATION principal #setTableAuthorization + | ALTER TABLE tableName=qualifiedName + EXECUTE procedureName=identifier + (WITH properties)? + (WHERE where=booleanExpression)? #tableExecute | ANALYZE qualifiedName (WITH properties)? #analyze | CREATE (OR REPLACE)? MATERIALIZED VIEW (IF NOT EXISTS)? qualifiedName diff --git a/core/trino-parser/src/main/java/io/trino/sql/SqlFormatter.java b/core/trino-parser/src/main/java/io/trino/sql/SqlFormatter.java index 853619175fc8..462b66ff408a 100644 --- a/core/trino-parser/src/main/java/io/trino/sql/SqlFormatter.java +++ b/core/trino-parser/src/main/java/io/trino/sql/SqlFormatter.java @@ -117,6 +117,7 @@ import io.trino.sql.tree.SingleColumn; import io.trino.sql.tree.StartTransaction; import io.trino.sql.tree.Table; +import io.trino.sql.tree.TableExecute; import io.trino.sql.tree.TableSubquery; import io.trino.sql.tree.TransactionAccessMode; import io.trino.sql.tree.TransactionMode; @@ -1381,6 +1382,20 @@ protected Void visitDropColumn(DropColumn node, Integer indent) return null; } + protected Void visitTableExecute(TableExecute node, Integer indent) + { + builder.append("ALTER TABLE "); + builder.append(formatName(node.getTable().getName())); + builder.append(" EXECUTE "); + builder.append(formatExpression(node.getProcedureName())); + builder.append(formatPropertiesMultiLine(node.getProperties())); + node.getWhere().ifPresent(where -> + builder.append("\n") + .append(indentString(indent)) + .append("WHERE ").append(formatExpression(where))); + return null; + } + @Override protected Void visitAnalyze(Analyze node, Integer indent) { diff --git a/core/trino-parser/src/main/java/io/trino/sql/parser/AstBuilder.java b/core/trino-parser/src/main/java/io/trino/sql/parser/AstBuilder.java index 2f9122f7c206..22f98321ade7 100644 --- a/core/trino-parser/src/main/java/io/trino/sql/parser/AstBuilder.java +++ b/core/trino-parser/src/main/java/io/trino/sql/parser/AstBuilder.java @@ -206,6 +206,7 @@ import io.trino.sql.tree.SubsetDefinition; import io.trino.sql.tree.Table; import io.trino.sql.tree.TableElement; +import io.trino.sql.tree.TableExecute; import io.trino.sql.tree.TableSubquery; import io.trino.sql.tree.TimeLiteral; import io.trino.sql.tree.TimestampLiteral; @@ -643,6 +644,21 @@ public Node visitDropColumn(SqlBaseParser.DropColumnContext context) context.EXISTS().stream().anyMatch(node -> node.getSymbol().getTokenIndex() > context.COLUMN().getSymbol().getTokenIndex())); } + @Override + public Node visitTableExecute(SqlBaseParser.TableExecuteContext context) + { + List properties = ImmutableList.of(); + if (context.properties() != null) { + properties = visit(context.properties().property(), Property.class); + } + + return new TableExecute( + new Table(getLocation(context), getQualifiedName(context.tableName)), + (Identifier) visit(context.procedureName), + properties, + visitIfPresent(context.booleanExpression(), Expression.class)); + } + @Override public Node visitCreateView(SqlBaseParser.CreateViewContext context) { diff --git a/core/trino-parser/src/main/java/io/trino/sql/tree/AstVisitor.java b/core/trino-parser/src/main/java/io/trino/sql/tree/AstVisitor.java index a064856ae61d..831ad91b0321 100644 --- a/core/trino-parser/src/main/java/io/trino/sql/tree/AstVisitor.java +++ b/core/trino-parser/src/main/java/io/trino/sql/tree/AstVisitor.java @@ -662,6 +662,11 @@ protected R visitSetTableAuthorization(SetTableAuthorization node, C context) return visitStatement(node, context); } + protected R visitTableExecute(TableExecute node, C context) + { + return visitStatement(node, context); + } + protected R visitAnalyze(Analyze node, C context) { return visitStatement(node, context); diff --git a/core/trino-parser/src/main/java/io/trino/sql/tree/TableExecute.java b/core/trino-parser/src/main/java/io/trino/sql/tree/TableExecute.java new file mode 100644 index 000000000000..30689a19dae4 --- /dev/null +++ b/core/trino-parser/src/main/java/io/trino/sql/tree/TableExecute.java @@ -0,0 +1,133 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.sql.tree; + +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public class TableExecute + extends Statement +{ + private final Table table; + private final Identifier procedureName; + private final List properties; + private final Optional where; + + public TableExecute( + Table table, + Identifier procedureName, + List properties, + Optional where) + { + this(Optional.empty(), table, procedureName, properties, where); + } + + public TableExecute( + NodeLocation location, + Table table, + Identifier procedureName, + List properties, + Optional where) + { + this(Optional.of(location), table, procedureName, properties, where); + } + + private TableExecute( + Optional location, + Table table, + Identifier procedureName, + List properties, + Optional where) + { + super(location); + this.table = requireNonNull(table, "table is null"); + this.procedureName = requireNonNull(procedureName, "procedureName is null"); + this.properties = requireNonNull(properties, "properties is null"); + this.where = requireNonNull(where, "where is null"); + } + + public Table getTable() + { + return table; + } + + public Identifier getProcedureName() + { + return procedureName; + } + + public List getProperties() + { + return properties; + } + + public Optional getWhere() + { + return where; + } + + @Override + public R accept(AstVisitor visitor, C context) + { + return visitor.visitTableExecute(this, context); + } + + @Override + public List getChildren() + { + ImmutableList.Builder nodes = ImmutableList.builder(); + nodes.addAll(properties); + where.ifPresent(nodes::add); + return nodes.build(); + } + + @Override + public int hashCode() + { + return Objects.hash(table, procedureName, properties, where); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TableExecute that = (TableExecute) o; + return Objects.equals(table, that.table) && + Objects.equals(procedureName, that.procedureName) && + Objects.equals(properties, that.properties) && + Objects.equals(where, that.where); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("table", table) + .add("procedureNaem", procedureName) + .add("properties", properties) + .add("where", where) + .toString(); + } +} diff --git a/core/trino-parser/src/test/java/io/trino/sql/parser/TestSqlParser.java b/core/trino-parser/src/test/java/io/trino/sql/parser/TestSqlParser.java index 9fb6306255a5..53c054eaf480 100644 --- a/core/trino-parser/src/test/java/io/trino/sql/parser/TestSqlParser.java +++ b/core/trino-parser/src/test/java/io/trino/sql/parser/TestSqlParser.java @@ -171,6 +171,7 @@ import io.trino.sql.tree.SubscriptExpression; import io.trino.sql.tree.SubsetDefinition; import io.trino.sql.tree.Table; +import io.trino.sql.tree.TableExecute; import io.trino.sql.tree.TableSubquery; import io.trino.sql.tree.TimeLiteral; import io.trino.sql.tree.TimestampLiteral; @@ -1853,6 +1854,27 @@ public void testAlterViewSetAuthorization() new SetViewAuthorization(QualifiedName.of("foo", "bar", "baz"), new PrincipalSpecification(PrincipalSpecification.Type.ROLE, new Identifier("qux")))); } + @Test + public void testTableExecute() + { + Table table = new Table(QualifiedName.of("foo")); + Identifier procedure = new Identifier("bar"); + + assertStatement("ALTER TABLE foo EXECUTE bar", new TableExecute(table, procedure, ImmutableList.of(), Optional.empty())); + assertStatement( + "ALTER TABLE foo EXECUTE bar WITH(bah=1, wuh='clap') WHERE age > 17", + new TableExecute( + table, + procedure, + ImmutableList.of( + new Property(new Identifier("bah"), new LongLiteral("1")), + new Property(new Identifier("wuh"), new StringLiteral("clap"))), + Optional.of( + new ComparisonExpression(ComparisonExpression.Operator.GREATER_THAN, + new Identifier("age"), + new LongLiteral("17"))))); + } + @Test public void testAnalyze() { diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/Connector.java b/core/trino-spi/src/main/java/io/trino/spi/connector/Connector.java index 795f05f77ca5..b52b72c24f97 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/Connector.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/Connector.java @@ -108,6 +108,11 @@ default Set getProcedures() return emptySet(); } + default Set getTableProcedures() + { + return emptySet(); + } + /** * @return the system properties for this connector */ diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorAccessControl.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorAccessControl.java index 3a4477149f54..8f22639da988 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorAccessControl.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorAccessControl.java @@ -38,6 +38,7 @@ import static io.trino.spi.security.AccessDeniedException.denyDropTable; import static io.trino.spi.security.AccessDeniedException.denyDropView; import static io.trino.spi.security.AccessDeniedException.denyExecuteProcedure; +import static io.trino.spi.security.AccessDeniedException.denyExecuteTableProcedure; import static io.trino.spi.security.AccessDeniedException.denyGrantRoles; import static io.trino.spi.security.AccessDeniedException.denyGrantSchemaPrivilege; import static io.trino.spi.security.AccessDeniedException.denyGrantTablePrivilege; @@ -532,6 +533,11 @@ default void checkCanExecuteProcedure(ConnectorSecurityContext context, SchemaRo denyExecuteProcedure(procedure.toString()); } + default void checkCanExecuteTableProcedure(ConnectorSecurityContext context, SchemaTableName tableName, String procedure) + { + denyExecuteTableProcedure(tableName.toString(), procedure); + } + /** * Get a row filter associated with the given table and identity. *

diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorHandleResolver.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorHandleResolver.java index 8e65748008e2..347e47a4a3ae 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorHandleResolver.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorHandleResolver.java @@ -50,6 +50,11 @@ default Class getInsertTableHandleClass() throw new UnsupportedOperationException(); } + default Class getTableExecuteHandleClass() + { + throw new UnsupportedOperationException(); + } + default Class getPartitioningHandleClass() { throw new UnsupportedOperationException(); diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java index aec03b2c8e5a..8e23ed531e7d 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java @@ -91,6 +91,43 @@ default ConnectorTableHandle getTableHandleForStatisticsCollection(ConnectorSess throw new TrinoException(NOT_SUPPORTED, "This connector does not support analyze"); } + /** + * Create initial handle for execution of table procedure. The handle will be used through planning process. It will be converted to final + * handle used for execution via @{link {@link ConnectorMetadata#}beginTableExecute} + */ + default Optional getTableHandleForExecute( + ConnectorSession session, + ConnectorTableHandle tableHandle, + String procedureName, + Map executeProperties, + Constraint constraint) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support table procedures"); + } + + default Optional getLayoutForTableExecute(ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle) + { + return Optional.empty(); + } + + /** + * Begin execution of table procedure + */ + default ConnectorTableExecuteHandle beginTableExecute(ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support table procedures"); + } + + /** + * Finish table execute + * + * @param fragments all fragments returned by {@link ConnectorPageSink#finish()} + */ + default void finishTableExecute(ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle, Collection fragments, List tableExecuteState) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support table procedures"); + } + /** * Returns the system table for the specified table name, if one exists. * The system tables handled via {@link #getSystemTable} differ form those returned by {@link Connector#getSystemTables()}. diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorNewTableLayout.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorNewTableLayout.java index 718e08ea4bcf..c57c428a7c0b 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorNewTableLayout.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorNewTableLayout.java @@ -18,6 +18,7 @@ import static java.util.Objects.requireNonNull; +// TODO ConnectorNewTableLayout is used not only for "new" tables. Rename to be less specific. Preferably to ConnectorTableLayout after https://github.com/trinodb/trino/issues/781 public class ConnectorNewTableLayout { private final Optional partitioning; diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorPageSinkProvider.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorPageSinkProvider.java index 345b3d72cdeb..2b7363621f26 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorPageSinkProvider.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorPageSinkProvider.java @@ -18,4 +18,9 @@ public interface ConnectorPageSinkProvider ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorOutputTableHandle outputTableHandle); ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorInsertTableHandle insertTableHandle); + + default ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle) + { + throw new IllegalArgumentException("createPageSink not supported for tableExecuteHandle"); + } } diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorSplitSource.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorSplitSource.java index c459e29b19fb..d585644d6420 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorSplitSource.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorSplitSource.java @@ -15,6 +15,7 @@ import java.io.Closeable; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import static java.util.Objects.requireNonNull; @@ -37,6 +38,11 @@ public interface ConnectorSplitSource */ boolean isFinished(); + default Optional> getTableExecuteSplitsInfo() + { + return Optional.empty(); + } + class ConnectorSplitBatch { private final List splits; diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorTableExecuteHandle.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorTableExecuteHandle.java new file mode 100644 index 000000000000..b18c297cfd71 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorTableExecuteHandle.java @@ -0,0 +1,19 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.spi.connector; + +public interface ConnectorTableExecuteHandle +{ + ConnectorTableHandle getSourceTableHandle(); +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/FixedSplitSource.java b/core/trino-spi/src/main/java/io/trino/spi/connector/FixedSplitSource.java index e6b6db94a404..f7ab8f914c5a 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/FixedSplitSource.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/FixedSplitSource.java @@ -14,7 +14,9 @@ package io.trino.spi.connector; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static java.util.Objects.requireNonNull; @@ -26,12 +28,25 @@ public class FixedSplitSource implements ConnectorSplitSource { private final List splits; + private final Optional> tableExecuteSplitsInfo; private int offset; public FixedSplitSource(Iterable splits) + { + this(splits, Optional.empty()); + } + + public FixedSplitSource(Iterable splits, List tableExecuteSplitsInfo) + { + this(splits, Optional.of(tableExecuteSplitsInfo)); + } + + private FixedSplitSource(Iterable splits, Optional> tableExecuteSplitsInfo) { requireNonNull(splits, "splits is null"); + requireNonNull(tableExecuteSplitsInfo, "tableExecuteSplitsInfo is null"); this.splits = stream(splits.spliterator(), false).collect(toUnmodifiableList()); + this.tableExecuteSplitsInfo = tableExecuteSplitsInfo.map(infos -> infos.stream().collect(Collectors.toUnmodifiableList())); } @SuppressWarnings("ObjectEquality") @@ -56,6 +71,12 @@ public boolean isFinished() return offset >= splits.size(); } + @Override + public Optional> getTableExecuteSplitsInfo() + { + return tableExecuteSplitsInfo; + } + @Override public void close() { diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/TableProcedureExecutionMode.java b/core/trino-spi/src/main/java/io/trino/spi/connector/TableProcedureExecutionMode.java new file mode 100644 index 000000000000..2bd5616eef24 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/TableProcedureExecutionMode.java @@ -0,0 +1,60 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.spi.connector; + +public final class TableProcedureExecutionMode +{ + private final boolean readsData; + private final boolean supportsFilter; + + public TableProcedureExecutionMode(boolean readsData, boolean supportsFilter) + { + this.readsData = readsData; + + if (!readsData) { + // TODO currently only table procedures which process data are supported + // this is temporary check to be dropped when execution flow will be added for + // table procedures which do not read data + throw new IllegalArgumentException("repartitioning not supported if table data is not processed"); + } + + if (!readsData) { + if (supportsFilter) { + throw new IllegalArgumentException("filtering not supported if table data is not processed"); + } + } + + this.supportsFilter = supportsFilter; + } + + public boolean isReadsData() + { + return readsData; + } + + public boolean supportsFilter() + { + return supportsFilter; + } + + public static TableProcedureExecutionMode coordinatorOnly() + { + return new TableProcedureExecutionMode(false, false); + } + + public static TableProcedureExecutionMode distributedWithFilteringAndRepartitioning() + { + return new TableProcedureExecutionMode(true, true); + } +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/TableProcedureMetadata.java b/core/trino-spi/src/main/java/io/trino/spi/connector/TableProcedureMetadata.java new file mode 100644 index 000000000000..226c2b2ef3a6 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/TableProcedureMetadata.java @@ -0,0 +1,51 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.spi.connector; + +import io.trino.spi.session.PropertyMetadata; + +import java.util.List; + +import static io.trino.spi.connector.SchemaUtil.checkNotEmpty; +import static java.util.Objects.requireNonNull; + +public class TableProcedureMetadata +{ + // Name must be uppercase if procedure is to be executed without delimitation via ALTER TABLE ... EXECUTE syntax + private final String name; + private final TableProcedureExecutionMode executionMode; + private final List> properties; + + public TableProcedureMetadata(String name, TableProcedureExecutionMode executionMode, List> properties) + { + this.name = checkNotEmpty(name, "name"); + this.executionMode = requireNonNull(executionMode, "executionMode is null"); + this.properties = List.copyOf(requireNonNull(properties, "properties is null")); + } + + public String getName() + { + return name; + } + + public TableProcedureExecutionMode getExecutionMode() + { + return executionMode; + } + + public List> getProperties() + { + return properties; + } +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/security/AccessDeniedException.java b/core/trino-spi/src/main/java/io/trino/spi/security/AccessDeniedException.java index 20529940713d..f0bb3d0a585c 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/security/AccessDeniedException.java +++ b/core/trino-spi/src/main/java/io/trino/spi/security/AccessDeniedException.java @@ -558,6 +558,11 @@ public static void denyExecuteFunction(String functionName) throw new AccessDeniedException(format("Cannot execute function %s", functionName)); } + public static void denyExecuteTableProcedure(String tableName, String procedureName) + { + throw new AccessDeniedException(format("Cannot execute table procedure %s on %s", procedureName, tableName)); + } + private static Object formatExtraInfo(String extraInfo) { if (extraInfo == null || extraInfo.isEmpty()) { diff --git a/core/trino-spi/src/main/java/io/trino/spi/security/SystemAccessControl.java b/core/trino-spi/src/main/java/io/trino/spi/security/SystemAccessControl.java index 438b429c3fad..4a74b7940b50 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/security/SystemAccessControl.java +++ b/core/trino-spi/src/main/java/io/trino/spi/security/SystemAccessControl.java @@ -45,6 +45,7 @@ import static io.trino.spi.security.AccessDeniedException.denyExecuteFunction; import static io.trino.spi.security.AccessDeniedException.denyExecuteProcedure; import static io.trino.spi.security.AccessDeniedException.denyExecuteQuery; +import static io.trino.spi.security.AccessDeniedException.denyExecuteTableProcedure; import static io.trino.spi.security.AccessDeniedException.denyGrantExecuteFunctionPrivilege; import static io.trino.spi.security.AccessDeniedException.denyGrantRoles; import static io.trino.spi.security.AccessDeniedException.denyGrantSchemaPrivilege; @@ -704,6 +705,16 @@ default void checkCanExecuteFunction(SystemSecurityContext systemSecurityContext denyExecuteFunction(functionName); } + /** + * Check if identity is allowed to execute the specified table procedure on specified table + * + * @throws AccessDeniedException if not allowed + */ + default void checkCanExecuteTableProcedure(SystemSecurityContext systemSecurityContext, CatalogSchemaTableName table, String procedure) + { + denyExecuteTableProcedure(table.toString(), procedure); + } + /** * Get a row filter associated with the given table and identity. *

diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorAccessControl.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorAccessControl.java index b5aaf346e943..0024f1fdfc44 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorAccessControl.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorAccessControl.java @@ -443,6 +443,14 @@ public void checkCanExecuteProcedure(ConnectorSecurityContext context, SchemaRou } } + @Override + public void checkCanExecuteTableProcedure(ConnectorSecurityContext context, SchemaTableName tableName, String procedure) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + delegate.checkCanExecuteTableProcedure(context, tableName, procedure); + } + } + @Override public Optional getRowFilter(ConnectorSecurityContext context, SchemaTableName tableName) { diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java index 0845c47d4fa4..122514ac1f5f 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java @@ -30,6 +30,7 @@ import io.trino.spi.connector.ConnectorPartitioningHandle; import io.trino.spi.connector.ConnectorResolvedIndex; import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTableExecuteHandle; import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.ConnectorTableLayout; import io.trino.spi.connector.ConnectorTableLayoutHandle; @@ -216,6 +217,38 @@ public ConnectorTableHandle getTableHandleForStatisticsCollection(ConnectorSessi } } + @Override + public Optional getTableHandleForExecute(ConnectorSession session, ConnectorTableHandle tableHandle, String procedureName, Map executeProperties, Constraint constraint) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + return delegate.getTableHandleForExecute(session, tableHandle, procedureName, executeProperties, constraint); + } + } + + @Override + public Optional getLayoutForTableExecute(ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + return delegate.getLayoutForTableExecute(session, tableExecuteHandle); + } + } + + @Override + public ConnectorTableExecuteHandle beginTableExecute(ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + return delegate.beginTableExecute(session, tableExecuteHandle); + } + } + + @Override + public void finishTableExecute(ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle, Collection fragments, List tableExecuteState) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + delegate.finishTableExecute(session, tableExecuteHandle, fragments, tableExecuteState); + } + } + @Override public Optional getSystemTable(ConnectorSession session, SchemaTableName tableName) { diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorPageSinkProvider.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorPageSinkProvider.java index 4898cf8da4f1..88ab689cca07 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorPageSinkProvider.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorPageSinkProvider.java @@ -19,6 +19,7 @@ import io.trino.spi.connector.ConnectorPageSink; import io.trino.spi.connector.ConnectorPageSinkProvider; import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTableExecuteHandle; import io.trino.spi.connector.ConnectorTransactionHandle; import javax.inject.Inject; @@ -53,4 +54,12 @@ public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHa return new ClassLoaderSafeConnectorPageSink(delegate.createPageSink(transactionHandle, session, insertTableHandle), classLoader); } } + + @Override + public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + return new ClassLoaderSafeConnectorPageSink(delegate.createPageSink(transactionHandle, session, tableExecuteHandle), classLoader); + } + } } diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorSplitSource.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorSplitSource.java index 11479eb101e5..ca388c69ccfa 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorSplitSource.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorSplitSource.java @@ -19,6 +19,8 @@ import javax.inject.Inject; +import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import static java.util.Objects.requireNonNull; @@ -44,6 +46,14 @@ public CompletableFuture getNextBatch(ConnectorPartitionHan } } + @Override + public Optional> getTableExecuteSplitsInfo() + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + return delegate.getTableExecuteSplitsInfo(); + } + } + @Override public void close() { diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/AllowAllAccessControl.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/AllowAllAccessControl.java index aa14a5bb118d..8775f8573eed 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/AllowAllAccessControl.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/AllowAllAccessControl.java @@ -284,6 +284,11 @@ public void checkCanExecuteProcedure(ConnectorSecurityContext context, SchemaRou { } + @Override + public void checkCanExecuteTableProcedure(ConnectorSecurityContext context, SchemaTableName tableName, String procedure) + { + } + @Override public Optional getRowFilter(ConnectorSecurityContext context, SchemaTableName tableName) { diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/AllowAllSystemAccessControl.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/AllowAllSystemAccessControl.java index 9aaefeb100e8..955200de0646 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/AllowAllSystemAccessControl.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/AllowAllSystemAccessControl.java @@ -380,6 +380,11 @@ public void checkCanExecuteFunction(SystemSecurityContext systemSecurityContext, { } + @Override + public void checkCanExecuteTableProcedure(SystemSecurityContext systemSecurityContext, CatalogSchemaTableName table, String procedure) + { + } + @Override public Iterable getEventListeners() { diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/FileBasedAccessControl.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/FileBasedAccessControl.java index 3a1de62a8c79..cb8387aa7510 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/FileBasedAccessControl.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/FileBasedAccessControl.java @@ -540,6 +540,11 @@ public void checkCanExecuteProcedure(ConnectorSecurityContext context, SchemaRou { } + @Override + public void checkCanExecuteTableProcedure(ConnectorSecurityContext context, SchemaTableName tableName, String procedure) + { + } + @Override public Optional getRowFilter(ConnectorSecurityContext context, SchemaTableName tableName) { diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/FileBasedSystemAccessControl.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/FileBasedSystemAccessControl.java index 2e3767611cbe..bef441980d98 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/FileBasedSystemAccessControl.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/FileBasedSystemAccessControl.java @@ -890,6 +890,11 @@ public void checkCanExecuteFunction(SystemSecurityContext systemSecurityContext, { } + @Override + public void checkCanExecuteTableProcedure(SystemSecurityContext systemSecurityContext, CatalogSchemaTableName table, String procedure) + { + } + @Override public Iterable getEventListeners() { diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/ForwardingConnectorAccessControl.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/ForwardingConnectorAccessControl.java index 099628f7ab44..049fcd960654 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/ForwardingConnectorAccessControl.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/ForwardingConnectorAccessControl.java @@ -348,6 +348,12 @@ public void checkCanExecuteProcedure(ConnectorSecurityContext context, SchemaRou delegate().checkCanExecuteProcedure(context, procedure); } + @Override + public void checkCanExecuteTableProcedure(ConnectorSecurityContext context, SchemaTableName tableName, String procedure) + { + delegate().checkCanExecuteTableProcedure(context, tableName, procedure); + } + @Override public Optional getRowFilter(ConnectorSecurityContext context, SchemaTableName tableName) { diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/ForwardingSystemAccessControl.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/ForwardingSystemAccessControl.java index 395af5d2fa26..5957c3451fcf 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/ForwardingSystemAccessControl.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/security/ForwardingSystemAccessControl.java @@ -416,6 +416,12 @@ public void checkCanExecuteFunction(SystemSecurityContext systemSecurityContext, delegate().checkCanExecuteFunction(systemSecurityContext, functionName); } + @Override + public void checkCanExecuteTableProcedure(SystemSecurityContext systemSecurityContext, CatalogSchemaTableName table, String procedure) + { + delegate().checkCanExecuteTableProcedure(systemSecurityContext, table, procedure); + } + @Override public Iterable getEventListeners() { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java index fecc6d4bb132..597707953b2f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java @@ -164,6 +164,7 @@ public class BackgroundHiveSplitLoader private final ConcurrentLazyQueue partitions; private final Deque> fileIterators = new ConcurrentLinkedDeque<>(); private final Optional validWriteIds; + private final Optional maxSplitFileSize; // Purpose of this lock: // * Write lock: when you need a consistent view across partitions, fileIterators, and hiveSplitSource. @@ -204,7 +205,8 @@ public BackgroundHiveSplitLoader( boolean recursiveDirWalkerEnabled, boolean ignoreAbsentPartitions, boolean optimizeSymlinkListing, - Optional validWriteIds) + Optional validWriteIds, + Optional maxSplitFileSize) { this.table = table; this.transaction = requireNonNull(transaction, "transaction is null"); @@ -226,6 +228,7 @@ public BackgroundHiveSplitLoader( this.partitions = new ConcurrentLazyQueue<>(partitions); this.hdfsContext = new HdfsContext(session); this.validWriteIds = requireNonNull(validWriteIds, "validWriteIds is null"); + this.maxSplitFileSize = requireNonNull(maxSplitFileSize, "maxSplitFileSize is null"); } @Override @@ -465,7 +468,8 @@ private ListenableFuture loadPartition(HivePartitionMetadata partition) getMaxInitialSplitSize(session), isForceLocalScheduling(session), s3SelectPushdownEnabled, - transaction); + transaction, + maxSplitFileSize); // To support custom input formats, we want to call getSplits() // on the input format to obtain file splits. @@ -653,7 +657,8 @@ private ListenableFuture createHiveSymlinkSplits( getMaxInitialSplitSize(session), isForceLocalScheduling(session), s3SelectPushdownEnabled, - transaction); + transaction, + maxSplitFileSize); lastResult = addSplitsToSource(targetSplits, splitFactory); if (stopped) { return COMPLETED_FUTURE; @@ -709,7 +714,8 @@ Optional> buildManifestFileIterator( getMaxInitialSplitSize(session), isForceLocalScheduling(session), s3SelectPushdownEnabled, - transaction); + transaction, + maxSplitFileSize); return Optional.of(locatedFileStatuses.stream() .map(locatedFileStatus -> splitFactory.createInternalHiveSplit(locatedFileStatus, OptionalInt.empty(), splittable, Optional.empty())) .filter(Optional::isPresent) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConnector.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConnector.java index c11dce46aeff..8c1f2f883dad 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConnector.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConnector.java @@ -29,6 +29,7 @@ import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.ConnectorTransactionHandle; import io.trino.spi.connector.SystemTable; +import io.trino.spi.connector.TableProcedureMetadata; import io.trino.spi.eventlistener.EventListener; import io.trino.spi.procedure.Procedure; import io.trino.spi.session.PropertyMetadata; @@ -55,6 +56,7 @@ public class HiveConnector private final ConnectorNodePartitioningProvider nodePartitioningProvider; private final Set systemTables; private final Set procedures; + private final Set tableProcedures; private final Set eventListeners; private final List> sessionProperties; private final List> schemaProperties; @@ -77,6 +79,7 @@ public HiveConnector( ConnectorNodePartitioningProvider nodePartitioningProvider, Set systemTables, Set procedures, + Set tableProcedures, Set eventListeners, Set sessionPropertiesProviders, List> schemaProperties, @@ -95,6 +98,7 @@ public HiveConnector( this.nodePartitioningProvider = requireNonNull(nodePartitioningProvider, "nodePartitioningProvider is null"); this.systemTables = ImmutableSet.copyOf(requireNonNull(systemTables, "systemTables is null")); this.procedures = ImmutableSet.copyOf(requireNonNull(procedures, "procedures is null")); + this.tableProcedures = ImmutableSet.copyOf(requireNonNull(tableProcedures, "tableProcedures is null")); this.eventListeners = ImmutableSet.copyOf(requireNonNull(eventListeners, "eventListeners is null")); this.sessionProperties = requireNonNull(sessionPropertiesProviders, "sessionPropertiesProviders is null").stream() .flatMap(sessionPropertiesProvider -> sessionPropertiesProvider.getSessionProperties().stream()) @@ -241,4 +245,10 @@ public final void shutdown() { lifeCycleManager.stop(); } + + @Override + public Set getTableProcedures() + { + return tableProcedures; + } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveHandleResolver.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveHandleResolver.java index f02a499af864..f7a889e524d0 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveHandleResolver.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveHandleResolver.java @@ -19,6 +19,7 @@ import io.trino.spi.connector.ConnectorOutputTableHandle; import io.trino.spi.connector.ConnectorPartitioningHandle; import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorTableExecuteHandle; import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.ConnectorTransactionHandle; @@ -55,6 +56,12 @@ public Class getInsertTableHandleClass() return HiveInsertTableHandle.class; } + @Override + public Class getTableExecuteHandleClass() + { + return HiveTableExecuteHandle.class; + } + @Override public Class getTransactionHandleClass() { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveLocationService.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveLocationService.java index 3e7f03f1bfe6..78d84c4f2106 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveLocationService.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveLocationService.java @@ -88,6 +88,14 @@ public LocationHandle forExistingTable(SemiTransactionalHiveMetastore metastore, } } + @Override + public LocationHandle forOptimize(SemiTransactionalHiveMetastore metastore, ConnectorSession session, Table table) + { + // For OPTIMIZE write result files directly to table directory; that is needed by the commit logic in HiveMetadata#finishTableExecute + Path targetPath = new Path(table.getStorage().getLocation()); + return new LocationHandle(targetPath, targetPath, true, DIRECT_TO_TARGET_EXISTING_DIRECTORY); + } + private boolean shouldUseTemporaryDirectory(ConnectorSession session, HdfsContext context, Path path, Optional externalLocation) { return isTemporaryStagingDirectoryEnabled(session) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java index 1a0525026c73..1a144f0701d3 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java @@ -25,7 +25,9 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import io.airlift.json.JsonCodec; +import io.airlift.log.Logger; import io.airlift.slice.Slice; +import io.airlift.units.DataSize; import io.trino.plugin.base.CatalogName; import io.trino.plugin.hive.HdfsEnvironment.HdfsContext; import io.trino.plugin.hive.HiveApplyProjectionUtil.ProjectedColumnRepresentation; @@ -45,6 +47,7 @@ import io.trino.plugin.hive.metastore.SortingColumn; import io.trino.plugin.hive.metastore.StorageFormat; import io.trino.plugin.hive.metastore.Table; +import io.trino.plugin.hive.procedure.OptimizeTableProcedure; import io.trino.plugin.hive.security.AccessControlMetadata; import io.trino.plugin.hive.statistics.HiveStatisticsProvider; import io.trino.plugin.hive.util.HiveBucketing; @@ -65,6 +68,7 @@ import io.trino.spi.connector.ConnectorOutputTableHandle; import io.trino.spi.connector.ConnectorPartitioningHandle; import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTableExecuteHandle; import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.ConnectorTableMetadata; import io.trino.spi.connector.ConnectorTablePartitioning; @@ -123,6 +127,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; @@ -135,6 +140,7 @@ import java.util.function.Function; import java.util.function.Predicate; import java.util.function.Supplier; +import java.util.stream.Collectors; import static com.google.common.base.MoreObjects.firstNonNull; import static com.google.common.base.Preconditions.checkArgument; @@ -219,6 +225,7 @@ import static io.trino.plugin.hive.HiveTableProperties.getPartitionedBy; import static io.trino.plugin.hive.HiveTableProperties.getSingleCharacterProperty; import static io.trino.plugin.hive.HiveTableProperties.isTransactional; +import static io.trino.plugin.hive.HiveTimestampPrecision.NANOSECONDS; import static io.trino.plugin.hive.HiveType.HIVE_STRING; import static io.trino.plugin.hive.HiveType.toHiveType; import static io.trino.plugin.hive.HiveWriterFactory.computeBucketedFileName; @@ -257,6 +264,7 @@ import static io.trino.plugin.hive.util.HiveWriteUtils.isFileCreatedByQuery; import static io.trino.plugin.hive.util.HiveWriteUtils.isS3FileSystem; import static io.trino.plugin.hive.util.HiveWriteUtils.isWritableType; +import static io.trino.plugin.hive.util.RetryDriver.retry; import static io.trino.plugin.hive.util.Statistics.ReduceOperator.ADD; import static io.trino.plugin.hive.util.Statistics.createComputedStatisticsToPartitionMap; import static io.trino.plugin.hive.util.Statistics.createEmptyPartitionStatistics; @@ -278,7 +286,6 @@ import static java.util.Objects.requireNonNull; import static java.util.function.Function.identity; import static java.util.stream.Collectors.joining; -import static java.util.stream.Collectors.toList; import static java.util.stream.Collectors.toMap; import static org.apache.hadoop.hive.metastore.TableType.EXTERNAL_TABLE; import static org.apache.hadoop.hive.metastore.TableType.MANAGED_TABLE; @@ -290,6 +297,8 @@ public class HiveMetadata implements TransactionalMetadata { + private static final Logger log = Logger.get(HiveMetadata.class); + public static final String PRESTO_VERSION_NAME = "presto_version"; public static final String TRINO_CREATED_BY = "trino_created_by"; public static final String PRESTO_QUERY_ID_NAME = "presto_query_id"; @@ -545,7 +554,7 @@ private ConnectorTableMetadata doGetTableMetadata(ConnectorSession session, Sche // Partitioning property List partitionedBy = table.getPartitionColumns().stream() .map(Column::getName) - .collect(toList()); + .collect(toImmutableList()); if (!partitionedBy.isEmpty()) { properties.put(PARTITIONED_BY_PROPERTY, partitionedBy); } @@ -814,7 +823,7 @@ public void createTable(ConnectorSession session, ConnectorTableMetadata tableMe List partitionColumns = partitionedBy.stream() .map(columnHandlesByName::get) .map(HiveColumnHandle::toMetastoreColumn) - .collect(toList()); + .collect(toImmutableList()); checkPartitionTypesSupported(partitionColumns); Optional targetPath; @@ -1060,7 +1069,7 @@ private static Table buildTableObject( List partitionColumns = partitionedBy.stream() .map(columnHandlesByName::get) .map(HiveColumnHandle::toMetastoreColumn) - .collect(toList()); + .collect(toImmutableList()); Set partitionColumnNames = ImmutableSet.copyOf(partitionedBy); @@ -1305,7 +1314,7 @@ public HiveOutputTableHandle beginCreateTable(ConnectorSession session, Connecto List partitionColumns = partitionedBy.stream() .map(columnHandlesByName::get) .map(HiveColumnHandle::toMetastoreColumn) - .collect(toList()); + .collect(toImmutableList()); checkPartitionTypesSupported(partitionColumns); LocationHandle locationHandle = locationService.forNewTable(metastore, session, schemaName, tableName, externalLocation); @@ -1342,7 +1351,7 @@ public Optional finishCreateTable(ConnectorSession sess List partitionUpdates = fragments.stream() .map(Slice::getBytes) .map(partitionUpdateCodec::fromJson) - .collect(toList()); + .collect(toImmutableList()); WriteInfo writeInfo = locationService.getQueryWriteInfo(handle.getLocationHandle()); Table table = buildTableObject( @@ -1570,7 +1579,7 @@ public ConnectorTableHandle beginUpdate(ConnectorSession session, ConnectorTable List allDataColumns = getRegularColumnHandles(table, typeManager, getTimestampPrecision(session)).stream() .filter(columnHandle -> !columnHandle.isHidden()) - .collect(toList()); + .collect(toImmutableList()); List hiveUpdatedColumns = updatedColumns.stream().map(HiveColumnHandle.class::cast).collect(toImmutableList()); if (table.getParameters().containsKey(SKIP_HEADER_COUNT_KEY)) { @@ -1606,7 +1615,7 @@ public void finishUpdate(ConnectorSession session, ConnectorTableHandle tableHan List partitionAndStatementIds = fragments.stream() .map(Slice::getBytes) .map(PartitionAndStatementId.CODEC::fromJson) - .collect(toList()); + .collect(toImmutableList()); HdfsContext context = new HdfsContext(session); for (PartitionAndStatementId ps : partitionAndStatementIds) { @@ -1636,7 +1645,7 @@ public HiveInsertTableHandle beginInsert(ConnectorSession session, ConnectorTabl List handles = hiveColumnHandles(table, typeManager, getTimestampPrecision(session)).stream() .filter(columnHandle -> !columnHandle.isHidden()) - .collect(toList()); + .collect(toImmutableList()); HiveStorageFormat tableStorageFormat = extractHiveStorageFormat(table); Optional.ofNullable(table.getParameters().get(SKIP_HEADER_COUNT_KEY)).map(Integer::parseInt).ifPresent(headerSkipCount -> { @@ -1680,7 +1689,7 @@ public Optional finishInsert(ConnectorSession session, List partitionUpdates = fragments.stream() .map(Slice::getBytes) .map(partitionUpdateCodec::fromJson) - .collect(toList()); + .collect(toImmutableList()); HiveStorageFormat tableStorageFormat = handle.getTableStorageFormat(); partitionUpdates = PartitionUpdate.mergePartitionUpdates(partitionUpdates); @@ -1894,6 +1903,229 @@ private static Map getColumnStatistics(Map getTableHandleForExecute(ConnectorSession session, ConnectorTableHandle tableHandle, String procedureName, Map executeProperties, Constraint constraint) + { + if (procedureName.equals(OptimizeTableProcedure.NAME)) { + return getTableHandleForOptimize(session, tableHandle, executeProperties, constraint); + } + throw new IllegalArgumentException("Unknown procedure '" + procedureName + "'"); + } + + private Optional getTableHandleForOptimize(ConnectorSession session, ConnectorTableHandle tableHandle, Map executeProperties, Constraint constraint) + { + // TODO lots of that is copied from beginInsert; rafactoring opportunity + + HiveIdentity identity = new HiveIdentity(session); + HiveTableHandle hiveTableHandle = (HiveTableHandle) tableHandle; + SchemaTableName tableName = hiveTableHandle.getSchemaTableName(); + + if (constraint.getSummary().isNone()) { + throw new TrinoException(NOT_SUPPORTED, format("Optimizing Hive table %s with predicate filtering out all data is not supported", tableName)); + } + + if (constraint.predicate().isPresent() || !isOnPartitionColumnsOnly(constraint.getSummary(), hiveTableHandle.getPartitionColumns())) { + throw new TrinoException(NOT_SUPPORTED, "Predicate used for optimize must match whole partitions"); + } + + Table table = metastore.getTable(identity, tableName.getSchemaName(), tableName.getTableName()) + .orElseThrow(() -> new TableNotFoundException(tableName)); + + checkTableIsWritable(table, writesToNonManagedTablesEnabled); + + for (Column column : table.getDataColumns()) { + if (!isWritableType(column.getType())) { + throw new TrinoException(NOT_SUPPORTED, format("Optimizing Hive table %s with column type %s not supported", tableName, column.getType())); + } + } + + if (isTransactionalTable(table.getParameters())) { + throw new TrinoException(NOT_SUPPORTED, format("Optimizing transactional Hive table %s is not supported", tableName)); + } + + if (table.getStorage().getBucketProperty().isPresent()) { + throw new TrinoException(NOT_SUPPORTED, format("Optimizing bucketed Hive table %s is not supported", tableName)); + } + + // TODO forcing NANOSECONDS precision here so we do not loose data. In future we may be smarter; options: + // - respect timestamp_precision but recognize situation when rounding occurs, and fail query + // - detect data's precision and maintain it + List columns = hiveColumnHandles(table, typeManager, NANOSECONDS).stream() + .filter(columnHandle -> !columnHandle.isHidden()) + .collect(toImmutableList()); + + HiveStorageFormat tableStorageFormat = extractHiveStorageFormat(table); + Optional.ofNullable(table.getParameters().get(SKIP_HEADER_COUNT_KEY)).map(Integer::parseInt).ifPresent(headerSkipCount -> { + if (headerSkipCount > 1) { + throw new TrinoException(NOT_SUPPORTED, format("Optimizing Hive table %s with value of %s property greater than 1 is not supported", tableName, SKIP_HEADER_COUNT_KEY)); + } + }); + + if (table.getParameters().containsKey(SKIP_FOOTER_COUNT_KEY)) { + throw new TrinoException(NOT_SUPPORTED, format("Optimizing Hive table %s with %s property not supported", tableName, SKIP_FOOTER_COUNT_KEY)); + } + LocationHandle locationHandle = locationService.forOptimize(metastore, session, table); + + DataSize fileSizeThreshold = (DataSize) executeProperties.get("file_size_threshold"); + hiveTableHandle = hiveTableHandle + .withRecordScannedFiles(true) + .withMaxScannedFileSize(Optional.of(fileSizeThreshold.toBytes())); + + if (!constraint.getSummary().isAll()) { + HivePartitionResult partitionResult = partitionManager.getPartitions(metastore, new HiveIdentity(session), hiveTableHandle, constraint); + checkArgument(partitionResult.getUnenforcedConstraint().isAll(), "expected whole predicate to be consumed; remaining: " + partitionResult.getUnenforcedConstraint()); + hiveTableHandle = partitionManager.applyPartitionResult(hiveTableHandle, partitionResult, constraint.getPredicateColumns()); + } + + return Optional.of(new HiveTableExecuteHandle( + OptimizeTableProcedure.NAME, + hiveTableHandle, + Optional.empty(), + tableName.getSchemaName(), + tableName.getTableName(), + columns, + metastore.generatePageSinkMetadata(identity, tableName), + locationHandle, + table.getStorage().getBucketProperty(), + tableStorageFormat, + // TODO: test with multiple partitions using different storage format + tableStorageFormat, + NO_ACID_TRANSACTION)); + } + + private boolean isOnPartitionColumnsOnly(TupleDomain summary, List partitionColumns) + { + if (summary.isAll() || summary.isNone()) { + return true; + } + + Set partitionColumnsSet = ImmutableSet.copyOf(partitionColumns); + for (ColumnHandle columnHandle : summary.getDomains().get().keySet()) { + if (!partitionColumnsSet.contains(columnHandle)) { + return false; + } + } + return true; + } + + @Override + public ConnectorTableExecuteHandle beginTableExecute(ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle) + { + String procedureName = ((HiveTableExecuteHandle) tableExecuteHandle).getProcedureName(); + + if (procedureName.equals(OptimizeTableProcedure.NAME)) { + return beginOptimize(session, tableExecuteHandle); + } + throw new IllegalArgumentException("Unknown procedure '" + procedureName + "'"); + } + + private ConnectorTableExecuteHandle beginOptimize(ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle) + { + HiveTableExecuteHandle hiveExecuteHandle = (HiveTableExecuteHandle) tableExecuteHandle; + WriteInfo writeInfo = locationService.getQueryWriteInfo(hiveExecuteHandle.getLocationHandle()); + String writeDeclarationId = metastore.declareIntentionToWrite(session, writeInfo.getWriteMode(), writeInfo.getWritePath(), hiveExecuteHandle.getSchemaTableName()); + return hiveExecuteHandle.withWriteDeclarationId(writeDeclarationId); + } + + @Override + public void finishTableExecute(ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle, Collection fragments, List tableExecuteState) + { + String procedureName = ((HiveTableExecuteHandle) tableExecuteHandle).getProcedureName(); + + if (procedureName.equals(OptimizeTableProcedure.NAME)) { + finishOptimize(session, tableExecuteHandle, fragments, tableExecuteState); + return; + } + throw new IllegalArgumentException("Unknown procedure '" + procedureName + "'"); + } + + private void finishOptimize(ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle, Collection fragments, List tableExecuteState) + { + // TODO lots of that is copied from finishInsert; rafactoring opportunity + + HiveTableExecuteHandle handle = (HiveTableExecuteHandle) tableExecuteHandle; + checkArgument(handle.getWriteDeclarationId().isPresent(), "no write declaration id present in tableExecuteHandle"); + + List partitionUpdates = fragments.stream() + .map(Slice::getBytes) + .map(partitionUpdateCodec::fromJson) + .collect(toImmutableList()); + + HiveStorageFormat tableStorageFormat = handle.getTableStorageFormat(); + partitionUpdates = PartitionUpdate.mergePartitionUpdates(partitionUpdates); + + Table table = metastore.getTable(new HiveIdentity(session), handle.getSchemaName(), handle.getTableName()) + .orElseThrow(() -> new TableNotFoundException(handle.getSchemaTableName())); + if (!table.getStorage().getStorageFormat().getInputFormat().equals(tableStorageFormat.getInputFormat()) && isRespectTableFormat(session)) { + throw new TrinoException(HIVE_CONCURRENT_MODIFICATION_DETECTED, "Table format changed during optimize"); + } + + // Support for bucketed tables disabled mostly so we do not need to think about grouped execution in intial version. Possibly no change apart from testing required. + verify(handle.getBucketProperty().isEmpty(), "bucketed table not supported"); + + for (PartitionUpdate partitionUpdate : partitionUpdates) { + verify(partitionUpdate.getUpdateMode() == APPEND, "Expected partionUpdate mode to be APPEND but got %s", partitionUpdate.getUpdateMode()); + + if (partitionUpdate.getName().isEmpty()) { + // operating on an unpartitioned table + if (!table.getStorage().getStorageFormat().getInputFormat().equals(handle.getPartitionStorageFormat().getInputFormat()) && isRespectTableFormat(session)) { + throw new TrinoException(HIVE_CONCURRENT_MODIFICATION_DETECTED, "Table format changed during optimize"); + } + + metastore.finishInsertIntoExistingTable( + session, + handle.getSchemaName(), + handle.getTableName(), + partitionUpdate.getWritePath(), + partitionUpdate.getFileNames(), + PartitionStatistics.empty()); + } + else { + // operating on a partition + List partitionValues = toPartitionValues(partitionUpdate.getName()); + metastore.finishInsertIntoExistingPartition( + session, + handle.getSchemaName(), + handle.getTableName(), + partitionValues, + partitionUpdate.getWritePath(), + partitionUpdate.getFileNames(), + PartitionStatistics.empty()); + } + } + + boolean someDeleted = false; + + // track remaining files to be delted for error reporting + Set remainingFilesToDelete = tableExecuteState.stream() + .map(value -> (String) value) + .collect(Collectors.toCollection(HashSet::new)); + + try { + FileSystem fs = hdfsEnvironment.getFileSystem(new HdfsContext(session), new Path(table.getStorage().getLocation())); + for (Object scannedPathObject : tableExecuteState) { + someDeleted = true; + Path scannedPath = new Path((String) scannedPathObject); + retry().run("delete " + scannedPath, () -> fs.delete(scannedPath, false)); + remainingFilesToDelete.remove(scannedPathObject); + } + } + catch (Exception e) { + if (!someDeleted) { + // we are good - we did not delete any source files so we can just throw error and allow rollback to happend + throw new TrinoException(HIVE_FILESYSTEM_ERROR, "Error while deleting original files", e); + } + + // If we already deleted some original files we disable rollback routine so written files are not deleted + // The reported exceptiona and log entry lists files which need to be cleaned up by user manually. + // Until table is cleaned up there will duplicate rows present. + metastore.dropDeclaredIntentionToWrite(handle.getWriteDeclarationId().get()); + String errorMessage = "Error while deleting data files in FINISH phase of OPTIMIZE for table " + table.getTableName() + "; remaining files need to be deleted manually: " + remainingFilesToDelete; + log.error(e, errorMessage); + throw new TrinoException(HIVE_FILESYSTEM_ERROR, errorMessage, e); + } + } + @Override public void createView(ConnectorSession session, SchemaTableName viewName, ConnectorViewDefinition definition, boolean replace) { @@ -2100,7 +2332,7 @@ public void finishDelete(ConnectorSession session, ConnectorTableHandle tableHan List partitionAndStatementIds = fragments.stream() .map(Slice::getBytes) .map(PartitionAndStatementId.CODEC::fromJson) - .collect(toList()); + .collect(toImmutableList()); HdfsContext context = new HdfsContext(session); for (PartitionAndStatementId ps : partitionAndStatementIds) { @@ -2498,7 +2730,9 @@ public ConnectorTableHandle makeCompatiblePartitioning(ConnectorSession session, hiveTable.getAnalyzeColumnNames(), Optional.empty(), Optional.empty(), // Projected columns is used only during optimization phase of planning - hiveTable.getTransaction()); + hiveTable.getTransaction(), + hiveTable.isRecordScannedFiles(), + hiveTable.getMaxScannedFileSize()); } @VisibleForTesting @@ -2611,7 +2845,7 @@ else if (isFullAcidTable(table.getParameters())) { hiveBucketHandle.get().getTableBucketCount(), hiveBucketHandle.get().getColumns().stream() .map(HiveColumnHandle::getHiveType) - .collect(toList()), + .collect(toImmutableList()), OptionalInt.of(hiveBucketHandle.get().getTableBucketCount()), !partitionColumns.isEmpty() && isParallelPartitionedBucketedWrites(session)); return Optional.of(new ConnectorNewTableLayout(partitioningHandle, partitioningColumns.build())); @@ -2852,7 +3086,7 @@ private static void validatePartitionColumns(ConnectorTableMetadata tableMetadat List allColumns = tableMetadata.getColumns().stream() .map(ColumnMetadata::getName) - .collect(toList()); + .collect(toImmutableList()); if (!allColumns.containsAll(partitionedBy)) { throw new TrinoException(INVALID_TABLE_PROPERTY, format("Partition columns %s not present in schema", Sets.difference(ImmutableSet.copyOf(partitionedBy), ImmutableSet.copyOf(allColumns)))); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java index 2588be4b8898..d632c2a13c3a 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java @@ -32,6 +32,7 @@ import io.trino.spi.connector.ConnectorPageSink; import io.trino.spi.connector.ConnectorPageSinkProvider; import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTableExecuteHandle; import io.trino.spi.connector.ConnectorTransactionHandle; import io.trino.spi.type.TypeManager; import org.joda.time.DateTimeZone; @@ -121,6 +122,13 @@ public ConnectorPageSink createPageSink(ConnectorTransactionHandle transaction, return createPageSink(handle, false, session, ImmutableMap.of() /* for insert properties are taken from metastore */); } + @Override + public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle) + { + HiveTableExecuteHandle handle = (HiveTableExecuteHandle) tableExecuteHandle; + return createPageSink(handle, false, session, ImmutableMap.of()); + } + private ConnectorPageSink createPageSink(HiveWritableTableHandle handle, boolean isCreateTable, ConnectorSession session, Map additionalTableParameters) { OptionalInt bucketCount = OptionalInt.empty(); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionManager.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionManager.java index 7f8fc751cb0a..beb34811f4ce 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionManager.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionManager.java @@ -193,7 +193,9 @@ public HiveTableHandle applyPartitionResult(HiveTableHandle handle, HivePartitio handle.getAnalyzeColumnNames(), Optionals.combine(handle.getConstraintColumns(), columns, Sets::union), handle.getProjectedColumns(), - handle.getTransaction()); + handle.getTransaction(), + handle.isRecordScannedFiles(), + handle.getMaxScannedFileSize()); } public List getOrLoadPartitions(SemiTransactionalHiveMetastore metastore, HiveIdentity identity, HiveTableHandle table) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java index b26a670db96f..9b863a13976e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java @@ -207,6 +207,9 @@ public ConnectorSplitSource getSplits( // short circuit if we don't have any partitions if (partitions.isEmpty()) { + if (hiveTable.isRecordScannedFiles()) { + return new FixedSplitSource(ImmutableList.of(), ImmutableList.of()); + } return new FixedSplitSource(ImmutableList.of()); } @@ -245,7 +248,8 @@ public ConnectorSplitSource getSplits( !hiveTable.getPartitionColumns().isEmpty() && isIgnoreAbsentPartitions(session), isOptimizeSymlinkListing(session), metastore.getValidWriteIds(session, hiveTable) - .map(validTxnWriteIdList -> validTxnWriteIdList.getTableValidWriteIdList(table.getDatabaseName() + "." + table.getTableName()))); + .map(validTxnWriteIdList -> validTxnWriteIdList.getTableValidWriteIdList(table.getDatabaseName() + "." + table.getTableName())), + hiveTable.getMaxScannedFileSize()); HiveSplitSource splitSource; switch (splitSchedulingStrategy) { @@ -260,7 +264,8 @@ public ConnectorSplitSource getSplits( maxSplitsPerSecond, hiveSplitLoader, executor, - highMemorySplitSourceCounter); + highMemorySplitSourceCounter, + hiveTable.isRecordScannedFiles()); break; case GROUPED_SCHEDULING: splitSource = HiveSplitSource.bucketed( @@ -273,7 +278,8 @@ public ConnectorSplitSource getSplits( maxSplitsPerSecond, hiveSplitLoader, executor, - highMemorySplitSourceCounter); + highMemorySplitSourceCounter, + hiveTable.isRecordScannedFiles()); break; default: throw new IllegalArgumentException("Unknown splitSchedulingStrategy: " + splitSchedulingStrategy); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java index 68c5758adcc8..735584413dd1 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java @@ -33,6 +33,7 @@ import java.io.FileNotFoundException; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.OptionalInt; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -89,6 +90,9 @@ class HiveSplitSource private final CounterStat highMemorySplitSourceCounter; private final AtomicBoolean loggedHighMemoryWarning = new AtomicBoolean(); + private final boolean recordScannedFiles; + private final ImmutableList.Builder scannedFilePaths = ImmutableList.builder(); + private HiveSplitSource( ConnectorSession session, String databaseName, @@ -98,7 +102,8 @@ private HiveSplitSource( DataSize maxOutstandingSplitsSize, HiveSplitLoader splitLoader, AtomicReference stateReference, - CounterStat highMemorySplitSourceCounter) + CounterStat highMemorySplitSourceCounter, + boolean recordScannedFiles) { requireNonNull(session, "session is null"); this.queryId = session.getQueryId(); @@ -114,6 +119,7 @@ private HiveSplitSource( this.maxInitialSplitSize = getMaxInitialSplitSize(session); this.remainingInitialSplits = new AtomicInteger(maxInitialSplits); this.numberOfProcessedSplits = new AtomicLong(0); + this.recordScannedFiles = recordScannedFiles; } public static HiveSplitSource allAtOnce( @@ -126,7 +132,8 @@ public static HiveSplitSource allAtOnce( int maxSplitsPerSecond, HiveSplitLoader splitLoader, Executor executor, - CounterStat highMemorySplitSourceCounter) + CounterStat highMemorySplitSourceCounter, + boolean recordScannedFiles) { AtomicReference stateReference = new AtomicReference<>(State.initial()); return new HiveSplitSource( @@ -168,7 +175,8 @@ public boolean isFinished(OptionalInt bucketNumber) maxOutstandingSplitsSize, splitLoader, stateReference, - highMemorySplitSourceCounter); + highMemorySplitSourceCounter, + recordScannedFiles); } public static HiveSplitSource bucketed( @@ -181,7 +189,8 @@ public static HiveSplitSource bucketed( int maxSplitsPerSecond, HiveSplitLoader splitLoader, Executor executor, - CounterStat highMemorySplitSourceCounter) + CounterStat highMemorySplitSourceCounter, + boolean recordScannedFiles) { AtomicReference stateReference = new AtomicReference<>(State.initial()); return new HiveSplitSource( @@ -243,7 +252,8 @@ public AsyncQueue queueFor(OptionalInt bucketNumber) maxOutstandingSplitsSize, splitLoader, stateReference, - highMemorySplitSourceCounter); + highMemorySplitSourceCounter, + recordScannedFiles); } /** @@ -406,6 +416,9 @@ else if (maxSplitBytes * 2 >= remainingBlockBytes) { ListenableFuture transform = Futures.transform(future, splits -> { requireNonNull(splits, "splits is null"); + if (recordScannedFiles) { + splits.forEach(split -> scannedFilePaths.add(((HiveSplit) split).getPath())); + } if (noMoreSplits) { // Checking splits.isEmpty() here is required for thread safety. // Let's say there are 10 splits left, and max number of splits per batch is 5. @@ -446,6 +459,16 @@ public boolean isFinished() } } + @Override + public Optional> getTableExecuteSplitsInfo() + { + checkState(isFinished(), "HiveSplitSource must be finished before TableExecuteSplitsInfo is read"); + if (!recordScannedFiles) { + return Optional.empty(); + } + return Optional.of(scannedFilePaths.build()); + } + @Override public void close() { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveTableExecuteHandle.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveTableExecuteHandle.java new file mode 100644 index 000000000000..dac720bf999a --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveTableExecuteHandle.java @@ -0,0 +1,108 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hive; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.plugin.hive.acid.AcidTransaction; +import io.trino.plugin.hive.metastore.HivePageSinkMetadata; +import io.trino.spi.connector.ConnectorTableExecuteHandle; +import io.trino.spi.connector.ConnectorTableHandle; + +import java.util.List; +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public class HiveTableExecuteHandle + extends HiveWritableTableHandle + implements ConnectorTableExecuteHandle +{ + private final String procedureName; + private final HiveTableHandle sourceTableHandle; + private final Optional writeDeclarationId; + + @JsonCreator + public HiveTableExecuteHandle( + @JsonProperty("procedureName") String procedureName, + @JsonProperty("sourceTableHandle") HiveTableHandle sourceTableHandle, + @JsonProperty("writeDeclarationId") Optional writeDeclarationId, + @JsonProperty("schemaName") String schemaName, + @JsonProperty("tableName") String tableName, + @JsonProperty("inputColumns") List inputColumns, + @JsonProperty("pageSinkMetadata") HivePageSinkMetadata pageSinkMetadata, + @JsonProperty("locationHandle") LocationHandle locationHandle, + @JsonProperty("bucketProperty") Optional bucketProperty, + @JsonProperty("tableStorageFormat") HiveStorageFormat tableStorageFormat, + @JsonProperty("partitionStorageFormat") HiveStorageFormat partitionStorageFormat, + @JsonProperty("transaction") AcidTransaction transaction) + { + super( + schemaName, + tableName, + inputColumns, + pageSinkMetadata, + locationHandle, + bucketProperty, + tableStorageFormat, + partitionStorageFormat, + transaction); + + this.procedureName = requireNonNull(procedureName, "procedureName is null"); + this.sourceTableHandle = requireNonNull(sourceTableHandle, "sourceTableHandle is null"); + this.writeDeclarationId = requireNonNull(writeDeclarationId, "writeDeclarationId is null"); + } + + @JsonProperty + public String getProcedureName() + { + return procedureName; + } + + @Override + @JsonProperty + public ConnectorTableHandle getSourceTableHandle() + { + return sourceTableHandle; + } + + @JsonProperty + public Optional getWriteDeclarationId() + { + return writeDeclarationId; + } + + public HiveTableExecuteHandle withWriteDeclarationId(String writeDeclarationId) + { + return new HiveTableExecuteHandle( + procedureName, + sourceTableHandle, + Optional.of(writeDeclarationId), + getSchemaName(), + getTableName(), + getInputColumns(), + getPageSinkMetadata(), + getLocationHandle(), + getBucketProperty(), + getTableStorageFormat(), + getPartitionStorageFormat(), + getTransaction()); + } + + @Override + public String toString() + { + return procedureName + "(" + getSchemaName() + "." + getTableName() + ")"; + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveTableHandle.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveTableHandle.java index a5185c57c726..84ef23315525 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveTableHandle.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveTableHandle.java @@ -56,6 +56,8 @@ public class HiveTableHandle private final Optional> constraintColumns; private final Optional> projectedColumns; private final AcidTransaction transaction; + private final boolean recordScannedFiles; + private final Optional maxScannedFileSize; @JsonCreator public HiveTableHandle( @@ -86,7 +88,9 @@ public HiveTableHandle( analyzeColumnNames, Optional.empty(), Optional.empty(), - transaction); + transaction, + false, + Optional.empty()); } public HiveTableHandle( @@ -112,7 +116,9 @@ public HiveTableHandle( Optional.empty(), Optional.empty(), Optional.empty(), - NO_ACID_TRANSACTION); + NO_ACID_TRANSACTION, + false, + Optional.empty()); } public HiveTableHandle( @@ -130,7 +136,9 @@ public HiveTableHandle( Optional> analyzeColumnNames, Optional> constraintColumns, Optional> projectedColumns, - AcidTransaction transaction) + AcidTransaction transaction, + boolean recordScannedFiles, + Optional maxSplitFileSize) { this.schemaName = requireNonNull(schemaName, "schemaName is null"); this.tableName = requireNonNull(tableName, "tableName is null"); @@ -147,6 +155,8 @@ public HiveTableHandle( this.constraintColumns = requireNonNull(constraintColumns, "constraintColumns is null"); this.projectedColumns = requireNonNull(projectedColumns, "projectedColumns is null"); this.transaction = requireNonNull(transaction, "transaction is null"); + this.recordScannedFiles = recordScannedFiles; + this.maxScannedFileSize = requireNonNull(maxSplitFileSize, "maxSplitFileSize is null"); } public HiveTableHandle withAnalyzePartitionValues(List> analyzePartitionValues) @@ -166,7 +176,9 @@ public HiveTableHandle withAnalyzePartitionValues(List> analyzePart analyzeColumnNames, constraintColumns, projectedColumns, - transaction); + transaction, + recordScannedFiles, + maxScannedFileSize); } public HiveTableHandle withAnalyzeColumnNames(Set analyzeColumnNames) @@ -186,7 +198,9 @@ public HiveTableHandle withAnalyzeColumnNames(Set analyzeColumnNames) Optional.of(analyzeColumnNames), constraintColumns, projectedColumns, - transaction); + transaction, + recordScannedFiles, + maxScannedFileSize); } public HiveTableHandle withTransaction(AcidTransaction transaction) @@ -206,7 +220,9 @@ public HiveTableHandle withTransaction(AcidTransaction transaction) analyzeColumnNames, constraintColumns, projectedColumns, - transaction); + transaction, + recordScannedFiles, + maxScannedFileSize); } public HiveTableHandle withUpdateProcessor(AcidTransaction transaction, HiveUpdateProcessor updateProcessor) @@ -227,7 +243,9 @@ public HiveTableHandle withUpdateProcessor(AcidTransaction transaction, HiveUpda analyzeColumnNames, constraintColumns, projectedColumns, - transaction); + transaction, + recordScannedFiles, + maxScannedFileSize); } public HiveTableHandle withProjectedColumns(Set projectedColumns) @@ -247,7 +265,53 @@ public HiveTableHandle withProjectedColumns(Set projectedColumns) analyzeColumnNames, constraintColumns, Optional.of(projectedColumns), - transaction); + transaction, + recordScannedFiles, + maxScannedFileSize); + } + + public HiveTableHandle withRecordScannedFiles(boolean recordScannedFiles) + { + return new HiveTableHandle( + schemaName, + tableName, + tableParameters, + partitionColumns, + dataColumns, + partitions, + compactEffectivePredicate, + enforcedConstraint, + bucketHandle, + bucketFilter, + analyzePartitionValues, + analyzeColumnNames, + constraintColumns, + projectedColumns, + transaction, + recordScannedFiles, + maxScannedFileSize); + } + + public HiveTableHandle withMaxScannedFileSize(Optional maxScannedFileSize) + { + return new HiveTableHandle( + schemaName, + tableName, + tableParameters, + partitionColumns, + dataColumns, + partitions, + compactEffectivePredicate, + enforcedConstraint, + bucketHandle, + bucketFilter, + analyzePartitionValues, + analyzeColumnNames, + constraintColumns, + projectedColumns, + transaction, + recordScannedFiles, + maxScannedFileSize); } @JsonProperty @@ -387,6 +451,18 @@ public long getWriteId() return transaction.getWriteId(); } + @JsonIgnore + public boolean isRecordScannedFiles() + { + return recordScannedFiles; + } + + @JsonIgnore + public Optional getMaxScannedFileSize() + { + return maxScannedFileSize; + } + @Override public boolean equals(Object o) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java index 0bfd5a8e880c..39efaeb050a4 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java @@ -58,6 +58,7 @@ import io.trino.spi.connector.ConnectorPageSourceProvider; import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.SystemTable; +import io.trino.spi.connector.TableProcedureMetadata; import io.trino.spi.eventlistener.EventListener; import io.trino.spi.procedure.Procedure; import org.weakref.jmx.guice.MBeanModule; @@ -133,6 +134,7 @@ public static Connector createConnector(String catalogName, Map HiveMaterializedViewPropertiesProvider hiveMaterializedViewPropertiesProvider = injector.getInstance(HiveMaterializedViewPropertiesProvider.class); ConnectorAccessControl accessControl = new ClassLoaderSafeConnectorAccessControl(injector.getInstance(SystemTableAwareAccessControl.class), classLoader); Set procedures = injector.getInstance(Key.get(new TypeLiteral>() {})); + Set tableProcedures = injector.getInstance(Key.get(new TypeLiteral>() {})); Set systemTables = injector.getInstance(Key.get(new TypeLiteral>() {})); Set eventListeners = injector.getInstance(Key.get(new TypeLiteral>() {})) .stream() @@ -149,6 +151,7 @@ public static Connector createConnector(String catalogName, Map new ClassLoaderSafeNodePartitioningProvider(connectorDistributionProvider, classLoader), systemTables, procedures, + tableProcedures, eventListeners, sessionPropertiesProviders, HiveSchemaProperties.SCHEMA_PROPERTIES, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/LocationService.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/LocationService.java index 74e4e6a7f2f8..2dab8e14dfed 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/LocationService.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/LocationService.java @@ -29,6 +29,8 @@ public interface LocationService LocationHandle forExistingTable(SemiTransactionalHiveMetastore metastore, ConnectorSession session, Table table); + LocationHandle forOptimize(SemiTransactionalHiveMetastore metastore, ConnectorSession session, Table table); + /** * targetPath and writePath will be root directory of all partition and table paths * that may be returned by {@link #getTableWriteInfo(LocationHandle, boolean)} and {@link #getPartitionWriteInfo(LocationHandle, Optional, String)} method. diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java index c57ea719b7ff..3fd17b67ba4f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java @@ -142,6 +142,8 @@ public class SemiTransactionalHiveMetastore @GuardedBy("this") private final Map, Action>> partitionActions = new HashMap<>(); @GuardedBy("this") + private long declaredIntentionsToWriteCounter; + @GuardedBy("this") private final List declaredIntentionsToWrite = new ArrayList<>(); @GuardedBy("this") private ExclusiveOperation bufferedExclusiveOperation; @@ -1138,7 +1140,7 @@ public synchronized void revokeTablePrivileges(HiveIdentity identity, String dat setExclusive((delegate, hdfsEnvironment) -> delegate.revokeTablePrivileges(databaseName, tableName, getTableOwner(identity, databaseName, tableName), grantee, privileges)); } - public synchronized void declareIntentionToWrite(ConnectorSession session, WriteMode writeMode, Path stagingPathRoot, SchemaTableName schemaTableName) + public synchronized String declareIntentionToWrite(ConnectorSession session, WriteMode writeMode, Path stagingPathRoot, SchemaTableName schemaTableName) { setShared(); if (writeMode == WriteMode.DIRECT_TO_TARGET_EXISTING_DIRECTORY) { @@ -1149,7 +1151,19 @@ public synchronized void declareIntentionToWrite(ConnectorSession session, Write } HdfsContext hdfsContext = new HdfsContext(session); HiveIdentity identity = new HiveIdentity(session); - declaredIntentionsToWrite.add(new DeclaredIntentionToWrite(writeMode, hdfsContext, identity, session.getQueryId(), stagingPathRoot, schemaTableName)); + String queryId = session.getQueryId(); + String declarationId = queryId + "_" + declaredIntentionsToWriteCounter; + declaredIntentionsToWriteCounter++; + declaredIntentionsToWrite.add(new DeclaredIntentionToWrite(declarationId, writeMode, hdfsContext, identity, queryId, stagingPathRoot, schemaTableName)); + return declarationId; + } + + public synchronized void dropDeclaredIntentionToWrite(String declarationId) + { + boolean removed = declaredIntentionsToWrite.removeIf(intention -> intention.getDeclarationId().equals(declarationId)); + if (!removed) { + throw new IllegalArgumentException("Declaration with id " + declarationId + " not found"); + } } public boolean isFinished() @@ -2865,6 +2879,7 @@ public String toString() private static class DeclaredIntentionToWrite { + private final String declarationId; private final WriteMode mode; private final HdfsContext hdfsContext; private final HiveIdentity identity; @@ -2872,8 +2887,9 @@ private static class DeclaredIntentionToWrite private final Path rootPath; private final SchemaTableName schemaTableName; - public DeclaredIntentionToWrite(WriteMode mode, HdfsContext hdfsContext, HiveIdentity identity, String queryId, Path stagingPathRoot, SchemaTableName schemaTableName) + public DeclaredIntentionToWrite(String declarationId, WriteMode mode, HdfsContext hdfsContext, HiveIdentity identity, String queryId, Path stagingPathRoot, SchemaTableName schemaTableName) { + this.declarationId = requireNonNull(declarationId, "declarationId is null"); this.mode = requireNonNull(mode, "mode is null"); this.hdfsContext = requireNonNull(hdfsContext, "hdfsContext is null"); this.identity = requireNonNull(identity, "identity is null"); @@ -2882,6 +2898,11 @@ public DeclaredIntentionToWrite(WriteMode mode, HdfsContext hdfsContext, HiveIde this.schemaTableName = requireNonNull(schemaTableName, "schemaTableName is null"); } + public String getDeclarationId() + { + return declarationId; + } + public WriteMode getMode() { return mode; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/HiveProcedureModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/HiveProcedureModule.java index 7e1aa2090eb9..465d4d70ebe0 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/HiveProcedureModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/HiveProcedureModule.java @@ -17,6 +17,7 @@ import com.google.inject.Module; import com.google.inject.Scopes; import com.google.inject.multibindings.Multibinder; +import io.trino.spi.connector.TableProcedureMetadata; import io.trino.spi.procedure.Procedure; import static com.google.inject.multibindings.Multibinder.newSetBinder; @@ -33,5 +34,8 @@ public void configure(Binder binder) procedures.addBinding().toProvider(UnregisterPartitionProcedure.class).in(Scopes.SINGLETON); procedures.addBinding().toProvider(SyncPartitionMetadataProcedure.class).in(Scopes.SINGLETON); procedures.addBinding().toProvider(DropStatsProcedure.class).in(Scopes.SINGLETON); + + Multibinder tableProcedures = newSetBinder(binder, TableProcedureMetadata.class); + tableProcedures.addBinding().toProvider(OptimizeTableProcedure.class).in(Scopes.SINGLETON); } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/OptimizeTableProcedure.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/OptimizeTableProcedure.java new file mode 100644 index 000000000000..7b31a076a46f --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/OptimizeTableProcedure.java @@ -0,0 +1,42 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hive.procedure; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Provider; +import io.airlift.units.DataSize; +import io.trino.spi.connector.TableProcedureMetadata; + +import static io.trino.plugin.base.session.PropertyMetadataUtil.dataSizeProperty; +import static io.trino.spi.connector.TableProcedureExecutionMode.distributedWithFilteringAndRepartitioning; + +public class OptimizeTableProcedure + implements Provider +{ + public static final String NAME = "OPTIMIZE"; + + @Override + public TableProcedureMetadata get() + { + return new TableProcedureMetadata( + NAME, + distributedWithFilteringAndRepartitioning(), + ImmutableList.of( + dataSizeProperty( + "file_size_threshold", + "Only compact files smaller than given threshold in bytes", + DataSize.of(100, DataSize.Unit.MEGABYTE), + false))); + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/security/LegacyAccessControl.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/security/LegacyAccessControl.java index 54f4a23ff710..f97402d26133 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/security/LegacyAccessControl.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/security/LegacyAccessControl.java @@ -355,6 +355,11 @@ public void checkCanExecuteProcedure(ConnectorSecurityContext context, SchemaRou { } + @Override + public void checkCanExecuteTableProcedure(ConnectorSecurityContext context, SchemaTableName tableName, String procedure) + { + } + @Override public Optional getRowFilter(ConnectorSecurityContext context, SchemaTableName tableName) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/security/SqlStandardAccessControl.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/security/SqlStandardAccessControl.java index e62cdebd1324..7386f82a7e42 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/security/SqlStandardAccessControl.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/security/SqlStandardAccessControl.java @@ -68,6 +68,7 @@ import static io.trino.spi.security.AccessDeniedException.denyDropSchema; import static io.trino.spi.security.AccessDeniedException.denyDropTable; import static io.trino.spi.security.AccessDeniedException.denyDropView; +import static io.trino.spi.security.AccessDeniedException.denyExecuteTableProcedure; import static io.trino.spi.security.AccessDeniedException.denyGrantRoles; import static io.trino.spi.security.AccessDeniedException.denyGrantTablePrivilege; import static io.trino.spi.security.AccessDeniedException.denyInsertTable; @@ -519,6 +520,14 @@ public void checkCanExecuteProcedure(ConnectorSecurityContext context, SchemaRou { } + @Override + public void checkCanExecuteTableProcedure(ConnectorSecurityContext context, SchemaTableName tableName, String procedure) + { + if (!isTableOwner(context, tableName)) { + denyExecuteTableProcedure(tableName.toString(), tableName.toString()); + } + } + @Override public Optional getRowFilter(ConnectorSecurityContext context, SchemaTableName tableName) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/InternalHiveSplitFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/InternalHiveSplitFactory.java index d3e69503609a..2c006f54fd38 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/InternalHiveSplitFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/InternalHiveSplitFactory.java @@ -69,6 +69,7 @@ public class InternalHiveSplitFactory private final Optional bucketConversion; private final Optional bucketValidation; private final long minimumTargetSplitSizeInBytes; + private final Optional maxSplitFileSize; private final boolean forceLocalScheduling; private final boolean s3SelectPushdownEnabled; private final Map bucketStatementCounters = new ConcurrentHashMap<>(); @@ -87,7 +88,8 @@ public InternalHiveSplitFactory( DataSize minimumTargetSplitSize, boolean forceLocalScheduling, boolean s3SelectPushdownEnabled, - AcidTransaction transaction) + AcidTransaction transaction, + Optional maxSplitFileSize) { this.fileSystem = requireNonNull(fileSystem, "fileSystem is null"); this.partitionName = requireNonNull(partitionName, "partitionName is null"); @@ -102,6 +104,7 @@ public InternalHiveSplitFactory( this.forceLocalScheduling = forceLocalScheduling; this.s3SelectPushdownEnabled = s3SelectPushdownEnabled; this.minimumTargetSplitSizeInBytes = requireNonNull(minimumTargetSplitSize, "minimumTargetSplitSize is null").toBytes(); + this.maxSplitFileSize = requireNonNull(maxSplitFileSize, "maxSplitFileSize is null"); checkArgument(minimumTargetSplitSizeInBytes > 0, "minimumTargetSplitSize must be > 0, found: %s", minimumTargetSplitSize); } @@ -166,6 +169,10 @@ private Optional createInternalHiveSplit( return Optional.empty(); } + if (maxSplitFileSize.isPresent() && estimatedFileSize > maxSplitFileSize.get()) { + return Optional.empty(); + } + boolean forceLocalScheduling = this.forceLocalScheduling; // For empty files, some filesystem (e.g. LocalFileSystem) produce one empty block diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java index ffcbc34e4ec3..1c1d41c460ae 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java @@ -546,6 +546,7 @@ public HivePartitionMetadata next() false, false, true, + Optional.empty(), Optional.empty()); HiveSplitSource hiveSplitSource = hiveSplitSource(backgroundHiveSplitLoader); @@ -1078,7 +1079,8 @@ private BackgroundHiveSplitLoader backgroundHiveSplitLoader( false, false, true, - validWriteIds); + validWriteIds, + Optional.empty()); } private BackgroundHiveSplitLoader backgroundHiveSplitLoader(List files, DirectoryLister directoryLister) @@ -1110,6 +1112,7 @@ private BackgroundHiveSplitLoader backgroundHiveSplitLoader(List initialFiles = getTableFiles(tableName); + assertThat(initialFiles).hasSize(10); + + assertUpdate("ALTER TABLE " + tableName + " EXECUTE optimize WITH (file_size_threshold = '10kB')"); + assertNationNTimes(tableName, 10); + + Set compactedFiles = getTableFiles(tableName); + // we expect at most 3 files due to write parallelism + assertThat(compactedFiles).hasSizeLessThanOrEqualTo(3); + assertThat(intersection(initialFiles, compactedFiles)).isEmpty(); + + // compact with low threshold; nothing should change + assertUpdate("ALTER TABLE " + tableName + " EXECUTE optimize WITH (file_size_threshold = '10B')"); + + assertThat(getTableFiles(tableName)).hasSameElementsAs(compactedFiles); + } + + @Test + public void testOptimizeWithWriterScaling() + { + String tableName = "test_optimize_witer_scaling" + randomTableSuffix(); + assertUpdate("CREATE TABLE " + tableName + " AS SELECT * FROM tpch.sf1.nation WITH NO DATA", 0); + + insertNationNTimes(tableName, 4); + assertNationNTimes(tableName, 4); + + Set initialFiles = getTableFiles(tableName); + assertThat(initialFiles).hasSize(4); + + Session writerScalingSession = Session.builder(getSession()) + .setSystemProperty("scale_writers", "true") + .setSystemProperty("writer_min_size", "100GB") + .build(); + + assertUpdate(writerScalingSession, "ALTER TABLE " + tableName + " EXECUTE optimize WITH (file_size_threshold = '10kB')"); + assertNationNTimes(tableName, 4); + + Set compactedFiles = getTableFiles(tableName); + assertThat(compactedFiles).hasSize(1); + assertThat(intersection(initialFiles, compactedFiles)).isEmpty(); + } + + @Test + public void testOptimizeWithPartitioning() + { + int insertCount = 4; + int partitionsCount = 5; + + String tableName = "test_optimize_with_partitioning_" + randomTableSuffix(); + assertUpdate("CREATE TABLE " + tableName + "(" + + " nationkey BIGINT, " + + " name VARCHAR, " + + " comment VARCHAR, " + + " regionkey BIGINT" + + ")" + + "WITH (partitioned_by = ARRAY['regionkey'])"); + + insertNationNTimes(tableName, insertCount); + assertNationNTimes(tableName, insertCount); + + Set initialFiles = getTableFiles(tableName); + assertThat(initialFiles).hasSize(insertCount * partitionsCount); + + Session writerScalingSession = Session.builder(getSession()) + .setSystemProperty("scale_writers", "true") + .setSystemProperty("writer_min_size", "100GB") + .build(); + + // optimize with unsupported WHERE + assertThatThrownBy(() -> computeActual("ALTER TABLE " + tableName + " EXECUTE optimize WITH (file_size_threshold = '10kB') WHERE nationkey = 1")) + .hasMessageContaining("Predicate used for optimize must match whole partitions"); + assertNationNTimes(tableName, insertCount); + assertThat(getTableFiles(tableName)).hasSameElementsAs(initialFiles); + + // optimize using predicate on on partition key but not matching any partitions + assertUpdate(writerScalingSession, "ALTER TABLE " + tableName + " EXECUTE optimize WITH (file_size_threshold = '10kB') WHERE regionkey > 5"); + assertNationNTimes(tableName, insertCount); + assertThat(getTableFiles(tableName)).hasSameElementsAs(initialFiles); + + // optimize two partitions + assertUpdate(writerScalingSession, "ALTER TABLE " + tableName + " EXECUTE optimize WITH (file_size_threshold = '10kB') WHERE regionkey IN (1,2)"); + assertNationNTimes(tableName, insertCount); + assertThat(getTableFiles(tableName)).hasSize(2 + 3 * insertCount); + + // optimize one more partition + assertUpdate(writerScalingSession, "ALTER TABLE " + tableName + " EXECUTE optimize WITH (file_size_threshold = '10kB') WHERE regionkey > 3"); + assertNationNTimes(tableName, insertCount); + assertThat(getTableFiles(tableName)).hasSize(3 + 2 * insertCount); + + // optimize remaining partitions + assertUpdate(writerScalingSession, "ALTER TABLE " + tableName + " EXECUTE optimize WITH (file_size_threshold = '10kB')"); + assertNationNTimes(tableName, insertCount); + + Set compactedFiles = getTableFiles(tableName); + assertThat(compactedFiles).hasSize(partitionsCount); + assertThat(intersection(initialFiles, compactedFiles)).isEmpty(); + } + + private void insertNationNTimes(String tableName, int times) + { + for (int i = 0; i < times; i++) { + assertUpdate("INSERT INTO " + tableName + "(nationkey, name, regionkey, comment) SELECT * FROM tpch.sf1.nation", 25); + } + } + + private void assertNationNTimes(String tableName, int times) + { + String verifyQuery = join(" UNION ALL ", nCopies(times, "SELECT * FROM nation")); + assertQuery("SELECT nationkey, name, regionkey, comment FROM " + tableName, verifyQuery); + } + + private Set getTableFiles(String tableName) + { + return computeActual("SELECT DISTINCT \"$path\" FROM " + tableName).getMaterializedRows().stream() + .map(row -> (String) row.getField(0)) + .collect(toImmutableSet()); + } + @Test public void testTimestampPrecisionInsert() { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java index 7b2b615abcfd..fddbe34a73d6 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java @@ -57,7 +57,8 @@ public void testOutstandingSplitCount() Integer.MAX_VALUE, new TestingHiveSplitLoader(), Executors.newFixedThreadPool(5), - new CounterStat()); + new CounterStat(), + false); // add 10 splits for (int i = 0; i < 10; i++) { @@ -91,7 +92,8 @@ public void testCorrectlyGeneratingInitialRowId() Integer.MAX_VALUE, new TestingHiveSplitLoader(), Executors.newFixedThreadPool(5), - new CounterStat()); + new CounterStat(), + false); // add 10 splits for (int i = 0; i < 10; i++) { @@ -119,7 +121,8 @@ public void testEvenlySizedSplitRemainder() Integer.MAX_VALUE, new TestingHiveSplitLoader(), Executors.newSingleThreadExecutor(), - new CounterStat()); + new CounterStat(), + false); // One byte larger than the initial split max size DataSize fileSize = DataSize.ofBytes(initialSplitSize.toBytes() + 1); @@ -146,7 +149,8 @@ public void testFail() Integer.MAX_VALUE, new TestingHiveSplitLoader(), Executors.newFixedThreadPool(5), - new CounterStat()); + new CounterStat(), + false); // add some splits for (int i = 0; i < 5; i++) { @@ -196,7 +200,8 @@ public void testReaderWaitsForSplits() Integer.MAX_VALUE, new TestingHiveSplitLoader(), Executors.newFixedThreadPool(5), - new CounterStat()); + new CounterStat(), + false); SettableFuture splits = SettableFuture.create(); @@ -250,7 +255,8 @@ public void testOutstandingSplitSize() Integer.MAX_VALUE, new TestingHiveSplitLoader(), Executors.newFixedThreadPool(5), - new CounterStat()); + new CounterStat(), + false); int testSplitSizeInBytes = new TestSplit(0).getEstimatedSizeInBytes(); int maxSplitCount = toIntExact(maxOutstandingSplitsSize.toBytes()) / testSplitSizeInBytes; @@ -283,7 +289,8 @@ public void testEmptyBucket() Integer.MAX_VALUE, new TestingHiveSplitLoader(), Executors.newFixedThreadPool(5), - new CounterStat()); + new CounterStat(), + false); hiveSplitSource.addToQueue(new TestSplit(0, OptionalInt.of(2))); hiveSplitSource.noMoreSplits(); assertEquals(getSplits(hiveSplitSource, OptionalInt.of(0), 10).size(), 0);