From 330fa47d6f31b9161ebe69b163835458fe772933 Mon Sep 17 00:00:00 2001 From: Kevin Wilfong Date: Mon, 29 Jul 2024 12:31:26 -0700 Subject: [PATCH 01/31] array_min/array_max throw if first element is an inner Array with a null element --- .../operator/scalar/ArrayMaxFunction.java | 6 ++- .../operator/scalar/ArrayMinFunction.java | 6 ++- .../presto/type/TestArrayOperators.java | 48 +++++++++++++++++++ 3 files changed, 58 insertions(+), 2 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayMaxFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayMaxFunction.java index 94debbaa2ac06..85fe8e3f02685 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayMaxFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayMaxFunction.java @@ -95,8 +95,12 @@ public static Block blockArrayMax( return null; } + if (block.isNull(0)) { + return null; + } + Block selectedValue = (Block) elementType.getObject(block, 0); - for (int i = 0; i < block.getPositionCount(); i++) { + for (int i = 1; i < block.getPositionCount(); i++) { if (block.isNull(i)) { return null; } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayMinFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayMinFunction.java index b605a0bfb731d..30a7ddec1ba00 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayMinFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayMinFunction.java @@ -95,8 +95,12 @@ public static Block blockArrayMin( return null; } + if (block.isNull(0)) { + return null; + } + Block selectedValue = (Block) elementType.getObject(block, 0); - for (int i = 0; i < block.getPositionCount(); i++) { + for (int i = 1; i < block.getPositionCount(); i++) { if (block.isNull(i)) { return null; } diff --git a/presto-main/src/test/java/com/facebook/presto/type/TestArrayOperators.java b/presto-main/src/test/java/com/facebook/presto/type/TestArrayOperators.java index 360bc3521fa1f..ebd6946d51e93 100644 --- a/presto-main/src/test/java/com/facebook/presto/type/TestArrayOperators.java +++ b/presto-main/src/test/java/com/facebook/presto/type/TestArrayOperators.java @@ -695,6 +695,30 @@ public void testArrayMin() assertDecimalFunction("ARRAY_MIN(ARRAY [2.22222222222222222, 2.3])", decimal("2.22222222222222222")); } + @Test + public void testArrayMinWithNullsInBothArraysNotComparedFirstIsMin() + { + assertFunction("ARRAY_MIN(ARRAY [ARRAY[1, NULL], ARRAY[2, NULL]])", new ArrayType(INTEGER), Lists.newArrayList(1, null)); + } + + @Test + public void testArrayMinWithNullsInBothArraysNotComparedSecondIsMin() + { + assertFunction("ARRAY_MIN(ARRAY [ARRAY[2, NULL], ARRAY[1, NULL]])", new ArrayType(INTEGER), Lists.newArrayList(1, null)); + } + + @Test + public void testArrayMinWithNullInFirstArrayIsCompared() + { + assertInvalidFunction("ARRAY_MIN(ARRAY [ARRAY[1, NULL], ARRAY[1, 2]])", NOT_SUPPORTED); + } + + @Test + public void testArrayMinWithNullInSecondArrayIsCompared() + { + assertInvalidFunction("ARRAY_MIN(ARRAY [ARRAY[1, 2], ARRAY[1, NULL]])", NOT_SUPPORTED); + } + @Test public void testArrayMax() { @@ -725,6 +749,30 @@ public void testArrayMax() assertDecimalFunction("ARRAY_MAX(ARRAY [2.22222222222222222, 2.3])", decimal("2.30000000000000000")); } + @Test + public void testArrayMaxWithNullsInBothArraysNotComparedSecondIsMax() + { + assertFunction("ARRAY_MAX(ARRAY [ARRAY[1, NULL], ARRAY[2, NULL]])", new ArrayType(INTEGER), Lists.newArrayList(2, null)); + } + + @Test + public void testArrayMaxWithNullsInBothArraysNotComparedFirstIsMax() + { + assertFunction("ARRAY_MAX(ARRAY [ARRAY[2, NULL], ARRAY[1, NULL]])", new ArrayType(INTEGER), Lists.newArrayList(2, null)); + } + + @Test + public void testArrayMaxWithNullInFirstArrayIsCompared() + { + assertInvalidFunction("ARRAY_MAX(ARRAY [ARRAY[1, NULL], ARRAY[1, 2]])", NOT_SUPPORTED); + } + + @Test + public void testArrayMaxWithNullInSecondArrayIsCompared() + { + assertInvalidFunction("ARRAY_MAX(ARRAY [ARRAY[1, 2], ARRAY[1, NULL]])", NOT_SUPPORTED); + } + @Test public void testArrayPosition() { From bdf89f9982a34d79c643ae5bb6f83e9842f99e5c Mon Sep 17 00:00:00 2001 From: prithvip Date: Wed, 29 May 2024 17:47:00 -0700 Subject: [PATCH 02/31] Extract interface from FailedDispatchQueryFactory --- .../FailedDispatchQueryFactory.java | 57 +++++----------- .../FailedLocalDispatchQueryFactory.java | 65 +++++++++++++++++++ .../presto/server/CoordinatorModule.java | 3 +- 3 files changed, 83 insertions(+), 42 deletions(-) create mode 100644 presto-main/src/main/java/com/facebook/presto/dispatcher/FailedLocalDispatchQueryFactory.java diff --git a/presto-main/src/main/java/com/facebook/presto/dispatcher/FailedDispatchQueryFactory.java b/presto-main/src/main/java/com/facebook/presto/dispatcher/FailedDispatchQueryFactory.java index 173f7f6757e74..ead196db10e19 100644 --- a/presto-main/src/main/java/com/facebook/presto/dispatcher/FailedDispatchQueryFactory.java +++ b/presto-main/src/main/java/com/facebook/presto/dispatcher/FailedDispatchQueryFactory.java @@ -14,50 +14,25 @@ package com.facebook.presto.dispatcher; import com.facebook.presto.Session; -import com.facebook.presto.event.QueryMonitor; -import com.facebook.presto.execution.ExecutionFailureInfo; -import com.facebook.presto.execution.LocationFactory; -import com.facebook.presto.server.BasicQueryInfo; import com.facebook.presto.spi.resourceGroups.ResourceGroupId; -import javax.inject.Inject; - import java.util.Optional; -import java.util.concurrent.ExecutorService; - -import static com.facebook.presto.util.Failures.toFailure; -import static java.util.Objects.requireNonNull; -public class FailedDispatchQueryFactory +/** + * Factory interface to create FailedDispatchQuery + * + * This interface is required for https://github.com/prestodb/presto/issues/23455 + */ +public interface FailedDispatchQueryFactory { - private final QueryMonitor queryMonitor; - private final LocationFactory locationFactory; - private final ExecutorService executor; - - @Inject - public FailedDispatchQueryFactory(QueryMonitor queryMonitor, LocationFactory locationFactory, DispatchExecutor dispatchExecutor) - { - this.queryMonitor = requireNonNull(queryMonitor, "queryMonitor is null"); - this.locationFactory = requireNonNull(locationFactory, "locationFactory is null"); - this.executor = requireNonNull(dispatchExecutor, "dispatchExecutor is null").getExecutor(); - } - - public FailedDispatchQuery createFailedDispatchQuery(Session session, String query, Optional resourceGroup, Throwable throwable) - { - ExecutionFailureInfo failure = toFailure(throwable); - FailedDispatchQuery failedDispatchQuery = new FailedDispatchQuery( - session, - query, - locationFactory.createQueryLocation(session.getQueryId()), - resourceGroup, - failure, - executor); - - BasicQueryInfo queryInfo = failedDispatchQuery.getBasicQueryInfo(); - - queryMonitor.queryCreatedEvent(queryInfo); - queryMonitor.queryImmediateFailureEvent(queryInfo, failure); - - return failedDispatchQuery; - } + /** + * If DispatchManager fails to create a DispatchQuery, a FailedDispatchQuery is created instead + * + * @param session session + * @param query query text + * @param resourceGroup resource group of the query + * @param throwable failure information about the query + * @return {@link FailedDispatchQuery} + */ + FailedDispatchQuery createFailedDispatchQuery(Session session, String query, Optional resourceGroup, Throwable throwable); } diff --git a/presto-main/src/main/java/com/facebook/presto/dispatcher/FailedLocalDispatchQueryFactory.java b/presto-main/src/main/java/com/facebook/presto/dispatcher/FailedLocalDispatchQueryFactory.java new file mode 100644 index 0000000000000..0e518950e979d --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/dispatcher/FailedLocalDispatchQueryFactory.java @@ -0,0 +1,65 @@ +/* + * 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 com.facebook.presto.dispatcher; + +import com.facebook.presto.Session; +import com.facebook.presto.event.QueryMonitor; +import com.facebook.presto.execution.ExecutionFailureInfo; +import com.facebook.presto.execution.LocationFactory; +import com.facebook.presto.server.BasicQueryInfo; +import com.facebook.presto.spi.resourceGroups.ResourceGroupId; + +import javax.inject.Inject; + +import java.util.Optional; +import java.util.concurrent.ExecutorService; + +import static com.facebook.presto.util.Failures.toFailure; +import static java.util.Objects.requireNonNull; + +public class FailedLocalDispatchQueryFactory + implements FailedDispatchQueryFactory +{ + private final QueryMonitor queryMonitor; + private final LocationFactory locationFactory; + private final ExecutorService executor; + + @Inject + public FailedLocalDispatchQueryFactory(QueryMonitor queryMonitor, LocationFactory locationFactory, DispatchExecutor dispatchExecutor) + { + this.queryMonitor = requireNonNull(queryMonitor, "queryMonitor is null"); + this.locationFactory = requireNonNull(locationFactory, "locationFactory is null"); + this.executor = requireNonNull(dispatchExecutor, "dispatchExecutor is null").getExecutor(); + } + + @Override + public FailedDispatchQuery createFailedDispatchQuery(Session session, String query, Optional resourceGroup, Throwable throwable) + { + ExecutionFailureInfo failure = toFailure(throwable); + FailedDispatchQuery failedDispatchQuery = new FailedDispatchQuery( + session, + query, + locationFactory.createQueryLocation(session.getQueryId()), + resourceGroup, + failure, + executor); + + BasicQueryInfo queryInfo = failedDispatchQuery.getBasicQueryInfo(); + + queryMonitor.queryCreatedEvent(queryInfo); + queryMonitor.queryImmediateFailureEvent(queryInfo, failure); + + return failedDispatchQuery; + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java b/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java index 0306a44c24a52..85030b2095367 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java +++ b/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java @@ -28,6 +28,7 @@ import com.facebook.presto.dispatcher.DispatchManager; import com.facebook.presto.dispatcher.DispatchQueryFactory; import com.facebook.presto.dispatcher.FailedDispatchQueryFactory; +import com.facebook.presto.dispatcher.FailedLocalDispatchQueryFactory; import com.facebook.presto.dispatcher.LocalDispatchQueryFactory; import com.facebook.presto.event.QueryMonitor; import com.facebook.presto.event.QueryMonitorConfig; @@ -192,7 +193,7 @@ protected void setup(Binder binder) // dispatcher binder.bind(DispatchManager.class).in(Scopes.SINGLETON); newExporter(binder).export(DispatchManager.class).withGeneratedName(); - binder.bind(FailedDispatchQueryFactory.class).in(Scopes.SINGLETON); + binder.bind(FailedDispatchQueryFactory.class).to(FailedLocalDispatchQueryFactory.class); binder.bind(DispatchExecutor.class).in(Scopes.SINGLETON); newExporter(binder).export(DispatchExecutor.class).withGeneratedName(); From 0c273df259fc2589e330102712c31fd2775d4d4d Mon Sep 17 00:00:00 2001 From: prithvip Date: Wed, 29 May 2024 17:03:31 -0700 Subject: [PATCH 03/31] Separate QueryPreparer from QueryAnalyzer --- .../presto/dispatcher/DispatchManager.java | 17 +++--- .../dispatcher/DispatchQueryFactory.java | 3 - .../dispatcher/LocalDispatchQueryFactory.java | 11 ++-- .../facebook/presto/server/PluginManager.java | 10 ++++ .../presto/server/ServerMainModule.java | 4 ++ .../sql/analyzer/BuiltInAnalyzerProvider.java | 11 +--- .../BuiltInQueryPreparerProvider.java | 46 ++++++++++++++++ .../QueryPreparerProviderManager.java | 55 +++++++++++++++++++ .../presto/testing/LocalQueryRunner.java | 7 ++- .../presto/spark/PrestoSparkModule.java | 4 ++ .../java/com/facebook/presto/spi/Plugin.java | 6 ++ .../presto/spi/analyzer/AnalyzerProvider.java | 2 - .../spi/analyzer/QueryPreparerProvider.java | 21 +++++++ 13 files changed, 168 insertions(+), 29 deletions(-) create mode 100644 presto-main/src/main/java/com/facebook/presto/sql/analyzer/BuiltInQueryPreparerProvider.java create mode 100644 presto-main/src/main/java/com/facebook/presto/sql/analyzer/QueryPreparerProviderManager.java create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/analyzer/QueryPreparerProvider.java diff --git a/presto-main/src/main/java/com/facebook/presto/dispatcher/DispatchManager.java b/presto-main/src/main/java/com/facebook/presto/dispatcher/DispatchManager.java index 8af2a9fcadafc..edd0a86f25ab5 100644 --- a/presto-main/src/main/java/com/facebook/presto/dispatcher/DispatchManager.java +++ b/presto-main/src/main/java/com/facebook/presto/dispatcher/DispatchManager.java @@ -35,12 +35,12 @@ import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.QueryId; import com.facebook.presto.spi.analyzer.AnalyzerOptions; -import com.facebook.presto.spi.analyzer.AnalyzerProvider; +import com.facebook.presto.spi.analyzer.QueryPreparerProvider; import com.facebook.presto.spi.resourceGroups.SelectionContext; import com.facebook.presto.spi.resourceGroups.SelectionCriteria; import com.facebook.presto.spi.security.AccessControl; import com.facebook.presto.spi.security.AuthorizedIdentity; -import com.facebook.presto.sql.analyzer.AnalyzerProviderManager; +import com.facebook.presto.sql.analyzer.QueryPreparerProviderManager; import com.facebook.presto.transaction.TransactionManager; import com.google.common.util.concurrent.AbstractFuture; import com.google.common.util.concurrent.ListenableFuture; @@ -94,7 +94,7 @@ public class DispatchManager private final QueryManagerStats stats = new QueryManagerStats(); private final SecurityConfig securityConfig; - private final AnalyzerProviderManager analyzerProviderManager; + private final QueryPreparerProviderManager queryPreparerProviderManager; /** * Dispatch Manager is used for the pre-queuing part of queries prior to the query execution phase. @@ -102,7 +102,7 @@ public class DispatchManager * Dispatch Manager object is instantiated when the presto server is launched by server bootstrap time. It is a critical component in resource management section of the query. * * @param queryIdGenerator query ID generator for generating a new query ID when a query is created - * @param analyzerProviderManager provides access to registered analyzer providers + * @param queryPreparerProviderManager provides access to registered query preparer providers * @param resourceGroupManager the resource group manager to select corresponding resource group for query to retrieve basic information from session context for selection context * @param warningCollectorFactory the warning collector factory to collect presto warning in a query session * @param dispatchQueryFactory the dispatch query factory is used to create a {@link DispatchQuery} object. The dispatch query is submitted to the {@link ResourceGroupManager} which enqueues the query. @@ -118,7 +118,7 @@ public class DispatchManager @Inject public DispatchManager( QueryIdGenerator queryIdGenerator, - AnalyzerProviderManager analyzerProviderManager, + QueryPreparerProviderManager queryPreparerProviderManager, @SuppressWarnings("rawtypes") ResourceGroupManager resourceGroupManager, WarningCollectorFactory warningCollectorFactory, DispatchQueryFactory dispatchQueryFactory, @@ -134,7 +134,7 @@ public DispatchManager( Optional clusterQueryTrackerService) { this.queryIdGenerator = requireNonNull(queryIdGenerator, "queryIdGenerator is null"); - this.analyzerProviderManager = requireNonNull(analyzerProviderManager, "analyzerProviderManager is null"); + this.queryPreparerProviderManager = requireNonNull(queryPreparerProviderManager, "queryPreparerProviderManager is null"); this.resourceGroupManager = requireNonNull(resourceGroupManager, "resourceGroupManager is null"); this.warningCollectorFactory = requireNonNull(warningCollectorFactory, "warningCollectorFactory is null"); this.dispatchQueryFactory = requireNonNull(dispatchQueryFactory, "dispatchQueryFactory is null"); @@ -286,8 +286,8 @@ private void createQueryInternal(QueryId queryId, String slug, int retryCoun // prepare query AnalyzerOptions analyzerOptions = createAnalyzerOptions(session, session.getWarningCollector()); - AnalyzerProvider analyzerProvider = analyzerProviderManager.getAnalyzerProvider(getAnalyzerType(session)); - preparedQuery = analyzerProvider.getQueryPreparer().prepareQuery(analyzerOptions, query, session.getPreparedStatements(), session.getWarningCollector()); + QueryPreparerProvider queryPreparerProvider = queryPreparerProviderManager.getQueryPreparerProvider(getAnalyzerType(session)); + preparedQuery = queryPreparerProvider.getQueryPreparer().prepareQuery(analyzerOptions, query, session.getPreparedStatements(), session.getWarningCollector()); query = preparedQuery.getFormattedQuery().orElse(query); // select resource group @@ -311,7 +311,6 @@ private void createQueryInternal(QueryId queryId, String slug, int retryCoun DispatchQuery dispatchQuery = dispatchQueryFactory.createDispatchQuery( session, - analyzerProvider, query, preparedQuery, slug, diff --git a/presto-main/src/main/java/com/facebook/presto/dispatcher/DispatchQueryFactory.java b/presto-main/src/main/java/com/facebook/presto/dispatcher/DispatchQueryFactory.java index 0e19219b6728d..b4f9c4dd17d31 100644 --- a/presto-main/src/main/java/com/facebook/presto/dispatcher/DispatchQueryFactory.java +++ b/presto-main/src/main/java/com/facebook/presto/dispatcher/DispatchQueryFactory.java @@ -17,7 +17,6 @@ import com.facebook.presto.common.analyzer.PreparedQuery; import com.facebook.presto.common.resourceGroups.QueryType; import com.facebook.presto.spi.WarningCollector; -import com.facebook.presto.spi.analyzer.AnalyzerProvider; import com.facebook.presto.spi.resourceGroups.ResourceGroupId; import java.util.Optional; @@ -32,7 +31,6 @@ public interface DispatchQueryFactory * This interface API is defined to setting up all preparation works for query before it being executed. * * @param session the session - * @param analyzerProvider the analyzer provider * @param query the query * @param preparedQuery the prepared query * @param slug the unique query slug for each {@code Query} object @@ -45,7 +43,6 @@ public interface DispatchQueryFactory */ DispatchQuery createDispatchQuery( Session session, - AnalyzerProvider analyzerProvider, String query, PreparedQuery preparedQuery, String slug, diff --git a/presto-main/src/main/java/com/facebook/presto/dispatcher/LocalDispatchQueryFactory.java b/presto-main/src/main/java/com/facebook/presto/dispatcher/LocalDispatchQueryFactory.java index 7d9fdeec5e46d..14981c4f33fed 100644 --- a/presto-main/src/main/java/com/facebook/presto/dispatcher/LocalDispatchQueryFactory.java +++ b/presto-main/src/main/java/com/facebook/presto/dispatcher/LocalDispatchQueryFactory.java @@ -28,9 +28,9 @@ import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.WarningCollector; -import com.facebook.presto.spi.analyzer.AnalyzerProvider; import com.facebook.presto.spi.resourceGroups.ResourceGroupId; import com.facebook.presto.spi.security.AccessControl; +import com.facebook.presto.sql.analyzer.AnalyzerProviderManager; import com.facebook.presto.tracing.NoopTracerProvider; import com.facebook.presto.tracing.QueryStateTracingListener; import com.facebook.presto.transaction.TransactionManager; @@ -42,6 +42,7 @@ import java.util.Optional; import java.util.function.Consumer; +import static com.facebook.presto.SystemSessionProperties.getAnalyzerType; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static java.util.Objects.requireNonNull; @@ -64,6 +65,7 @@ public class LocalDispatchQueryFactory private final ListeningExecutorService executor; private final QueryPrerequisitesManager queryPrerequisitesManager; + private final AnalyzerProviderManager analyzerProviderManager; /** * Instantiates a new Local dispatch query factory. @@ -90,7 +92,8 @@ public LocalDispatchQueryFactory( ExecutionFactoriesManager executionFactoriesManager, ClusterSizeMonitor clusterSizeMonitor, DispatchExecutor dispatchExecutor, - QueryPrerequisitesManager queryPrerequisitesManager) + QueryPrerequisitesManager queryPrerequisitesManager, + AnalyzerProviderManager analyzerProviderManager) { this.queryManager = requireNonNull(queryManager, "queryManager is null"); this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); @@ -104,6 +107,7 @@ public LocalDispatchQueryFactory( this.executor = requireNonNull(dispatchExecutor, "executorService is null").getExecutor(); this.queryPrerequisitesManager = requireNonNull(queryPrerequisitesManager, "queryPrerequisitesManager is null"); + this.analyzerProviderManager = requireNonNull(analyzerProviderManager, "analyzerProviderManager is null"); } /** @@ -131,7 +135,6 @@ public LocalDispatchQueryFactory( @Override public DispatchQuery createDispatchQuery( Session session, - AnalyzerProvider analyzerProvider, String query, PreparedQuery preparedQuery, String slug, @@ -164,7 +167,7 @@ public DispatchQuery createDispatchQuery( throw new PrestoException(NOT_SUPPORTED, "Unsupported statement type: " + preparedQuery.getStatementClass().getSimpleName()); } - return queryExecutionFactory.createQueryExecution(analyzerProvider, preparedQuery, stateMachine, slug, retryCount, warningCollector, queryType); + return queryExecutionFactory.createQueryExecution(analyzerProviderManager.getAnalyzerProvider(getAnalyzerType(session)), preparedQuery, stateMachine, slug, retryCount, warningCollector, queryType); }); return new LocalDispatchQuery( diff --git a/presto-main/src/main/java/com/facebook/presto/server/PluginManager.java b/presto-main/src/main/java/com/facebook/presto/server/PluginManager.java index 8593c32cf899a..22ecb4b1c4731 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PluginManager.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PluginManager.java @@ -30,6 +30,7 @@ import com.facebook.presto.spi.CoordinatorPlugin; import com.facebook.presto.spi.Plugin; import com.facebook.presto.spi.analyzer.AnalyzerProvider; +import com.facebook.presto.spi.analyzer.QueryPreparerProvider; import com.facebook.presto.spi.classloader.ThreadContextClassLoader; import com.facebook.presto.spi.connector.ConnectorFactory; import com.facebook.presto.spi.eventlistener.EventListenerFactory; @@ -46,6 +47,7 @@ import com.facebook.presto.spi.ttl.ClusterTtlProviderFactory; import com.facebook.presto.spi.ttl.NodeTtlFetcherFactory; import com.facebook.presto.sql.analyzer.AnalyzerProviderManager; +import com.facebook.presto.sql.analyzer.QueryPreparerProviderManager; import com.facebook.presto.storage.TempStorageManager; import com.facebook.presto.tracing.TracerProviderManager; import com.facebook.presto.ttl.clusterttlprovidermanagers.ClusterTtlProviderManager; @@ -127,6 +129,7 @@ public class PluginManager private final HistoryBasedPlanStatisticsManager historyBasedPlanStatisticsManager; private final TracerProviderManager tracerProviderManager; private final AnalyzerProviderManager analyzerProviderManager; + private final QueryPreparerProviderManager queryPreparerProviderManager; private final NodeStatusNotificationManager nodeStatusNotificationManager; @Inject @@ -137,6 +140,7 @@ public PluginManager( Metadata metadata, ResourceGroupManager resourceGroupManager, AnalyzerProviderManager analyzerProviderManager, + QueryPreparerProviderManager queryPreparerProviderManager, AccessControlManager accessControlManager, PasswordAuthenticatorManager passwordAuthenticatorManager, EventListenerManager eventListenerManager, @@ -178,6 +182,7 @@ public PluginManager( this.historyBasedPlanStatisticsManager = requireNonNull(historyBasedPlanStatisticsManager, "historyBasedPlanStatisticsManager is null"); this.tracerProviderManager = requireNonNull(tracerProviderManager, "tracerProviderManager is null"); this.analyzerProviderManager = requireNonNull(analyzerProviderManager, "analyzerProviderManager is null"); + this.queryPreparerProviderManager = requireNonNull(queryPreparerProviderManager, "queryPreparerProviderManager is null"); this.nodeStatusNotificationManager = requireNonNull(nodeStatusNotificationManager, "nodeStatusNotificationManager is null"); } @@ -334,6 +339,11 @@ public void installPlugin(Plugin plugin) analyzerProviderManager.addAnalyzerProvider(analyzerProvider); } + for (QueryPreparerProvider preparerProvider : plugin.getQueryPreparerProviders()) { + log.info("Registering query preparer provider %s", preparerProvider.getType()); + queryPreparerProviderManager.addQueryPreparerProvider(preparerProvider); + } + for (NodeStatusNotificationProviderFactory nodeStatusNotificationProviderFactory : plugin.getNodeStatusNotificationProviderFactory()) { log.info("Registering node status notification provider %s", nodeStatusNotificationProviderFactory.getName()); nodeStatusNotificationManager.addNodeStatusNotificationProviderFactory(nodeStatusNotificationProviderFactory); diff --git a/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java b/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java index 07584e5018b48..fb1b860ae73cb 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java +++ b/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java @@ -177,12 +177,14 @@ import com.facebook.presto.sql.analyzer.BuiltInAnalyzerProvider; import com.facebook.presto.sql.analyzer.BuiltInQueryAnalyzer; import com.facebook.presto.sql.analyzer.BuiltInQueryPreparer; +import com.facebook.presto.sql.analyzer.BuiltInQueryPreparerProvider; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.sql.analyzer.FeaturesConfig.SingleStreamSpillerChoice; import com.facebook.presto.sql.analyzer.ForMetadataExtractor; import com.facebook.presto.sql.analyzer.MetadataExtractor; import com.facebook.presto.sql.analyzer.MetadataExtractorMBean; import com.facebook.presto.sql.analyzer.QueryExplainer; +import com.facebook.presto.sql.analyzer.QueryPreparerProviderManager; import com.facebook.presto.sql.gen.ExpressionCompiler; import com.facebook.presto.sql.gen.JoinCompiler; import com.facebook.presto.sql.gen.JoinFilterFunctionCompiler; @@ -317,6 +319,8 @@ else if (serverConfig.isCoordinator()) { // analyzer binder.bind(BuiltInQueryPreparer.class).in(Scopes.SINGLETON); + binder.bind(BuiltInQueryPreparerProvider.class).in(Scopes.SINGLETON); + binder.bind(QueryPreparerProviderManager.class).in(Scopes.SINGLETON); newOptionalBinder(binder, QueryExplainer.class); binder.bind(BuiltInQueryAnalyzer.class).in(Scopes.SINGLETON); binder.bind(BuiltInAnalyzerProvider.class).in(Scopes.SINGLETON); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/BuiltInAnalyzerProvider.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/BuiltInAnalyzerProvider.java index 9f879593bd26a..747d4ca7ccfda 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/BuiltInAnalyzerProvider.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/BuiltInAnalyzerProvider.java @@ -15,7 +15,6 @@ import com.facebook.presto.spi.analyzer.AnalyzerProvider; import com.facebook.presto.spi.analyzer.QueryAnalyzer; -import com.facebook.presto.spi.analyzer.QueryPreparer; import com.google.inject.Inject; import static java.util.Objects.requireNonNull; @@ -24,13 +23,11 @@ public class BuiltInAnalyzerProvider implements AnalyzerProvider { private static final String PROVIDER_NAME = "BUILTIN"; - private final BuiltInQueryPreparer queryPreparer; private final BuiltInQueryAnalyzer queryAnalyzer; @Inject - public BuiltInAnalyzerProvider(BuiltInQueryPreparer queryPreparer, BuiltInQueryAnalyzer queryAnalyzer) + public BuiltInAnalyzerProvider(BuiltInQueryAnalyzer queryAnalyzer) { - this.queryPreparer = requireNonNull(queryPreparer, "queryPreparer is null"); this.queryAnalyzer = requireNonNull(queryAnalyzer, "queryAnalyzer is null"); } @@ -40,12 +37,6 @@ public String getType() return PROVIDER_NAME; } - @Override - public QueryPreparer getQueryPreparer() - { - return queryPreparer; - } - @Override public QueryAnalyzer getQueryAnalyzer() { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/BuiltInQueryPreparerProvider.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/BuiltInQueryPreparerProvider.java new file mode 100644 index 0000000000000..ab0abef0c9dc8 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/BuiltInQueryPreparerProvider.java @@ -0,0 +1,46 @@ +/* + * 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 com.facebook.presto.sql.analyzer; + +import com.facebook.presto.spi.analyzer.QueryPreparer; +import com.facebook.presto.spi.analyzer.QueryPreparerProvider; + +import javax.inject.Inject; + +import static java.util.Objects.requireNonNull; + +public class BuiltInQueryPreparerProvider + implements QueryPreparerProvider +{ + private static final String PROVIDER_NAME = "BUILTIN"; + private final BuiltInQueryPreparer queryPreparer; + + @Inject + public BuiltInQueryPreparerProvider(BuiltInQueryPreparer queryPreparer) + { + this.queryPreparer = requireNonNull(queryPreparer, "queryPreparer is null"); + } + + @Override + public String getType() + { + return PROVIDER_NAME; + } + + @Override + public QueryPreparer getQueryPreparer() + { + return queryPreparer; + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/QueryPreparerProviderManager.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/QueryPreparerProviderManager.java new file mode 100644 index 0000000000000..6cd80e0586539 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/QueryPreparerProviderManager.java @@ -0,0 +1,55 @@ +/* + * 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 com.facebook.presto.sql.analyzer; + +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.analyzer.QueryPreparerProvider; + +import javax.inject.Inject; + +import java.util.HashMap; +import java.util.Map; + +import static com.facebook.presto.spi.StandardErrorCode.UNSUPPORTED_ANALYZER_TYPE; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class QueryPreparerProviderManager +{ + private final Map queryPreparerProviders = new HashMap<>(); + + @Inject + public QueryPreparerProviderManager(BuiltInQueryPreparerProvider queryPreparerProvider) + { + addQueryPreparerProvider(queryPreparerProvider); + } + + public void addQueryPreparerProvider(QueryPreparerProvider preparerProvider) + { + requireNonNull(preparerProvider, "preparerProvider is null"); + + if (queryPreparerProviders.putIfAbsent(preparerProvider.getType(), preparerProvider) != null) { + throw new IllegalArgumentException(format("Query preparer provider '%s' is already registered", preparerProvider.getType())); + } + } + + public QueryPreparerProvider getQueryPreparerProvider(String preparerType) + { + if (queryPreparerProviders.containsKey(preparerType)) { + return queryPreparerProviders.get(preparerType); + } + + throw new PrestoException(UNSUPPORTED_ANALYZER_TYPE, "Unsupported query preparer type: " + preparerType); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/testing/LocalQueryRunner.java b/presto-main/src/main/java/com/facebook/presto/testing/LocalQueryRunner.java index 4c0d205a45722..68a23ec52cf7d 100644 --- a/presto-main/src/main/java/com/facebook/presto/testing/LocalQueryRunner.java +++ b/presto-main/src/main/java/com/facebook/presto/testing/LocalQueryRunner.java @@ -161,8 +161,10 @@ import com.facebook.presto.sql.analyzer.BuiltInQueryAnalyzer; import com.facebook.presto.sql.analyzer.BuiltInQueryPreparer; import com.facebook.presto.sql.analyzer.BuiltInQueryPreparer.BuiltInPreparedQuery; +import com.facebook.presto.sql.analyzer.BuiltInQueryPreparerProvider; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.sql.analyzer.QueryExplainer; +import com.facebook.presto.sql.analyzer.QueryPreparerProviderManager; import com.facebook.presto.sql.gen.ExpressionCompiler; import com.facebook.presto.sql.gen.JoinCompiler; import com.facebook.presto.sql.gen.JoinFilterFunctionCompiler; @@ -488,7 +490,9 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig, ImmutableSet.of()); BuiltInQueryAnalyzer queryAnalyzer = new BuiltInQueryAnalyzer(metadata, sqlParser, accessControl, Optional.empty(), metadataExtractorExecutor); - BuiltInAnalyzerProvider analyzerProvider = new BuiltInAnalyzerProvider(new BuiltInQueryPreparer(sqlParser), queryAnalyzer); + BuiltInAnalyzerProvider analyzerProvider = new BuiltInAnalyzerProvider(queryAnalyzer); + BuiltInQueryPreparer queryPreparer = new BuiltInQueryPreparer(sqlParser); + BuiltInQueryPreparerProvider queryPreparerProvider = new BuiltInQueryPreparerProvider(queryPreparer); this.pluginManager = new PluginManager( nodeInfo, @@ -497,6 +501,7 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig, metadata, new NoOpResourceGroupManager(), new AnalyzerProviderManager(analyzerProvider), + new QueryPreparerProviderManager(queryPreparerProvider), accessControl, new PasswordAuthenticatorManager(), new EventListenerManager(), diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkModule.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkModule.java index 37b4ed99ce1d2..35d82c76da81d 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkModule.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkModule.java @@ -162,11 +162,13 @@ import com.facebook.presto.sql.analyzer.BuiltInAnalyzerProvider; import com.facebook.presto.sql.analyzer.BuiltInQueryAnalyzer; import com.facebook.presto.sql.analyzer.BuiltInQueryPreparer; +import com.facebook.presto.sql.analyzer.BuiltInQueryPreparerProvider; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.sql.analyzer.ForMetadataExtractor; import com.facebook.presto.sql.analyzer.MetadataExtractor; import com.facebook.presto.sql.analyzer.MetadataExtractorMBean; import com.facebook.presto.sql.analyzer.QueryExplainer; +import com.facebook.presto.sql.analyzer.QueryPreparerProviderManager; import com.facebook.presto.sql.gen.ExpressionCompiler; import com.facebook.presto.sql.gen.JoinCompiler; import com.facebook.presto.sql.gen.JoinFilterFunctionCompiler; @@ -428,6 +430,8 @@ protected void setup(Binder binder) // analyzer binder.bind(BuiltInQueryPreparer.class).in(Scopes.SINGLETON); + binder.bind(BuiltInQueryPreparerProvider.class).in(Scopes.SINGLETON); + binder.bind(QueryPreparerProviderManager.class).in(Scopes.SINGLETON); newOptionalBinder(binder, QueryExplainer.class); binder.bind(BuiltInQueryAnalyzer.class).in(Scopes.SINGLETON); binder.bind(BuiltInAnalyzerProvider.class).in(Scopes.SINGLETON); diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/Plugin.java b/presto-spi/src/main/java/com/facebook/presto/spi/Plugin.java index 64d58edba0db0..81e8f55b0a665 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/Plugin.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/Plugin.java @@ -17,6 +17,7 @@ import com.facebook.presto.common.type.ParametricType; import com.facebook.presto.common.type.Type; import com.facebook.presto.spi.analyzer.AnalyzerProvider; +import com.facebook.presto.spi.analyzer.QueryPreparerProvider; import com.facebook.presto.spi.connector.ConnectorFactory; import com.facebook.presto.spi.eventlistener.EventListenerFactory; import com.facebook.presto.spi.function.FunctionNamespaceManagerFactory; @@ -132,6 +133,11 @@ default Iterable getAnalyzerProviders() return emptyList(); } + default Iterable getQueryPreparerProviders() + { + return emptyList(); + } + default Iterable getNodeStatusNotificationProviderFactory() { return emptyList(); diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/analyzer/AnalyzerProvider.java b/presto-spi/src/main/java/com/facebook/presto/spi/analyzer/AnalyzerProvider.java index 07fd0ce5abc3c..9864d8811fdd9 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/analyzer/AnalyzerProvider.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/analyzer/AnalyzerProvider.java @@ -17,7 +17,5 @@ public interface AnalyzerProvider { String getType(); - QueryPreparer getQueryPreparer(); - QueryAnalyzer getQueryAnalyzer(); } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/analyzer/QueryPreparerProvider.java b/presto-spi/src/main/java/com/facebook/presto/spi/analyzer/QueryPreparerProvider.java new file mode 100644 index 0000000000000..2d10798adceb3 --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/analyzer/QueryPreparerProvider.java @@ -0,0 +1,21 @@ +/* + * 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 com.facebook.presto.spi.analyzer; + +public interface QueryPreparerProvider +{ + String getType(); + + QueryPreparer getQueryPreparer(); +} From 6072d373184ba802eb0a3fd6a4e0d211114c6956 Mon Sep 17 00:00:00 2001 From: prithvip Date: Thu, 8 Aug 2024 10:43:36 -0700 Subject: [PATCH 04/31] Separate ExecutingStatement response from QueuedStatement --- .../presto/server/CoordinatorModule.java | 3 + .../ExecutingQueryResponseProvider.java | 64 +++++++++++++++++ .../LocalExecutingQueryResponseProvider.java | 70 +++++++++++++++++++ .../protocol/QueuedStatementResource.java | 62 +++++++++------- 4 files changed, 174 insertions(+), 25 deletions(-) create mode 100644 presto-main/src/main/java/com/facebook/presto/server/protocol/ExecutingQueryResponseProvider.java create mode 100644 presto-main/src/main/java/com/facebook/presto/server/protocol/LocalExecutingQueryResponseProvider.java diff --git a/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java b/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java index 85030b2095367..28f14f60315e0 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java +++ b/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java @@ -72,7 +72,9 @@ import com.facebook.presto.operator.OperatorInfo; import com.facebook.presto.resourcemanager.ForResourceManager; import com.facebook.presto.resourcemanager.ResourceManagerProxy; +import com.facebook.presto.server.protocol.ExecutingQueryResponseProvider; import com.facebook.presto.server.protocol.ExecutingStatementResource; +import com.facebook.presto.server.protocol.LocalExecutingQueryResponseProvider; import com.facebook.presto.server.protocol.LocalQueryProvider; import com.facebook.presto.server.protocol.QueryBlockingRateLimiter; import com.facebook.presto.server.protocol.QueuedStatementResource; @@ -187,6 +189,7 @@ protected void setup(Binder binder) newExporter(binder).export(QueryBlockingRateLimiter.class).withGeneratedName(); binder.bind(LocalQueryProvider.class).in(Scopes.SINGLETON); + binder.bind(ExecutingQueryResponseProvider.class).to(LocalExecutingQueryResponseProvider.class).in(Scopes.SINGLETON); jaxrsBinder(binder).bind(TaskInfoResource.class); diff --git a/presto-main/src/main/java/com/facebook/presto/server/protocol/ExecutingQueryResponseProvider.java b/presto-main/src/main/java/com/facebook/presto/server/protocol/ExecutingQueryResponseProvider.java new file mode 100644 index 0000000000000..e9de9c13b2727 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/server/protocol/ExecutingQueryResponseProvider.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 com.facebook.presto.server.protocol; + +import com.facebook.presto.dispatcher.DispatchInfo; +import com.facebook.presto.spi.QueryId; +import com.google.common.util.concurrent.ListenableFuture; +import io.airlift.units.DataSize; +import io.airlift.units.Duration; + +import javax.ws.rs.core.Response; +import javax.ws.rs.core.UriInfo; + +import java.util.Optional; + +public interface ExecutingQueryResponseProvider +{ + /** + * Generally, the Presto protocol redirects the client from QueuedStatementResource + * to ExecutingStatementResource once the query has been de-queued and begun execution. + * But this redirect might add too much latency for certain very low latency use cases. + * This interface allows for a response from ExecutingStatementResource to be wired into + * QueuedStatementResource, so that the client can receive results directly from the + * QueuedStatement endpoint, without having to be redirected to the ExecutingStatement endpoint. + * + * This interface is required for https://github.com/prestodb/presto/issues/23455 + * + * @param queryId query id + * @param slug nonce to protect the query + * @param dispatchInfo information about state of the query + * @param uriInfo endpoint URI + * @param xPrestoPrefixUrl prefix URL, that is useful if a proxy is being used + * @param scheme HTTP scheme + * @param maxWait duration to wait for query results + * @param targetResultSize target result size of first response + * @param compressionEnabled enable compression + * @param nestedDataSerializationEnabled enable nested data serialization + * @param binaryResults generate results in binary format, rather than JSON + * @return the ExecutingStatement's Response, if available + */ + Optional> waitForExecutingResponse( + QueryId queryId, + String slug, + DispatchInfo dispatchInfo, + UriInfo uriInfo, + String xPrestoPrefixUrl, + String scheme, + Duration maxWait, + DataSize targetResultSize, + boolean compressionEnabled, + boolean nestedDataSerializationEnabled, + boolean binaryResults); +} diff --git a/presto-main/src/main/java/com/facebook/presto/server/protocol/LocalExecutingQueryResponseProvider.java b/presto-main/src/main/java/com/facebook/presto/server/protocol/LocalExecutingQueryResponseProvider.java new file mode 100644 index 0000000000000..e0d7dc0c8d855 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/server/protocol/LocalExecutingQueryResponseProvider.java @@ -0,0 +1,70 @@ +/* + * 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 com.facebook.presto.server.protocol; + +import com.facebook.presto.dispatcher.DispatchInfo; +import com.facebook.presto.spi.QueryId; +import com.google.common.util.concurrent.ListenableFuture; +import io.airlift.units.DataSize; +import io.airlift.units.Duration; + +import javax.inject.Inject; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.UriInfo; + +import java.util.Optional; + +import static com.google.common.util.concurrent.Futures.transform; +import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static java.util.Objects.requireNonNull; + +public class LocalExecutingQueryResponseProvider + implements ExecutingQueryResponseProvider +{ + private final LocalQueryProvider queryProvider; + + @Inject + public LocalExecutingQueryResponseProvider(LocalQueryProvider queryProvider) + { + this.queryProvider = requireNonNull(queryProvider, "queryProvider is null"); + } + + @Override + public Optional> waitForExecutingResponse( + QueryId queryId, + String slug, + DispatchInfo dispatchInfo, + UriInfo uriInfo, + String xPrestoPrefixUrl, + String scheme, + Duration maxWait, + DataSize targetResultSize, + boolean compressionEnabled, + boolean nestedDataSerializationEnabled, + boolean binaryResults) + { + Query query; + try { + query = queryProvider.getQuery(queryId, slug); + } + catch (WebApplicationException e) { + return Optional.empty(); + } + return Optional.of(transform( + query.waitForResults(0, uriInfo, scheme, maxWait, targetResultSize, binaryResults), + results -> QueryResourceUtil.toResponse(query, results, xPrestoPrefixUrl, compressionEnabled, nestedDataSerializationEnabled), + directExecutor())); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/server/protocol/QueuedStatementResource.java b/presto-main/src/main/java/com/facebook/presto/server/protocol/QueuedStatementResource.java index 3f62c930d0792..16a0c240d067b 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/protocol/QueuedStatementResource.java +++ b/presto-main/src/main/java/com/facebook/presto/server/protocol/QueuedStatementResource.java @@ -90,7 +90,6 @@ import static com.google.common.net.HttpHeaders.X_FORWARDED_PROTO; import static com.google.common.util.concurrent.Futures.immediateFailedFuture; import static com.google.common.util.concurrent.Futures.immediateFuture; -import static com.google.common.util.concurrent.Futures.transform; import static com.google.common.util.concurrent.Futures.transformAsync; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; import static io.airlift.units.DataSize.Unit.MEGABYTE; @@ -116,7 +115,7 @@ public class QueuedStatementResource private static final Duration NO_DURATION = new Duration(0, MILLISECONDS); private final DispatchManager dispatchManager; - private final LocalQueryProvider queryResultsProvider; + private final ExecutingQueryResponseProvider executingQueryResponseProvider; private final Executor responseExecutor; private final ScheduledExecutorService timeoutExecutor; @@ -137,7 +136,7 @@ public class QueuedStatementResource public QueuedStatementResource( DispatchManager dispatchManager, DispatchExecutor executor, - LocalQueryProvider queryResultsProvider, + ExecutingQueryResponseProvider executingQueryResponseProvider, SqlParserOptions sqlParserOptions, ServerConfig serverConfig, TracerProviderManager tracerProviderManager, @@ -145,7 +144,7 @@ public QueuedStatementResource( QueryBlockingRateLimiter queryRateLimiter) { this.dispatchManager = requireNonNull(dispatchManager, "dispatchManager is null"); - this.queryResultsProvider = queryResultsProvider; + this.executingQueryResponseProvider = requireNonNull(executingQueryResponseProvider, "executingQueryResponseProvider is null"); this.sqlParserOptions = requireNonNull(sqlParserOptions, "sqlParserOptions is null"); this.compressionEnabled = requireNonNull(serverConfig, "serverConfig is null").isQueryResultsCompressionEnabled(); this.nestedDataSerializationEnabled = requireNonNull(serverConfig, "serverConfig is null").isNestedDataSerializationEnabled(); @@ -221,7 +220,7 @@ public Response postStatement( sqlParserOptions, tracerProviderManager.getTracerProvider(), Optional.of(sessionPropertyManager)); - Query query = new Query(statement, sessionContext, dispatchManager, queryResultsProvider, 0); + Query query = new Query(statement, sessionContext, dispatchManager, executingQueryResponseProvider, 0); queries.put(query.getQueryId(), query); return withCompressionConfiguration(Response.ok(query.getInitialQueryResults(uriInfo, xForwardedProto, xPrestoPrefixUrl, binaryResults)), compressionEnabled).build(); @@ -258,7 +257,7 @@ public Response retryFailedQuery( "-- retry query " + queryId + "; attempt: " + retryCount + "\n" + failedQuery.getQuery(), failedQuery.getSessionContext(), dispatchManager, - queryResultsProvider, + executingQueryResponseProvider, retryCount); retriedQueries.putIfAbsent(queryId, query); @@ -453,7 +452,7 @@ private static final class Query private final String query; private final SessionContext sessionContext; private final DispatchManager dispatchManager; - private final LocalQueryProvider queryProvider; + private final ExecutingQueryResponseProvider executingQueryResponseProvider; private final QueryId queryId; private final String slug = "x" + randomUUID().toString().toLowerCase(ENGLISH).replace("-", ""); private final AtomicLong lastToken = new AtomicLong(); @@ -462,12 +461,12 @@ private static final class Query @GuardedBy("this") private ListenableFuture querySubmissionFuture; - public Query(String query, SessionContext sessionContext, DispatchManager dispatchManager, LocalQueryProvider queryResultsProvider, int retryCount) + public Query(String query, SessionContext sessionContext, DispatchManager dispatchManager, ExecutingQueryResponseProvider executingQueryResponseProvider, int retryCount) { this.query = requireNonNull(query, "query is null"); this.sessionContext = requireNonNull(sessionContext, "sessionContext is null"); this.dispatchManager = requireNonNull(dispatchManager, "dispatchManager is null"); - this.queryProvider = requireNonNull(queryResultsProvider, "queryExecutor is null"); + this.executingQueryResponseProvider = requireNonNull(executingQueryResponseProvider, "executingQueryResponseProvider is null"); this.queryId = dispatchManager.createQueryId(); this.retryCount = retryCount; } @@ -565,7 +564,15 @@ public synchronized QueryResults getInitialQueryResults(UriInfo uriInfo, String binaryResults); } - public ListenableFuture toResponse(long token, UriInfo uriInfo, String xForwardedProto, String xPrestoPrefixUrl, Duration maxWait, boolean compressionEnabled, boolean nestedDataSerializationEnabled, boolean binaryResults) + public ListenableFuture toResponse( + long token, + UriInfo uriInfo, + String xForwardedProto, + String xPrestoPrefixUrl, + Duration maxWait, + boolean compressionEnabled, + boolean nestedDataSerializationEnabled, + boolean binaryResults) { long lastToken = this.lastToken.get(); // token should be the last token or the next token @@ -597,23 +604,28 @@ public ListenableFuture toResponse(long token, UriInfo uriInfo, String .build())); } - if (!waitForDispatched().isDone()) { - return immediateFuture(withCompressionConfiguration(Response.ok(createQueryResults(token + 1, uriInfo, xForwardedProto, xPrestoPrefixUrl, dispatchInfo.get(), binaryResults)), compressionEnabled).build()); + if (waitForDispatched().isDone()) { + Optional> executingQueryResponse = executingQueryResponseProvider.waitForExecutingResponse( + queryId, + slug, + dispatchInfo.get(), + uriInfo, + xPrestoPrefixUrl, + getScheme(xForwardedProto, uriInfo), + maxWait, + TARGET_RESULT_SIZE, + compressionEnabled, + nestedDataSerializationEnabled, + binaryResults); + + if (executingQueryResponse.isPresent()) { + return executingQueryResponse.get(); + } } - com.facebook.presto.server.protocol.Query query; - try { - query = queryProvider.getQuery(queryId, slug); - } - catch (WebApplicationException e) { - return immediateFuture(withCompressionConfiguration(Response.ok(createQueryResults(token + 1, uriInfo, xForwardedProto, xPrestoPrefixUrl, dispatchInfo.get(), binaryResults)), compressionEnabled).build()); - } - // If this future completes successfully, the next URI will redirect to the executing statement endpoint. - // Hence it is safe to hardcode the token to be 0. - return transform( - query.waitForResults(0, uriInfo, getScheme(xForwardedProto, uriInfo), maxWait, TARGET_RESULT_SIZE, binaryResults), - results -> QueryResourceUtil.toResponse(query, results, xPrestoPrefixUrl, compressionEnabled, nestedDataSerializationEnabled), - directExecutor()); + return immediateFuture(withCompressionConfiguration(Response.ok( + createQueryResults(token + 1, uriInfo, xForwardedProto, xPrestoPrefixUrl, dispatchInfo.get(), binaryResults)), compressionEnabled) + .build()); } public synchronized void cancel() From ddc33d6ab6ec78688e1c10e8b0c01dfebb024165 Mon Sep 17 00:00:00 2001 From: prithvip Date: Thu, 8 Aug 2024 12:02:14 -0700 Subject: [PATCH 05/31] Move QueuedStatementResource helpers to QueryResourceUtil --- .../server/protocol/QueryResourceUtil.java | 76 ++++++++++++++++++ .../protocol/QueuedStatementResource.java | 78 ++----------------- 2 files changed, 81 insertions(+), 73 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/server/protocol/QueryResourceUtil.java b/presto-main/src/main/java/com/facebook/presto/server/protocol/QueryResourceUtil.java index 980f81ffe46d6..79cdc1ba335c0 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/protocol/QueryResourceUtil.java +++ b/presto-main/src/main/java/com/facebook/presto/server/protocol/QueryResourceUtil.java @@ -16,6 +16,7 @@ import com.facebook.airlift.json.JsonCodec; import com.facebook.airlift.log.Logger; import com.facebook.presto.client.Column; +import com.facebook.presto.client.QueryError; import com.facebook.presto.client.QueryResults; import com.facebook.presto.client.StageStats; import com.facebook.presto.client.StatementStats; @@ -30,13 +31,18 @@ import com.facebook.presto.execution.StageExecutionStats; import com.facebook.presto.execution.StageInfo; import com.facebook.presto.execution.TaskInfo; +import com.facebook.presto.spi.QueryId; import com.facebook.presto.spi.function.SqlFunctionId; import com.facebook.presto.spi.function.SqlInvokedFunction; import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; +import io.airlift.units.Duration; import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Context; import javax.ws.rs.core.Response; +import javax.ws.rs.core.UriBuilder; +import javax.ws.rs.core.UriInfo; import java.io.UnsupportedEncodingException; import java.net.URI; @@ -49,6 +55,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import static com.facebook.airlift.json.JsonCodec.jsonCodec; @@ -70,11 +77,15 @@ import static com.facebook.presto.common.type.StandardTypes.MAP; import static com.facebook.presto.common.type.StandardTypes.ROW; import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; +import static com.facebook.presto.execution.QueryState.FAILED; +import static com.facebook.presto.execution.QueryState.QUEUED; +import static com.facebook.presto.execution.QueryState.WAITING_FOR_PREREQUISITES; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.isNullOrEmpty; import static java.lang.String.format; import static java.util.Collections.unmodifiableList; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.stream.Collectors.toList; import static javax.ws.rs.core.MediaType.TEXT_PLAIN_TYPE; @@ -85,6 +96,7 @@ public final class QueryResourceUtil private static final JsonCodec SQL_INVOKED_FUNCTION_JSON_CODEC = jsonCodec(SqlInvokedFunction.class); private static final JsonCodec> LIST_JSON_CODEC = listJsonCodec(Object.class); private static final JsonCodec> MAP_JSON_CODEC = mapJsonCodec(Object.class, Object.class); + static final Duration NO_DURATION = new Duration(0, MILLISECONDS); private QueryResourceUtil() {} @@ -369,4 +381,68 @@ public static Object parseToJson(TypeSignature signature, Object value) } return value; } + + private static URI getQueryHtmlUri(QueryId queryId, UriInfo uriInfo, String xForwardedProto, String xPrestoPrefixUrl) + { + URI uri = uriInfo.getRequestUriBuilder() + .scheme(getScheme(xForwardedProto, uriInfo)) + .replacePath("ui/query.html") + .replaceQuery(queryId.toString()) + .build(); + return prependUri(uri, xPrestoPrefixUrl); + } + + public static URI getQueuedUri(QueryId queryId, String slug, long token, UriInfo uriInfo, String xForwardedProto, String xPrestoPrefixUrl, boolean binaryResults) + { + UriBuilder uriBuilder = uriInfo.getBaseUriBuilder() + .scheme(getScheme(xForwardedProto, uriInfo)) + .replacePath("/v1/statement/queued") + .path(queryId.toString()) + .path(String.valueOf(token)) + .replaceQuery("") + .queryParam("slug", slug); + if (binaryResults) { + uriBuilder.queryParam("binaryResults", "true"); + } + URI uri = uriBuilder.build(); + return prependUri(uri, xPrestoPrefixUrl); + } + + public static String getScheme(String xForwardedProto, @Context UriInfo uriInfo) + { + return isNullOrEmpty(xForwardedProto) ? uriInfo.getRequestUri().getScheme() : xForwardedProto; + } + + public static QueryResults createQueuedQueryResults( + QueryId queryId, + URI nextUri, + Optional queryError, + UriInfo uriInfo, + String xForwardedProto, + String xPrestoPrefixUrl, + Duration elapsedTime, + Optional queuedTime, + Duration waitingForPrerequisitesTime) + { + QueryState state = queryError.map(error -> FAILED).orElse(queuedTime.isPresent() ? QUEUED : WAITING_FOR_PREREQUISITES); + return new QueryResults( + queryId.toString(), + getQueryHtmlUri(queryId, uriInfo, xForwardedProto, xPrestoPrefixUrl), + null, + nextUri, + null, + null, + null, + StatementStats.builder() + .setState(state.toString()) + .setWaitingForPrerequisites(state == WAITING_FOR_PREREQUISITES) + .setElapsedTimeMillis(elapsedTime.toMillis()) + .setQueuedTimeMillis(queuedTime.orElse(NO_DURATION).toMillis()) + .setWaitingForPrerequisitesTimeMillis(waitingForPrerequisitesTime.toMillis()) + .build(), + queryError.orElse(null), + ImmutableList.of(), + null, + null); + } } diff --git a/presto-main/src/main/java/com/facebook/presto/server/protocol/QueuedStatementResource.java b/presto-main/src/main/java/com/facebook/presto/server/protocol/QueuedStatementResource.java index 16a0c240d067b..ffbb26691398e 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/protocol/QueuedStatementResource.java +++ b/presto-main/src/main/java/com/facebook/presto/server/protocol/QueuedStatementResource.java @@ -17,13 +17,11 @@ import com.facebook.airlift.stats.TimeStat; import com.facebook.presto.client.QueryError; import com.facebook.presto.client.QueryResults; -import com.facebook.presto.client.StatementStats; import com.facebook.presto.common.ErrorCode; import com.facebook.presto.dispatcher.DispatchExecutor; import com.facebook.presto.dispatcher.DispatchInfo; import com.facebook.presto.dispatcher.DispatchManager; import com.facebook.presto.execution.ExecutionFailureInfo; -import com.facebook.presto.execution.QueryState; import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.server.HttpRequestSessionContext; import com.facebook.presto.server.ServerConfig; @@ -32,7 +30,6 @@ import com.facebook.presto.spi.QueryId; import com.facebook.presto.sql.parser.SqlParserOptions; import com.facebook.presto.tracing.TracerProviderManager; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Ordering; import com.google.common.util.concurrent.ListenableFuture; @@ -61,7 +58,6 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; -import javax.ws.rs.core.UriBuilder; import javax.ws.rs.core.UriInfo; import java.net.URI; @@ -77,10 +73,11 @@ import static com.facebook.airlift.concurrent.Threads.threadsNamed; import static com.facebook.airlift.http.server.AsyncResponseHandler.bindAsyncResponse; import static com.facebook.presto.client.PrestoHeaders.PRESTO_PREFIX_URL; -import static com.facebook.presto.execution.QueryState.FAILED; -import static com.facebook.presto.execution.QueryState.QUEUED; -import static com.facebook.presto.execution.QueryState.WAITING_FOR_PREREQUISITES; +import static com.facebook.presto.server.protocol.QueryResourceUtil.NO_DURATION; import static com.facebook.presto.server.protocol.QueryResourceUtil.abortIfPrefixUrlInvalid; +import static com.facebook.presto.server.protocol.QueryResourceUtil.createQueuedQueryResults; +import static com.facebook.presto.server.protocol.QueryResourceUtil.getQueuedUri; +import static com.facebook.presto.server.protocol.QueryResourceUtil.getScheme; import static com.facebook.presto.server.security.RoleType.USER; import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static com.facebook.presto.spi.StandardErrorCode.RETRY_QUERY_NOT_FOUND; @@ -112,7 +109,6 @@ public class QueuedStatementResource private static final Duration MAX_WAIT_TIME = new Duration(1, SECONDS); private static final DataSize TARGET_RESULT_SIZE = new DataSize(1, MEGABYTE); private static final Ordering> WAIT_ORDERING = Ordering.natural().nullsLast(); - private static final Duration NO_DURATION = new Duration(0, MILLISECONDS); private final DispatchManager dispatchManager; private final ExecutingQueryResponseProvider executingQueryResponseProvider; @@ -366,70 +362,6 @@ private void purgeQueries(Map queries) } } - private static URI getQueryHtmlUri(QueryId queryId, UriInfo uriInfo, String xForwardedProto, String xPrestoPrefixUrl) - { - URI uri = uriInfo.getRequestUriBuilder() - .scheme(getScheme(xForwardedProto, uriInfo)) - .replacePath("ui/query.html") - .replaceQuery(queryId.toString()) - .build(); - return QueryResourceUtil.prependUri(uri, xPrestoPrefixUrl); - } - - private static URI getQueuedUri(QueryId queryId, String slug, long token, UriInfo uriInfo, String xForwardedProto, String xPrestoPrefixUrl, boolean binaryResults) - { - UriBuilder uriBuilder = uriInfo.getBaseUriBuilder() - .scheme(getScheme(xForwardedProto, uriInfo)) - .replacePath("/v1/statement/queued") - .path(queryId.toString()) - .path(String.valueOf(token)) - .replaceQuery("") - .queryParam("slug", slug); - if (binaryResults) { - uriBuilder.queryParam("binaryResults", "true"); - } - URI uri = uriBuilder.build(); - return QueryResourceUtil.prependUri(uri, xPrestoPrefixUrl); - } - - private static String getScheme(String xForwardedProto, @Context UriInfo uriInfo) - { - return isNullOrEmpty(xForwardedProto) ? uriInfo.getRequestUri().getScheme() : xForwardedProto; - } - - private static QueryResults createQueryResults( - QueryId queryId, - URI nextUri, - Optional queryError, - UriInfo uriInfo, - String xForwardedProto, - String xPrestoPrefixUrl, - Duration elapsedTime, - Optional queuedTime, - Duration waitingForPrerequisitesTime) - { - QueryState state = queryError.map(error -> FAILED).orElse(queuedTime.isPresent() ? QUEUED : WAITING_FOR_PREREQUISITES); - return new QueryResults( - queryId.toString(), - getQueryHtmlUri(queryId, uriInfo, xForwardedProto, xPrestoPrefixUrl), - null, - nextUri, - null, - null, - null, - StatementStats.builder() - .setState(state.toString()) - .setWaitingForPrerequisites(state == WAITING_FOR_PREREQUISITES) - .setElapsedTimeMillis(elapsedTime.toMillis()) - .setQueuedTimeMillis(queuedTime.orElse(NO_DURATION).toMillis()) - .setWaitingForPrerequisitesTimeMillis(waitingForPrerequisitesTime.toMillis()) - .build(), - queryError.orElse(null), - ImmutableList.of(), - null, - null); - } - private static WebApplicationException badRequest(Status status, String message) { throw new WebApplicationException( @@ -640,7 +572,7 @@ private QueryResults createQueryResults(long token, UriInfo uriInfo, String xFor Optional queryError = dispatchInfo.getFailureInfo() .map(this::toQueryError); - return QueuedStatementResource.createQueryResults( + return createQueuedQueryResults( queryId, nextUri, queryError, From 4dab5836ba5ab99dc718584083d7b64f4a5de9d3 Mon Sep 17 00:00:00 2001 From: Sergey Pershin Date: Thu, 15 Aug 2024 17:11:15 -0700 Subject: [PATCH 06/31] [native] Comment out failing tests temporarely. --- .../AbstractTestNativeGeneralQueries.java | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeGeneralQueries.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeGeneralQueries.java index 472d0d01e0ddd..aa2a398dec305 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeGeneralQueries.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeGeneralQueries.java @@ -1051,10 +1051,11 @@ public void testFileSizeHiddenColumn() // A bug used to return all rows even though filters on hidden column were present in the query. // So checking the count here to verify the number of rows returned is correct. Since the bug was present // for both Java and Native Presto for non-$path columns, the assertQuery test above used to pass. - assertEquals( - (Long) computeActual(format("SELECT count(*) from test_hidden_columns WHERE \"$file_size\"=%d", fileSize)) - .getOnlyValue(), - 1L); + // TODO(spershin): Figure why this test fails in internal environment, fix and then enable it back. + // assertEquals( + // (Long) computeActual(format("SELECT count(*) from test_hidden_columns WHERE \"$file_size\"=%d", fileSize)) + // .getOnlyValue(), + // 1L); } @Test @@ -1065,12 +1066,12 @@ public void testFileModifiedTimeHiddenColumn() // Fetch one of the file modified times and use it as a filter. Long fileModifiedTime = (Long) computeActual("SELECT \"$file_modified_time\" from test_hidden_columns LIMIT 1").getOnlyValue(); assertQuery(format("SELECT * from test_hidden_columns WHERE \"$file_modified_time\"=%d", fileModifiedTime)); - - assertEquals( - (Long) computeActual( - format("SELECT count(*) from " + - "test_hidden_columns WHERE \"$file_modified_time\"=%d", fileModifiedTime)).getOnlyValue(), - 1L); + // TODO(spershin): Figure why this test fails in internal environment, fix and then enable it back. + // assertEquals( + // (Long) computeActual( + // format("SELECT count(*) from " + + // "test_hidden_columns WHERE \"$file_modified_time\"=%d", fileModifiedTime)).getOnlyValue(), + // 1L); } @Test From bd444ec9e0c44cd0f41fcbed2de04e7ba837d533 Mon Sep 17 00:00:00 2001 From: Rebecca Schlussel Date: Thu, 15 Aug 2024 13:49:22 -0400 Subject: [PATCH 07/31] Fix failure with enabling JDBC autocommit There was a bug where enabling autocommit when it had previously been false would cause failure like: java.sql.SQLException: Connection is in auto-commit mode This was due to calling commit() from setAutoCommit() after autocommit had already been changed to true. Fixes #15916 --- .../presto/jdbc/PrestoConnection.java | 3 +- .../presto/jdbc/TestJdbcConnection.java | 82 +++++++++++++++++-- 2 files changed, 75 insertions(+), 10 deletions(-) diff --git a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoConnection.java b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoConnection.java index 46d48b5dc928f..1f3f760631850 100644 --- a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoConnection.java +++ b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoConnection.java @@ -187,10 +187,11 @@ public void setAutoCommit(boolean autoCommit) throws SQLException { checkOpen(); - boolean wasAutoCommit = this.autoCommit.getAndSet(autoCommit); + boolean wasAutoCommit = this.autoCommit.get(); if (autoCommit && !wasAutoCommit) { commit(); } + this.autoCommit.set(autoCommit); } @Override diff --git a/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestJdbcConnection.java b/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestJdbcConnection.java index ed77488e19498..7e80977af6349 100644 --- a/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestJdbcConnection.java +++ b/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestJdbcConnection.java @@ -105,21 +105,85 @@ public void teardownServer() public void testCommit() throws SQLException { - try (Connection connection = createConnection()) { - connection.setAutoCommit(false); - try (Statement statement = connection.createStatement()) { - statement.execute("CREATE TABLE test_commit (x bigint)"); + try { + try (Connection connection = createConnection()) { + connection.setAutoCommit(false); + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE TABLE test_commit (x bigint)"); + } + + try (Connection otherConnection = createConnection()) { + assertThat(listTables(otherConnection)).doesNotContain("test_commit"); + } + + connection.commit(); } - try (Connection otherConnection = createConnection()) { - assertThat(listTables(otherConnection)).doesNotContain("test_commit"); + try (Connection connection = createConnection()) { + assertThat(listTables(connection)).contains("test_commit"); + } + } + finally { + try (Connection connection = createConnection()) { + try (Statement statement = connection.createStatement()) { + statement.execute("DROP TABLE test_commit"); + } + } + } + } + + @Test + public void testAutoCommit() + throws SQLException + { + try { + try (Connection connection = createConnection()) { + connection.setAutoCommit(true); + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE TABLE test_autocommit (x bigint)"); + } } - connection.commit(); + try (Connection connection = createConnection()) { + assertThat(listTables(connection)).contains("test_autocommit"); + } + } + finally { + try (Connection connection = createConnection()) { + try (Statement statement = connection.createStatement()) { + statement.execute("DROP TABLE test_autocommit"); + } + } } + } - try (Connection connection = createConnection()) { - assertThat(listTables(connection)).contains("test_commit"); + @Test + public void testResetAutoCommit() + throws SQLException + { + try { + try (Connection connection = createConnection()) { + connection.setAutoCommit(false); + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE TABLE test_reset_autocommit (x bigint)"); + } + + try (Connection otherConnection = createConnection()) { + assertThat(listTables(otherConnection)).doesNotContain("test_reset_autocommit"); + } + connection.setAutoCommit(true); + } + + try (Connection connection = createConnection()) { + assertThat(listTables(connection)).contains("test_reset_autocommit"); + } + } + finally { + try (Connection connection = createConnection()) { + try (Statement statement = connection.createStatement()) { + statement.execute("DROP TABLE test_reset_autocommit"); + } + } } } From 0f7aa2eda11b5c22ff3cd42e70e962efb9233f65 Mon Sep 17 00:00:00 2001 From: Jialiang Tan Date: Fri, 16 Aug 2024 10:21:27 -0700 Subject: [PATCH 08/31] [native] Fill protocol::TaskStats::completedDrivers with finished splits --- presto-native-execution/presto_cpp/main/PrestoTask.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/presto-native-execution/presto_cpp/main/PrestoTask.cpp b/presto-native-execution/presto_cpp/main/PrestoTask.cpp index 229be115e4d80..52589abe4b3de 100644 --- a/presto-native-execution/presto_cpp/main/PrestoTask.cpp +++ b/presto-native-execution/presto_cpp/main/PrestoTask.cpp @@ -593,7 +593,15 @@ void PrestoTask::updateExecutionInfoLocked( prestoTaskStats.queuedDrivers = veloxTaskStats.numQueuedDrivers; prestoTaskStats.totalDrivers = veloxTaskStats.numTotalDrivers; prestoTaskStats.runningDrivers = veloxTaskStats.numRunningDrivers; - prestoTaskStats.completedDrivers = veloxTaskStats.numCompletedDrivers; + // TODO: Presto Java reports number of drivers to number of splits in Presto + // UI because split and driver are 1 to 1 mapping relationship. This is not + // true in Prestissimo where 1 driver handles many splits. In order to + // quickly unblock developers from viewing the correct number of completed + // splits in Prestissimo's coordinator UI, we put numFinishedSplits in + // protocol::TaskStats::completedDrivers. We should really extend the API + // (protocol::TaskStats and Presto coordinator UI) to have splits information + // as a proper fix. + prestoTaskStats.completedDrivers = veloxTaskStats.numFinishedSplits; prestoTaskStats.pipelines.resize(veloxTaskStats.pipelineStats.size()); for (int i = 0; i < veloxTaskStats.pipelineStats.size(); ++i) { From c75087f144ccba5e1c8f642f2eaa4331bcb51afb Mon Sep 17 00:00:00 2001 From: Deepak Majeti Date: Wed, 10 Jul 2024 08:25:21 -0400 Subject: [PATCH 09/31] [native] Add Velox factory registrations Velox library will be removing the factory registrations for connectors, readers, sinks, etc... We now need to register the factories in Prestissimo. --- .../presto_cpp/main/PrestoServer.cpp | 61 ++++++++++++++++++- .../presto_cpp/main/PrestoServer.h | 10 ++- 2 files changed, 68 insertions(+), 3 deletions(-) diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.cpp b/presto-native-execution/presto_cpp/main/PrestoServer.cpp index cd9a396c59f65..e50caeaac3920 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.cpp +++ b/presto-native-execution/presto_cpp/main/PrestoServer.cpp @@ -47,6 +47,16 @@ #include "velox/common/memory/SharedArbitrator.h" #include "velox/connectors/Connector.h" #include "velox/connectors/hive/HiveConnector.h" +#include "velox/connectors/hive/HiveDataSink.h" +#include "velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.h" +#include "velox/connectors/hive/storage_adapters/gcs/RegisterGCSFileSystem.h" +#include "velox/connectors/hive/storage_adapters/hdfs/RegisterHdfsFileSystem.h" +#include "velox/connectors/hive/storage_adapters/s3fs/RegisterS3FileSystem.h" +#include "velox/connectors/tpch/TpchConnector.h" +#include "velox/dwio/dwrf/RegisterDwrfReader.h" +#include "velox/dwio/dwrf/RegisterDwrfWriter.h" +#include "velox/dwio/parquet/RegisterParquetReader.h" +#include "velox/dwio/parquet/RegisterParquetWriter.h" #include "velox/exec/OutputBufferManager.h" #include "velox/functions/prestosql/aggregates/RegisterAggregateFunctions.h" #include "velox/functions/prestosql/registration/RegistrationFunctions.h" @@ -64,8 +74,6 @@ #endif namespace facebook::presto { -using namespace facebook::velox; - namespace { constexpr char const* kHttp = "http"; @@ -73,6 +81,7 @@ constexpr char const* kHttps = "https"; constexpr char const* kTaskUriFormat = "{}://{}:{}"; // protocol, address and port constexpr char const* kConnectorName = "connector.name"; +constexpr char const* kHiveHadoop2ConnectorName = "hive-hadoop2"; protocol::NodeState convertNodeState(presto::NodeState nodeState) { switch (nodeState) { @@ -230,9 +239,11 @@ void PrestoServer::run() { registerFileSinks(); registerFileSystems(); + registerFileReadersAndWriters(); registerMemoryArbitrators(); registerShuffleInterfaceFactories(); registerCustomOperators(); + registerConnectorFactories(); // Register Velox connector factory for iceberg. // The iceberg catalog is handled by the hive connector factory. @@ -577,6 +588,8 @@ void PrestoServer::run() { PRESTO_SHUTDOWN_LOG(INFO) << "Destroying HTTP Server"; httpServer_.reset(); + unregisterFileReadersAndWriters(); + unregisterFileSystems(); unregisterConnectors(); PRESTO_SHUTDOWN_LOG(INFO) @@ -1052,6 +1065,24 @@ PrestoServer::getAdditionalHttpServerFilters() { return filters; } +void PrestoServer::registerConnectorFactories() { + // These checks for connector factories can be removed after we remove the + // registrations from the Velox library. + if (!connector::hasConnectorFactory( + connector::hive::HiveConnectorFactory::kHiveConnectorName)) { + connector::registerConnectorFactory( + std::make_shared()); + connector::registerConnectorFactory( + std::make_shared( + kHiveHadoop2ConnectorName)); + } + if (!connector::hasConnectorFactory( + connector::tpch::TpchConnectorFactory::kTpchConnectorName)) { + connector::registerConnectorFactory( + std::make_shared()); + } +} + std::vector PrestoServer::registerConnectors( const fs::path& configDirectoryPath) { static const std::string kPropertiesExtension = ".properties"; @@ -1212,14 +1243,40 @@ void PrestoServer::registerVectorSerdes() { } } +void PrestoServer::registerFileSinks() { + velox::dwio::common::registerFileSinks(); +} + void PrestoServer::registerFileSystems() { velox::filesystems::registerLocalFileSystem(); + velox::filesystems::registerS3FileSystem(); + velox::filesystems::registerHdfsFileSystem(); + velox::filesystems::registerGCSFileSystem(); + velox::filesystems::abfs::registerAbfsFileSystem(); +} + +void PrestoServer::unregisterFileSystems() { + velox::filesystems::finalizeS3FileSystem(); } void PrestoServer::registerMemoryArbitrators() { velox::memory::SharedArbitrator::registerFactory(); } +void PrestoServer::registerFileReadersAndWriters() { + velox::dwrf::registerDwrfReaderFactory(); + velox::dwrf::registerDwrfWriterFactory(); + velox::parquet::registerParquetReaderFactory(); + velox::parquet::registerParquetWriterFactory(); +} + +void PrestoServer::unregisterFileReadersAndWriters() { + velox::dwrf::unregisterDwrfReaderFactory(); + velox::dwrf::unregisterDwrfWriterFactory(); + velox::parquet::unregisterParquetReaderFactory(); + velox::parquet::unregisterParquetWriterFactory(); +} + void PrestoServer::registerStatsCounters() { registerPrestoMetrics(); registerVeloxMetrics(); diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.h b/presto-native-execution/presto_cpp/main/PrestoServer.h index e4b7466340f31..f74b7dc65e72c 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.h +++ b/presto-native-execution/presto_cpp/main/PrestoServer.h @@ -136,7 +136,13 @@ class PrestoServer { /// Invoked to register the required dwio data sinks which are used by /// connectors. - virtual void registerFileSinks() {} + virtual void registerFileSinks(); + + virtual void registerFileReadersAndWriters(); + + virtual void unregisterFileReadersAndWriters(); + + virtual void registerConnectorFactories(); /// Invoked by presto shutdown procedure to unregister connectors. virtual void unregisterConnectors(); @@ -153,6 +159,8 @@ class PrestoServer { virtual void registerFileSystems(); + virtual void unregisterFileSystems(); + virtual void registerMemoryArbitrators(); /// Invoked after creating global (singleton) config objects (SystemConfig and From 34e73df05b3d960a3ddb84140d37ba9867d0b680 Mon Sep 17 00:00:00 2001 From: Sergey Pershin Date: Fri, 16 Aug 2024 19:32:50 -0700 Subject: [PATCH 10/31] Reenabling recently disabled E2E tests. --- .../presto/hive/HiveSplitManager.java | 11 ---------- .../presto/hive/ManifestPartitionLoader.java | 4 ++-- .../presto/hive/StoragePartitionLoader.java | 2 +- .../hive/util/InternalHiveSplitFactory.java | 9 --------- .../AbstractTestNativeGeneralQueries.java | 20 +++++++++---------- 5 files changed, 12 insertions(+), 34 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java index 18adab6976cd6..62e620bee3e2f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java @@ -85,7 +85,6 @@ import static com.facebook.presto.common.type.Decimals.isShortDecimal; import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveColumnHandle.isInfoColumnHandle; -import static com.facebook.presto.hive.HiveColumnHandle.isPathColumnHandle; import static com.facebook.presto.hive.HiveCommonSessionProperties.isUseParquetColumnNames; import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; import static com.facebook.presto.hive.HiveErrorCode.HIVE_PARTITION_DROPPED_DURING_QUERY; @@ -426,16 +425,6 @@ private HiveSplitSource computeSplitSource(SplitSchedulingContext splitSchedulin return splitSource; } - private static Optional getPathDomain(TupleDomain domainPredicate, Map predicateColumns) - { - checkArgument(!domainPredicate.isNone(), "Unexpected domain predicate: none"); - - return domainPredicate.getDomains().get().entrySet().stream() - .filter(entry -> isPathColumnHandle(predicateColumns.get(entry.getKey().getRootName()))) - .findFirst() - .map(Map.Entry::getValue); - } - private static Map getInfoColumnConstraints(TupleDomain domainPredicate, Map predicateColumns) { checkArgument(!domainPredicate.isNone(), "Unexpected domain predicate: none"); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ManifestPartitionLoader.java b/presto-hive/src/main/java/com/facebook/presto/hive/ManifestPartitionLoader.java index 67a1e24cc84a7..de0f86e0ec043 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ManifestPartitionLoader.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ManifestPartitionLoader.java @@ -67,7 +67,7 @@ public class ManifestPartitionLoader private static final String[] BLOCK_LOCATION_HOSTS = {"localhost"}; private final Table table; - Map infoColumnConstraints; + private final Map infoColumnConstraints; private final ConnectorSession session; private final HdfsEnvironment hdfsEnvironment; private final HdfsContext hdfsContext; @@ -87,7 +87,7 @@ public ManifestPartitionLoader( boolean schedulerUsesHostAddresses) { this.table = requireNonNull(table, "table is null"); - this.infoColumnConstraints = requireNonNull(infoColumnConstraints, "pathDomain is null"); + this.infoColumnConstraints = requireNonNull(infoColumnConstraints, "infoColumnConstraints is null"); this.session = requireNonNull(session, "session is null"); this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); this.hdfsContext = new HdfsContext(session, table.getDatabaseName(), table.getTableName(), table.getStorage().getLocation(), false); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/StoragePartitionLoader.java b/presto-hive/src/main/java/com/facebook/presto/hive/StoragePartitionLoader.java index 3800fb9a2fdfc..676a68f2757e4 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/StoragePartitionLoader.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/StoragePartitionLoader.java @@ -100,7 +100,7 @@ public class StoragePartitionLoader private static final ListenableFuture COMPLETED_FUTURE = immediateFuture(null); private final Table table; - Map infoColumnConstraints; + private final Map infoColumnConstraints; private final Optional tableBucketInfo; private final HdfsEnvironment hdfsEnvironment; private final HdfsContext hdfsContext; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/util/InternalHiveSplitFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/util/InternalHiveSplitFactory.java index 2568b956e4049..41a7fc01fc988 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/util/InternalHiveSplitFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/util/InternalHiveSplitFactory.java @@ -248,15 +248,6 @@ private static List getHostAddresses(BlockLocation blockLocation) .collect(toImmutableList()); } - private static boolean pathMatchesPredicate(Optional pathDomain, String path) - { - if (!pathDomain.isPresent()) { - return true; - } - - return pathDomain.get().includesNullableValue(utf8Slice(path)); - } - private static boolean infoColumnsMatchPredicates(Map constraints, String path, long fileSize, diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeGeneralQueries.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeGeneralQueries.java index aa2a398dec305..64ab28cad8e92 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeGeneralQueries.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeGeneralQueries.java @@ -1051,11 +1051,10 @@ public void testFileSizeHiddenColumn() // A bug used to return all rows even though filters on hidden column were present in the query. // So checking the count here to verify the number of rows returned is correct. Since the bug was present // for both Java and Native Presto for non-$path columns, the assertQuery test above used to pass. - // TODO(spershin): Figure why this test fails in internal environment, fix and then enable it back. - // assertEquals( - // (Long) computeActual(format("SELECT count(*) from test_hidden_columns WHERE \"$file_size\"=%d", fileSize)) - // .getOnlyValue(), - // 1L); + assertEquals( + (Long) computeActual(format("SELECT count(*) from test_hidden_columns WHERE \"$file_size\"=%d", fileSize)) + .getOnlyValue(), + 1L); } @Test @@ -1066,12 +1065,11 @@ public void testFileModifiedTimeHiddenColumn() // Fetch one of the file modified times and use it as a filter. Long fileModifiedTime = (Long) computeActual("SELECT \"$file_modified_time\" from test_hidden_columns LIMIT 1").getOnlyValue(); assertQuery(format("SELECT * from test_hidden_columns WHERE \"$file_modified_time\"=%d", fileModifiedTime)); - // TODO(spershin): Figure why this test fails in internal environment, fix and then enable it back. - // assertEquals( - // (Long) computeActual( - // format("SELECT count(*) from " + - // "test_hidden_columns WHERE \"$file_modified_time\"=%d", fileModifiedTime)).getOnlyValue(), - // 1L); + assertEquals( + (Long) computeActual( + format("SELECT count(*) from " + + "test_hidden_columns WHERE \"$file_modified_time\"=%d", fileModifiedTime)).getOnlyValue(), + 1L); } @Test From b4eca42ff679282b58f09d6e5e688cdecad991a0 Mon Sep 17 00:00:00 2001 From: Jialiang Tan Date: Tue, 6 Aug 2024 10:55:33 -0700 Subject: [PATCH 11/31] [native] Add new arbitration configs --- .../presto_cpp/main/common/Configs.cpp | 36 +++++- .../presto_cpp/main/common/Configs.h | 108 +++++++++++------- 2 files changed, 97 insertions(+), 47 deletions(-) diff --git a/presto-native-execution/presto_cpp/main/common/Configs.cpp b/presto-native-execution/presto_cpp/main/common/Configs.cpp index 558454ab1946a..e815598ee4baf 100644 --- a/presto-native-execution/presto_cpp/main/common/Configs.cpp +++ b/presto-native-execution/presto_cpp/main/common/Configs.cpp @@ -186,8 +186,12 @@ SystemConfig::SystemConfig() { BOOL_PROP(kUseMmapAllocator, true), STR_PROP(kMemoryArbitratorKind, ""), BOOL_PROP(kMemoryArbitratorGlobalArbitrationEnabled, false), + BOOL_PROP(kSharedArbitratorGlobalArbitrationEnabled, false), NUM_PROP(kQueryMemoryGb, 38), NUM_PROP(kQueryReservedMemoryGb, 4), + STR_PROP(kSharedArbitratorReservedCapacity, "4GB"), + STR_PROP(kSharedArbitratorMemoryPoolReservedCapacity, "0B"), + STR_PROP(kSharedArbitratorMemoryPoolTransferCapacity, "128MB"), NUM_PROP(kLargestSizeClassPages, 256), BOOL_PROP(kEnableVeloxTaskLogging, false), BOOL_PROP(kEnableVeloxExprSetLogging, false), @@ -507,6 +511,11 @@ bool SystemConfig::memoryArbitratorGlobalArbitrationEnabled() const { .value_or(false); } +bool SystemConfig::sharedArbitratorGlobalArbitrationEnabled() const { + return optionalProperty(kSharedArbitratorGlobalArbitrationEnabled) + .value_or(false); +} + int32_t SystemConfig::queryMemoryGb() const { return optionalProperty(kQueryMemoryGb).value(); } @@ -515,6 +524,10 @@ int32_t SystemConfig::queryReservedMemoryGb() const { return optionalProperty(kQueryReservedMemoryGb).value(); } +int32_t SystemConfig::sharedArbitratorReservedCapacity() const { + return optionalProperty(kSharedArbitratorReservedCapacity).value(); +} + uint64_t SystemConfig::memoryPoolInitCapacity() const { static constexpr uint64_t kMemoryPoolInitCapacityDefault = 128 << 20; return optionalProperty(kMemoryPoolInitCapacity) @@ -527,16 +540,31 @@ uint64_t SystemConfig::memoryPoolReservedCapacity() const { .value_or(kMemoryPoolReservedCapacityDefault); } +uint64_t SystemConfig::sharedArbitratorMemoryPoolReservedCapacity() const { + static constexpr uint64_t kSharedArbitratorMemoryPoolReservedCapacityDefault = + 64 << 20; + return optionalProperty(kSharedArbitratorMemoryPoolReservedCapacity) + .value_or(kSharedArbitratorMemoryPoolReservedCapacityDefault); +} + uint64_t SystemConfig::memoryPoolTransferCapacity() const { static constexpr uint64_t kMemoryPoolTransferCapacityDefault = 32 << 20; return optionalProperty(kMemoryPoolTransferCapacity) .value_or(kMemoryPoolTransferCapacityDefault); } -uint64_t SystemConfig::memoryReclaimWaitMs() const { - static constexpr uint64_t kMemoryReclaimWaitMsDefault = {300'000}; // 5 mins. - return optionalProperty(kMemoryReclaimWaitMs) - .value_or(kMemoryReclaimWaitMsDefault); +uint64_t SystemConfig::sharedArbitratorMemoryPoolTransferCapacity() const { + static constexpr uint64_t kSharedArbitratorMemoryPoolTransferCapacityDefault = + 32 << 20; + return optionalProperty(kSharedArbitratorMemoryPoolTransferCapacity) + .value_or(kSharedArbitratorMemoryPoolTransferCapacityDefault); +} + +uint64_t SystemConfig::sharedArbitratorMemoryReclaimWaitMs() const { + static constexpr uint64_t kSharedArbitratorMemoryReclaimWaitMsDefault = { + 300'000}; // 5 mins. + return optionalProperty(kSharedArbitratorMemoryReclaimWaitMs) + .value_or(kSharedArbitratorMemoryReclaimWaitMsDefault); } bool SystemConfig::enableSystemMemoryPoolUsageTracking() const { diff --git a/presto-native-execution/presto_cpp/main/common/Configs.h b/presto-native-execution/presto_cpp/main/common/Configs.h index 1760a52754322..bb68a03378feb 100644 --- a/presto-native-execution/presto_cpp/main/common/Configs.h +++ b/presto-native-execution/presto_cpp/main/common/Configs.h @@ -23,7 +23,7 @@ namespace facebook::presto { class ConfigBase { public: - // Setting this to 'true' makes configs modifiable via server operations. + /// Setting this to 'true' makes configs modifiable via server operations. static constexpr std::string_view kMutableConfig{"mutable-config"}; /// Reads configuration properties from the specified file. Must be called @@ -257,27 +257,6 @@ class SystemConfig : public ConfigBase { /// Indicates if the process is configured as a sidecar. static constexpr std::string_view kNativeSidecar{"native-sidecar"}; - /// Specifies the total amount of memory in GB that the queries can use on a - /// single worker node. It should be configured to be less than the total - /// system memory capacity ('system-memory-gb') such that there is enough room - /// left for the system (as opposed to for the queries), such as disk spilling - /// and cache prefetch. - /// - /// NOTE: the query memory capacity is enforced by memory arbitrator so that - /// this config only applies if the memory arbitration has been enabled. - static constexpr std::string_view kQueryMemoryGb{"query-memory-gb"}; - - /// Specifies the total amount of memory in GB reserved for the queries on - /// a single worker node. A query can only allocate from this reserved space - /// if 1) the non-reserved space in "query-memory-gb" is used up; and 2) the - /// amount it tries to get is less than 'memory-pool-reserved-capacity'. - /// - /// NOTE: the reserved query memory capacity is enforced by memory arbitrator - /// so that this config only applies if the memory arbitration has been - /// enabled. - static constexpr std::string_view kQueryReservedMemoryGb{ - "query-reserved-memory-gb"}; - /// If true, enable memory pushback when the server is under low memory /// condition. This only applies if 'system-mem-limit-gb' is set. static constexpr std::string_view kSystemMemPushbackEnabled{ @@ -384,21 +363,42 @@ class SystemConfig : public ConfigBase { static constexpr std::string_view kEnableRuntimeMetricsCollection{ "runtime-metrics-collection-enabled"}; + /// Specifies the total amount of memory in GB that the queries can use on a + /// single worker node. It should be configured to be less than the total + /// system memory capacity ('system-memory-gb') such that there is enough room + /// left for the system (as opposed to for the queries), such as disk spilling + /// and cache prefetch. + /// + /// NOTE: the query memory capacity is enforced by memory arbitrator so that + /// this config only applies if the memory arbitration has been enabled. + static constexpr std::string_view kQueryMemoryGb{"query-memory-gb"}; + static constexpr std::string_view kArbitratorCapacity{ + "arbitrator-capacity"}; + /// Specifies the memory arbitrator kind. If it is empty, then there is no /// memory arbitration. static constexpr std::string_view kMemoryArbitratorKind{ "memory-arbitrator-kind"}; + /// Specifies the total amount of memory in GB reserved for the queries on + /// a single worker node. A query can only allocate from this reserved space + /// if 1) the non-reserved space in "query-memory-gb" is used up; and 2) the + /// amount it tries to get is less than 'memory-pool-reserved-capacity'. + /// + /// NOTE: the reserved query memory capacity is enforced by memory arbitrator + /// so that this config only applies if the memory arbitration has been + /// enabled. + static constexpr std::string_view kQueryReservedMemoryGb{ + "query-reserved-memory-gb"}; + static constexpr std::string_view kSharedArbitratorReservedCapacity{ + "shared-arbitrator.reserved-capacity"}; + /// If true, it allows memory arbitrator to reclaim used memory cross query /// memory pools. static constexpr std::string_view kMemoryArbitratorGlobalArbitrationEnabled{ "memory-arbitrator-global-arbitration-enabled"}; - - /// The initial memory pool capacity in bytes allocated on creation. - /// - /// NOTE: this config only applies if the memory arbitration has been enabled. - static constexpr std::string_view kMemoryPoolInitCapacity{ - "memory-pool-init-capacity"}; + static constexpr std::string_view kSharedArbitratorGlobalArbitrationEnabled{ + "shared-arbitrator.global-arbitration-enabled"}; /// The amount of memory in bytes reserved for each query memory pool. When /// a query tries to allocate memory from the reserved space whose size is @@ -406,6 +406,8 @@ class SystemConfig : public ConfigBase { /// value specified in 'memory-pool-reserved-capacity'. static constexpr std::string_view kMemoryPoolReservedCapacity{ "memory-pool-reserved-capacity"}; + static constexpr std::string_view kSharedArbitratorMemoryPoolReservedCapacity{ + "shared-arbitrator.memory-pool-reserved-capacity"}; /// The minimal memory capacity in bytes transferred between memory pools /// during memory arbitration. @@ -413,6 +415,8 @@ class SystemConfig : public ConfigBase { /// NOTE: this config only applies if the memory arbitration has been enabled. static constexpr std::string_view kMemoryPoolTransferCapacity{ "memory-pool-transfer-capacity"}; + static constexpr std::string_view kSharedArbitratorMemoryPoolTransferCapacity{ + "shared-arbitrator.memory-pool-transfer-capacity"}; /// Specifies the max time to wait for memory reclaim by arbitration. The /// memory reclaim might fail if the max wait time has exceeded. If it is @@ -421,6 +425,14 @@ class SystemConfig : public ConfigBase { /// NOTE: this config only applies if the memory arbitration has been enabled. static constexpr std::string_view kMemoryReclaimWaitMs{ "memory-reclaim-wait-ms"}; + static constexpr std::string_view kSharedArbitratorMemoryReclaimWaitMs{ + "shared-arbitrator.memory-reclaim-wait-ms"}; + + /// The initial memory pool capacity in bytes allocated on creation. + /// + /// NOTE: this config only applies if the memory arbitration has been enabled. + static constexpr std::string_view kMemoryPoolInitCapacity{ + "memory-pool-init-capacity"}; /// Enables the memory usage tracking for the system memory pool used for /// cases such as disk spilling. @@ -601,27 +613,27 @@ class SystemConfig : public ConfigBase { int httpServerHttpsPort() const; - // A list of ciphers (comma separated) that are supported by - // server and client. Note Java and folly::SSLContext use different names to - // refer to the same cipher. For e.g. TLS_RSA_WITH_AES_256_GCM_SHA384 in Java - // and AES256-GCM-SHA384 in folly::SSLContext. More details can be found here: - // https://www.openssl.org/docs/manmaster/man1/openssl-ciphers.html. The - // ciphers enable worker to worker, worker to coordinator and - // coordinator to worker communication. At least one cipher needs to be - // shared for the above 3 communication to work. + /// A list of ciphers (comma separated) that are supported by + /// server and client. Note Java and folly::SSLContext use different names to + /// refer to the same cipher. For e.g. TLS_RSA_WITH_AES_256_GCM_SHA384 in Java + /// and AES256-GCM-SHA384 in folly::SSLContext. More details can be found here: + /// https://www.openssl.org/docs/manmaster/man1/openssl-ciphers.html. The + /// ciphers enable worker to worker, worker to coordinator and + /// coordinator to worker communication. At least one cipher needs to be + /// shared for the above 3 communication to work. std::string httpsSupportedCiphers() const; - // Note: Java packages cert and key in combined JKS file. But CPP requires - // them separately. The HTTPS provides integrity and not - // security(authentication/authorization). But the HTTPS will protect against - // data corruption by bad router and man in middle attacks. + /// Note: Java packages cert and key in combined JKS file. But CPP requires + /// them separately. The HTTPS provides integrity and not + /// security(authentication/authorization). But the HTTPS will protect against + /// data corruption by bad router and man in middle attacks. folly::Optional httpsCertPath() const; folly::Optional httpsKeyPath() const; - // Http client expects the cert and key file to be packed into a single file - // (most commonly .pem format) The file should not be password protected. If - // required, break this down to 3 configs one for cert,key and password later. + /// Http client expects the cert and key file to be packed into a single file + /// (most commonly .pem format) The file should not be password protected. If + /// required, break this down to 3 configs one for cert,key and password later. folly::Optional httpsClientCertAndKeyPath() const; bool mutableConfig() const; @@ -723,6 +735,16 @@ class SystemConfig : public ConfigBase { bool memoryArbitratorGlobalArbitrationEnabled() const; + bool sharedArbitratorGlobalArbitrationEnabled() const; + + int32_t sharedArbitratorReservedCapacity() const; + + uint64_t sharedArbitratorMemoryPoolReservedCapacity() const; + + uint64_t sharedArbitratorMemoryPoolTransferCapacity() const; + + uint64_t sharedArbitratorMemoryReclaimWaitMs() const; + int32_t queryMemoryGb() const; int32_t queryReservedMemoryGb() const; From ae870326df4abda6dec73434014477f14594cd3e Mon Sep 17 00:00:00 2001 From: Jialiang Tan Date: Tue, 6 Aug 2024 15:51:07 -0700 Subject: [PATCH 12/31] [native] Add new shared arbitrator configs --- .../presto_cpp/main/PrestoServer.cpp | 10 +- .../presto_cpp/main/common/Configs.cpp | 103 ++++++++++++++---- .../presto_cpp/main/common/Configs.h | 94 ++++++++++++---- 3 files changed, 160 insertions(+), 47 deletions(-) diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.cpp b/presto-native-execution/presto_cpp/main/PrestoServer.cpp index e50caeaac3920..21384cf218b52 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.cpp +++ b/presto-native-execution/presto_cpp/main/PrestoServer.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include "CoordinatorDiscoverer.h" #include "presto_cpp/main/Announcer.h" @@ -36,7 +35,6 @@ #include "presto_cpp/main/operators/PartitionAndSerialize.h" #include "presto_cpp/main/operators/ShuffleRead.h" #include "presto_cpp/main/operators/UnsafeRowExchangeSource.h" -#include "presto_cpp/main/types/PrestoToVeloxConnector.h" #include "presto_cpp/main/types/PrestoToVeloxQueryPlan.h" #include "velox/common/base/Counters.h" #include "velox/common/base/StatsReporter.h" @@ -801,12 +799,16 @@ void PrestoServer::initializeVeloxMemory() { memoryGb, "Query memory capacity must not be larger than system memory capacity"); options.arbitratorCapacity = queryMemoryGb << 30; - const uint64_t queryReservedMemoryGb = - systemConfig->queryReservedMemoryGb(); + const uint64_t queryReservedMemoryGb = velox::config::toCapacity( + systemConfig->sharedArbitratorReservedCapacity(), + velox::config::CapacityUnit::GIGABYTE); VELOX_USER_CHECK_LE( queryReservedMemoryGb, queryMemoryGb, "Query reserved memory capacity must not be larger than query memory capacity"); + + // TODO(jtan6): [Config Refactor] Migrate these old settings to string based + // extra settings + grow & shrink settings. options.arbitratorReservedCapacity = queryReservedMemoryGb << 30; options.memoryPoolInitCapacity = systemConfig->memoryPoolInitCapacity(); options.memoryPoolReservedCapacity = diff --git a/presto-native-execution/presto_cpp/main/common/Configs.cpp b/presto-native-execution/presto_cpp/main/common/Configs.cpp index e815598ee4baf..fc6b4e366eca4 100644 --- a/presto-native-execution/presto_cpp/main/common/Configs.cpp +++ b/presto-native-execution/presto_cpp/main/common/Configs.cpp @@ -186,12 +186,14 @@ SystemConfig::SystemConfig() { BOOL_PROP(kUseMmapAllocator, true), STR_PROP(kMemoryArbitratorKind, ""), BOOL_PROP(kMemoryArbitratorGlobalArbitrationEnabled, false), - BOOL_PROP(kSharedArbitratorGlobalArbitrationEnabled, false), NUM_PROP(kQueryMemoryGb, 38), NUM_PROP(kQueryReservedMemoryGb, 4), STR_PROP(kSharedArbitratorReservedCapacity, "4GB"), - STR_PROP(kSharedArbitratorMemoryPoolReservedCapacity, "0B"), - STR_PROP(kSharedArbitratorMemoryPoolTransferCapacity, "128MB"), + STR_PROP(kSharedArbitratorMemoryPoolInitialCapacity, "128MB"), + STR_PROP(kSharedArbitratorMemoryPoolReservedCapacity, "64MB"), + STR_PROP(kSharedArbitratorMemoryPoolTransferCapacity, "32MB"), + STR_PROP(kSharedArbitratorMemoryReclaimMaxWaitTime, "5m"), + STR_PROP(kSharedArbitratorGlobalArbitrationEnabled, "false"), NUM_PROP(kLargestSizeClassPages, 256), BOOL_PROP(kEnableVeloxTaskLogging, false), BOOL_PROP(kEnableVeloxExprSetLogging, false), @@ -511,9 +513,10 @@ bool SystemConfig::memoryArbitratorGlobalArbitrationEnabled() const { .value_or(false); } -bool SystemConfig::sharedArbitratorGlobalArbitrationEnabled() const { - return optionalProperty(kSharedArbitratorGlobalArbitrationEnabled) - .value_or(false); +std::string SystemConfig::sharedArbitratorGlobalArbitrationEnabled() const { + return optionalProperty( + kSharedArbitratorGlobalArbitrationEnabled) + .value_or("false"); } int32_t SystemConfig::queryMemoryGb() const { @@ -524,8 +527,9 @@ int32_t SystemConfig::queryReservedMemoryGb() const { return optionalProperty(kQueryReservedMemoryGb).value(); } -int32_t SystemConfig::sharedArbitratorReservedCapacity() const { - return optionalProperty(kSharedArbitratorReservedCapacity).value(); +std::string SystemConfig::sharedArbitratorReservedCapacity() const { + return optionalProperty(kSharedArbitratorReservedCapacity) + .value(); } uint64_t SystemConfig::memoryPoolInitCapacity() const { @@ -534,17 +538,27 @@ uint64_t SystemConfig::memoryPoolInitCapacity() const { .value_or(kMemoryPoolInitCapacityDefault); } +std::string SystemConfig::sharedArbitratorMemoryPoolInitialCapacity() const { + static constexpr std::string_view + kSharedArbitratorMemoryPoolInitialCapacityDefault = "128MB"; + return optionalProperty( + kSharedArbitratorMemoryPoolInitialCapacity) + .value_or(std::string(kSharedArbitratorMemoryPoolInitialCapacityDefault)); +} + uint64_t SystemConfig::memoryPoolReservedCapacity() const { static constexpr uint64_t kMemoryPoolReservedCapacityDefault = 64 << 20; return optionalProperty(kMemoryPoolReservedCapacity) .value_or(kMemoryPoolReservedCapacityDefault); } -uint64_t SystemConfig::sharedArbitratorMemoryPoolReservedCapacity() const { - static constexpr uint64_t kSharedArbitratorMemoryPoolReservedCapacityDefault = - 64 << 20; - return optionalProperty(kSharedArbitratorMemoryPoolReservedCapacity) - .value_or(kSharedArbitratorMemoryPoolReservedCapacityDefault); +std::string SystemConfig::sharedArbitratorMemoryPoolReservedCapacity() const { + static constexpr std::string_view + kSharedArbitratorMemoryPoolReservedCapacityDefault = "64MB"; + return optionalProperty( + kSharedArbitratorMemoryPoolReservedCapacity) + .value_or( + std::string(kSharedArbitratorMemoryPoolReservedCapacityDefault)); } uint64_t SystemConfig::memoryPoolTransferCapacity() const { @@ -553,18 +567,61 @@ uint64_t SystemConfig::memoryPoolTransferCapacity() const { .value_or(kMemoryPoolTransferCapacityDefault); } -uint64_t SystemConfig::sharedArbitratorMemoryPoolTransferCapacity() const { - static constexpr uint64_t kSharedArbitratorMemoryPoolTransferCapacityDefault = - 32 << 20; - return optionalProperty(kSharedArbitratorMemoryPoolTransferCapacity) - .value_or(kSharedArbitratorMemoryPoolTransferCapacityDefault); +std::string SystemConfig::sharedArbitratorMemoryPoolTransferCapacity() const { + static constexpr std::string_view + kSharedArbitratorMemoryPoolTransferCapacityDefault = "32MB"; + return optionalProperty( + kSharedArbitratorMemoryPoolTransferCapacity) + .value_or( + std::string(kSharedArbitratorMemoryPoolTransferCapacityDefault)); } -uint64_t SystemConfig::sharedArbitratorMemoryReclaimWaitMs() const { - static constexpr uint64_t kSharedArbitratorMemoryReclaimWaitMsDefault = { - 300'000}; // 5 mins. - return optionalProperty(kSharedArbitratorMemoryReclaimWaitMs) - .value_or(kSharedArbitratorMemoryReclaimWaitMsDefault); +uint64_t SystemConfig::memoryReclaimWaitMs() const { + static constexpr uint64_t kMemoryReclaimWaitMsDefault = {300'000}; // 5 mins. + return optionalProperty(kMemoryReclaimWaitMs) + .value_or(kMemoryReclaimWaitMsDefault); +} + +std::string SystemConfig::sharedArbitratorMemoryReclaimWaitTime() const { + static constexpr std::string_view + kSharedArbitratorMemoryReclaimMaxWaitTimeDefault = "5m"; + return optionalProperty( + kSharedArbitratorMemoryReclaimMaxWaitTime) + .value_or(std::string(kSharedArbitratorMemoryReclaimMaxWaitTimeDefault)); +} + +std::string SystemConfig::sharedArbitratorFastExponentialGrowthCapacityLimit() + const { + static constexpr std::string_view + kSharedArbitratorFastExponentialGrowthCapacityLimitDefault = "512MB"; + return optionalProperty( + kSharedArbitratorFastExponentialGrowthCapacityLimit) + .value_or(std::string( + kSharedArbitratorFastExponentialGrowthCapacityLimitDefault)); +} + +std::string SystemConfig::sharedArbitratorSlowCapacityGrowPct() const { + static constexpr std::string_view + kSharedArbitratorSlowCapacityGrowPctDefault = "0.25"; + return optionalProperty(kSharedArbitratorSlowCapacityGrowPct) + .value_or(std::string(kSharedArbitratorSlowCapacityGrowPctDefault)); +} + +std::string SystemConfig::sharedArbitratorMemoryPoolMinFreeCapacity() const { + static constexpr std::string_view + kSharedArbitratorMemoryPoolMinFreeCapacityDefault = "128MB"; + return optionalProperty( + kSharedArbitratorMemoryPoolMinFreeCapacity) + .value_or(std::string(kSharedArbitratorMemoryPoolMinFreeCapacityDefault)); +} + +std::string SystemConfig::sharedArbitratorMemoryPoolMinFreeCapacityPct() const { + static constexpr std::string_view + kSharedArbitratorMemoryPoolMinFreeCapacityPctDefault = "0.25"; + return optionalProperty( + kSharedArbitratorMemoryPoolMinFreeCapacityPctDefault) + .value_or( + std::string(kSharedArbitratorMemoryPoolMinFreeCapacityPctDefault)); } bool SystemConfig::enableSystemMemoryPoolUsageTracking() const { diff --git a/presto-native-execution/presto_cpp/main/common/Configs.h b/presto-native-execution/presto_cpp/main/common/Configs.h index bb68a03378feb..74fa387ef6d1e 100644 --- a/presto-native-execution/presto_cpp/main/common/Configs.h +++ b/presto-native-execution/presto_cpp/main/common/Configs.h @@ -372,8 +372,7 @@ class SystemConfig : public ConfigBase { /// NOTE: the query memory capacity is enforced by memory arbitrator so that /// this config only applies if the memory arbitration has been enabled. static constexpr std::string_view kQueryMemoryGb{"query-memory-gb"}; - static constexpr std::string_view kArbitratorCapacity{ - "arbitrator-capacity"}; + static constexpr std::string_view kArbitratorCapacity{"arbitrator-capacity"}; /// Specifies the memory arbitrator kind. If it is empty, then there is no /// memory arbitration. @@ -393,6 +392,12 @@ class SystemConfig : public ConfigBase { static constexpr std::string_view kSharedArbitratorReservedCapacity{ "shared-arbitrator.reserved-capacity"}; + /// The initial memory pool capacity in bytes allocated on creation. + static constexpr std::string_view kMemoryPoolInitCapacity{ + "memory-pool-init-capacity"}; + static constexpr std::string_view kSharedArbitratorMemoryPoolInitialCapacity{ + "shared-arbitrator.memory-pool-initial-capacity"}; + /// If true, it allows memory arbitrator to reclaim used memory cross query /// memory pools. static constexpr std::string_view kMemoryArbitratorGlobalArbitrationEnabled{ @@ -425,14 +430,52 @@ class SystemConfig : public ConfigBase { /// NOTE: this config only applies if the memory arbitration has been enabled. static constexpr std::string_view kMemoryReclaimWaitMs{ "memory-reclaim-wait-ms"}; - static constexpr std::string_view kSharedArbitratorMemoryReclaimWaitMs{ - "shared-arbitrator.memory-reclaim-wait-ms"}; - - /// The initial memory pool capacity in bytes allocated on creation. + static constexpr std::string_view kSharedArbitratorMemoryReclaimMaxWaitTime{ + "shared-arbitrator.memory-reclaim-max-wait-time"}; + + /// When shared arbitrator grows memory pool's capacity, the growth bytes will + /// be adjusted in the following way: + /// - If 2 * current capacity is less than or equal to + /// 'fastExponentialGrowthCapacityLimit', grow through fast path by at + /// least doubling the current capacity, when conditions allow (see below + /// NOTE section). + /// - If 2 * current capacity is greater than + /// 'fastExponentialGrowthCapacityLimit', grow through slow path by growing + /// capacity by at least 'slowCapacityGrowPct' * current capacity if + /// allowed (see below NOTE section). /// - /// NOTE: this config only applies if the memory arbitration has been enabled. - static constexpr std::string_view kMemoryPoolInitCapacity{ - "memory-pool-init-capacity"}; + /// NOTE: If original requested growth bytes is larger than the adjusted + /// growth bytes or adjusted growth bytes reaches max capacity limit, the + /// adjusted growth bytes will not be respected. + /// + /// NOTE: Capacity growth adjust is only enabled if both + /// 'fastExponentialGrowthCapacityLimit' and 'slowCapacityGrowPct' are set, + /// otherwise it is disabled. + static constexpr std::string_view + kSharedArbitratorFastExponentialGrowthCapacityLimit{ + "shared-arbitrator.fast-exponential-growth-capacity-limit"}; + static constexpr std::string_view kSharedArbitratorSlowCapacityGrowPct{ + "shared-arbitrator.slow-capacity-grow-pct"}; + + /// When shared arbitrator shrinks memory pool's capacity, the shrink bytes + /// will be adjusted in a way such that AFTER shrink, the stricter (whichever + /// is smaller) of the following conditions is met, in order to better fit the + /// pool's current memory usage: + /// - Free capacity is greater or equal to capacity * + /// 'memoryPoolMinFreeCapacityPct' + /// - Free capacity is greater or equal to 'memoryPoolMinFreeCapacity' + /// + /// NOTE: In the conditions when original requested shrink bytes ends up + /// with more free capacity than above 2 conditions, the adjusted shrink + /// bytes is not respected. + /// + /// NOTE: Capacity shrink adjustment is enabled when both + /// 'memoryPoolMinFreeCapacityPct' and 'memoryPoolMinFreeCapacity' are set. + static constexpr std::string_view kSharedArbitratorMemoryPoolMinFreeCapacity{ + "shared-arbitrator.memory-pool-min-free-capacity"}; + static constexpr std::string_view + kSharedArbitratorMemoryPoolMinFreeCapacityPct{ + "shared-arbitrator.memory-pool-min-free-capacity-pct"}; /// Enables the memory usage tracking for the system memory pool used for /// cases such as disk spilling. @@ -616,11 +659,11 @@ class SystemConfig : public ConfigBase { /// A list of ciphers (comma separated) that are supported by /// server and client. Note Java and folly::SSLContext use different names to /// refer to the same cipher. For e.g. TLS_RSA_WITH_AES_256_GCM_SHA384 in Java - /// and AES256-GCM-SHA384 in folly::SSLContext. More details can be found here: - /// https://www.openssl.org/docs/manmaster/man1/openssl-ciphers.html. The - /// ciphers enable worker to worker, worker to coordinator and - /// coordinator to worker communication. At least one cipher needs to be - /// shared for the above 3 communication to work. + /// and AES256-GCM-SHA384 in folly::SSLContext. More details can be found + /// here: https://www.openssl.org/docs/manmaster/man1/openssl-ciphers.html. + /// The ciphers enable worker to worker, worker to coordinator and coordinator + /// to worker communication. At least one cipher needs to be shared for the + /// above 3 communication to work. std::string httpsSupportedCiphers() const; /// Note: Java packages cert and key in combined JKS file. But CPP requires @@ -633,7 +676,8 @@ class SystemConfig : public ConfigBase { /// Http client expects the cert and key file to be packed into a single file /// (most commonly .pem format) The file should not be password protected. If - /// required, break this down to 3 configs one for cert,key and password later. + /// required, break this down to 3 configs one for cert,key and password + /// later. folly::Optional httpsClientCertAndKeyPath() const; bool mutableConfig() const; @@ -735,15 +779,25 @@ class SystemConfig : public ConfigBase { bool memoryArbitratorGlobalArbitrationEnabled() const; - bool sharedArbitratorGlobalArbitrationEnabled() const; + std::string sharedArbitratorGlobalArbitrationEnabled() const; + + std::string sharedArbitratorReservedCapacity() const; + + std::string sharedArbitratorMemoryPoolReservedCapacity() const; + + std::string sharedArbitratorMemoryPoolTransferCapacity() const; + + std::string sharedArbitratorMemoryReclaimWaitTime() const; + + std::string sharedArbitratorMemoryPoolInitialCapacity() const; - int32_t sharedArbitratorReservedCapacity() const; + std::string sharedArbitratorFastExponentialGrowthCapacityLimit() const; - uint64_t sharedArbitratorMemoryPoolReservedCapacity() const; + std::string sharedArbitratorSlowCapacityGrowPct() const; - uint64_t sharedArbitratorMemoryPoolTransferCapacity() const; + std::string sharedArbitratorMemoryPoolMinFreeCapacity() const; - uint64_t sharedArbitratorMemoryReclaimWaitMs() const; + std::string sharedArbitratorMemoryPoolMinFreeCapacityPct() const; int32_t queryMemoryGb() const; From a707af8d11c917d5bc1b001b386ee27bcaba40b3 Mon Sep 17 00:00:00 2001 From: zuyu Date: Sat, 17 Aug 2024 20:39:26 +0800 Subject: [PATCH 13/31] Fix a build link in README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 10932804c8367..20fcc4ca4a2ed 100644 --- a/README.md +++ b/README.md @@ -122,7 +122,7 @@ resources will be hot-reloaded and changes are reflected on browser refresh. [Velox](https://github.com/facebookincubator/velox) is a C++ database library which provides reusable, extensible, and high-performance data processing components. -Check out [building instructions](https://github.com/prestodb/presto/tree/master/presto-native-execution#building) to get started. +Check out [building instructions](https://github.com/prestodb/presto/tree/master/presto-native-execution#build-from-source) to get started.
From c4e1ab3d85619fd6bfbb5903467f6a1af9769431 Mon Sep 17 00:00:00 2001 From: Amit Dutta Date: Sun, 18 Aug 2024 09:17:23 -0700 Subject: [PATCH 14/31] [native] Advance velox. --- presto-native-execution/velox | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-native-execution/velox b/presto-native-execution/velox index 9523840ac329b..54b16e0416ada 160000 --- a/presto-native-execution/velox +++ b/presto-native-execution/velox @@ -1 +1 @@ -Subproject commit 9523840ac329be4f7318338fb92cb92bcb0a8c07 +Subproject commit 54b16e0416ada85dd2904b2d2bee7650a1d27b36 From 56223ebbe21fc60d5aa4cbffbaa5050cb3ff0a24 Mon Sep 17 00:00:00 2001 From: Zac Blanco Date: Thu, 18 Jul 2024 17:02:34 -0700 Subject: [PATCH 15/31] Aggressively prune statistics from query info after completion Previously, we only pruned information from old queries after the expiration queue filled. With more the addition of histograms query statistics can take two orders of magnitude more memory than previous statistics. Without pruning statistics we will much more quickly hit memory limits of the JVM. This change aggressively prunes query plan metadata before query expiry. This is mostly limited to query plan statistics. --- .../dispatcher/FailedDispatchQuery.java | 7 +- .../presto/dispatcher/LocalDispatchQuery.java | 10 +- .../AccessControlCheckerExecution.java | 8 +- .../execution/DataDefinitionExecution.java | 9 +- .../presto/execution/QueryStateMachine.java | 166 ++++++++++++- .../presto/execution/QueryTracker.java | 24 +- .../presto/execution/SqlQueryExecution.java | 17 +- .../presto/execution/StateMachine.java | 5 + .../presto/execution/MockQueryExecution.java | 6 +- .../spi/statistics/ColumnStatistics.java | 10 + .../spi/statistics/TableStatistics.java | 16 ++ .../presto/execution/TestEventListener.java | 2 +- .../execution/TestEventListenerPlugin.java | 6 +- .../presto/tests/TestQueryManager.java | 230 ++++++++++++++++++ 14 files changed, 492 insertions(+), 24 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/dispatcher/FailedDispatchQuery.java b/presto-main/src/main/java/com/facebook/presto/dispatcher/FailedDispatchQuery.java index 3c828ea620791..f4e93eed051c8 100644 --- a/presto-main/src/main/java/com/facebook/presto/dispatcher/FailedDispatchQuery.java +++ b/presto-main/src/main/java/com/facebook/presto/dispatcher/FailedDispatchQuery.java @@ -115,7 +115,10 @@ public void fail(Throwable throwable) {} public void cancel() {} @Override - public void pruneInfo() {} + public void pruneExpiredQueryInfo() {} + + @Override + public void pruneFinishedQueryInfo() {} @Override public QueryId getQueryId() @@ -194,5 +197,5 @@ public Optional getResourceGroupQueryLimits() @Override public void setResourceGroupQueryLimits(ResourceGroupQueryLimits resourceGroupQueryLimits) - { } + {} } diff --git a/presto-main/src/main/java/com/facebook/presto/dispatcher/LocalDispatchQuery.java b/presto-main/src/main/java/com/facebook/presto/dispatcher/LocalDispatchQuery.java index 4babf8a95575e..50b7dea3fba32 100644 --- a/presto-main/src/main/java/com/facebook/presto/dispatcher/LocalDispatchQuery.java +++ b/presto-main/src/main/java/com/facebook/presto/dispatcher/LocalDispatchQuery.java @@ -357,9 +357,15 @@ public void cancel() } @Override - public void pruneInfo() + public void pruneExpiredQueryInfo() { - stateMachine.pruneQueryInfo(); + stateMachine.pruneQueryInfoExpired(); + } + + @Override + public void pruneFinishedQueryInfo() + { + stateMachine.pruneQueryInfoFinished(); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/execution/AccessControlCheckerExecution.java b/presto-main/src/main/java/com/facebook/presto/execution/AccessControlCheckerExecution.java index 2fb96b05b2f0f..160759e3f0cf0 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/AccessControlCheckerExecution.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/AccessControlCheckerExecution.java @@ -327,7 +327,13 @@ public void recordHeartbeat() } @Override - public void pruneInfo() + public void pruneExpiredQueryInfo() + { + // no-op + } + + @Override + public void pruneFinishedQueryInfo() { // no-op } diff --git a/presto-main/src/main/java/com/facebook/presto/execution/DataDefinitionExecution.java b/presto-main/src/main/java/com/facebook/presto/execution/DataDefinitionExecution.java index a130b447516f3..cd22e10a5c983 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/DataDefinitionExecution.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/DataDefinitionExecution.java @@ -162,6 +162,7 @@ public DataSize getWrittenIntermediateDataSize() { return DataSize.succinctBytes(0); } + @Override public long getOutputPositions() { @@ -290,7 +291,13 @@ public void recordHeartbeat() } @Override - public void pruneInfo() + public void pruneExpiredQueryInfo() + { + // no-op + } + + @Override + public void pruneFinishedQueryInfo() { // no-op } diff --git a/presto-main/src/main/java/com/facebook/presto/execution/QueryStateMachine.java b/presto-main/src/main/java/com/facebook/presto/execution/QueryStateMachine.java index aeee080928ae4..b22822fc0149e 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/QueryStateMachine.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/QueryStateMachine.java @@ -19,7 +19,9 @@ import com.facebook.presto.common.resourceGroups.QueryType; import com.facebook.presto.common.transaction.TransactionId; import com.facebook.presto.common.type.Type; +import com.facebook.presto.cost.PlanNodeStatsEstimate; import com.facebook.presto.cost.StatsAndCosts; +import com.facebook.presto.cost.VariableStatsEstimate; import com.facebook.presto.execution.QueryExecution.QueryOutputInfo; import com.facebook.presto.execution.StateMachine.StateChangeListener; import com.facebook.presto.memory.VersionedMemoryPoolId; @@ -38,13 +40,17 @@ import com.facebook.presto.spi.resourceGroups.ResourceGroupId; import com.facebook.presto.spi.security.AccessControl; import com.facebook.presto.spi.security.SelectedRole; +import com.facebook.presto.spi.statistics.ColumnStatistics; +import com.facebook.presto.spi.statistics.TableStatistics; import com.facebook.presto.sql.planner.CanonicalPlanWithInfo; +import com.facebook.presto.sql.planner.PlanFragment; import com.facebook.presto.transaction.TransactionInfo; import com.facebook.presto.transaction.TransactionManager; import com.google.common.base.Ticker; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.collect.Streams; import com.google.common.util.concurrent.FutureCallback; @@ -93,6 +99,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; import static io.airlift.units.DataSize.succinctBytes; import static java.lang.String.format; @@ -1046,7 +1054,24 @@ public QueryInfo updateQueryInfo(Optional stageInfo) * Remove large objects from the query info object graph, e.g : plan, stats, stage summaries, failed attempts * Used when pruning expired queries from the state machine */ - public void pruneQueryInfo() + public void pruneQueryInfoExpired() + { + Optional finalInfo = finalQueryInfo.get(); + if (!finalInfo.isPresent() || !finalInfo.get().getOutputStage().isPresent()) { + return; + } + QueryInfo queryInfo = finalInfo.get(); + QueryInfo prunedQueryInfo; + + prunedQueryInfo = pruneExpiredQueryInfo(queryInfo, getMemoryPool()); + finalQueryInfo.compareAndSet(finalInfo, Optional.of(prunedQueryInfo)); + } + + /** + * Remove the largest objects from the query info object graph, e.g : extraneous stats, costs, + * and histograms to reduce memory utilization + */ + public void pruneQueryInfoFinished() { Optional finalInfo = finalQueryInfo.get(); if (!finalInfo.isPresent() || !finalInfo.get().getOutputStage().isPresent()) { @@ -1054,20 +1079,150 @@ public void pruneQueryInfo() } QueryInfo queryInfo = finalInfo.get(); + QueryInfo prunedQueryInfo; + + // no longer needed in the session after query finishes + session.getPlanNodeStatsMap().clear(); + session.getPlanNodeCostMap().clear(); + // inputs contain some statistics which should be cleared + inputs.getAndUpdate(QueryStateMachine::pruneInputHistograms); + // query listeners maintain state in their arguments which holds + // onto plan nodes and statistics. Since finalQueryInfo was + // already set it should be in a terminal state and be safe to + // clear the listeners. + finalQueryInfo.clearEventListeners(); + planStatsAndCosts.getAndUpdate(stats -> Optional.ofNullable(stats) + .map(QueryStateMachine::pruneHistogramsFromStatsAndCosts) + .orElse(null)); + prunedQueryInfo = pruneFinishedQueryInfo(queryInfo, inputs.get()); + finalQueryInfo.compareAndSet(finalInfo, Optional.of(prunedQueryInfo)); + } + + private static QueryInfo pruneFinishedQueryInfo(QueryInfo queryInfo, Set prunedInputs) + { + return new QueryInfo( + queryInfo.getQueryId(), + queryInfo.getSession(), + queryInfo.getState(), + queryInfo.getMemoryPool(), + queryInfo.isScheduled(), + queryInfo.getSelf(), + queryInfo.getFieldNames(), + queryInfo.getQuery(), + queryInfo.getExpandedQuery(), + queryInfo.getPreparedQuery(), + queryInfo.getQueryStats(), + queryInfo.getSetCatalog(), + queryInfo.getSetSchema(), + queryInfo.getSetSessionProperties(), + queryInfo.getResetSessionProperties(), + queryInfo.getSetRoles(), + queryInfo.getAddedPreparedStatements(), + queryInfo.getDeallocatedPreparedStatements(), + queryInfo.getStartedTransactionId(), + queryInfo.isClearTransactionId(), + queryInfo.getUpdateType(), + queryInfo.getOutputStage().map(QueryStateMachine::pruneStatsFromStageInfo), + queryInfo.getFailureInfo(), + queryInfo.getErrorCode(), + queryInfo.getWarnings(), + prunedInputs, + queryInfo.getOutput(), + queryInfo.isFinalQueryInfo(), + queryInfo.getResourceGroupId(), + queryInfo.getQueryType(), + queryInfo.getFailedTasks(), + queryInfo.getRuntimeOptimizedStages(), + queryInfo.getAddedSessionFunctions(), + queryInfo.getRemovedSessionFunctions(), + pruneHistogramsFromStatsAndCosts(queryInfo.getPlanStatsAndCosts()), + queryInfo.getOptimizerInformation(), + queryInfo.getCteInformationList(), + queryInfo.getScalarFunctions(), + queryInfo.getAggregateFunctions(), + queryInfo.getWindowFunctions(), + ImmutableList.of(), + ImmutableMap.of(), + queryInfo.getPrestoSparkExecutionContext()); + } + + private static Set pruneInputHistograms(Set inputs) + { + return inputs.stream().map(input -> new Input(input.getConnectorId(), + input.getSchema(), + input.getTable(), + input.getConnectorInfo(), + input.getColumns(), + input.getStatistics().map(tableStats -> TableStatistics.buildFrom(tableStats) + .setColumnStatistics(ImmutableMap.copyOf( + Maps.transformValues(tableStats.getColumnStatistics(), + columnStats -> ColumnStatistics.buildFrom(columnStats) + .setHistogram(Optional.empty()) + .build()))) + .build()), + input.getSerializedCommitOutput())) + .collect(toImmutableSet()); + } + + protected static StatsAndCosts pruneHistogramsFromStatsAndCosts(StatsAndCosts statsAndCosts) + { + Map newStats = statsAndCosts.getStats() + .entrySet() + .stream() + .collect(toImmutableMap(entry -> entry.getKey(), + entry -> PlanNodeStatsEstimate.buildFrom(entry.getValue()) + .addVariableStatistics(ImmutableMap.copyOf( + Maps.transformValues( + entry.getValue().getVariableStatistics(), + variableStats -> VariableStatsEstimate.buildFrom(variableStats) + .setHistogram(Optional.empty()) + .build()))) + .build())); + + return new StatsAndCosts(newStats, + statsAndCosts.getCosts()); + } + + private static StageInfo pruneStatsFromStageInfo(StageInfo stage) + { + return new StageInfo( + stage.getStageId(), + stage.getSelf(), + stage.getPlan().map(plan -> new PlanFragment( + plan.getId(), + plan.getRoot(), + plan.getVariables(), + plan.getPartitioning(), + plan.getTableScanSchedulingOrder(), + plan.getPartitioningScheme(), + plan.getStageExecutionDescriptor(), + plan.isOutputTableWriterFragment(), + plan.getStatsAndCosts().map(QueryStateMachine::pruneHistogramsFromStatsAndCosts), + plan.getJsonRepresentation())), // Remove the plan + stage.getLatestAttemptExecutionInfo(), + stage.getPreviousAttemptsExecutionInfos(), // Remove failed attempts + stage.getSubStages().stream() + .map(QueryStateMachine::pruneStatsFromStageInfo) + .collect(toImmutableList()), // Remove the substages + stage.isRuntimeOptimized()); + } + + private static QueryInfo pruneExpiredQueryInfo(QueryInfo queryInfo, VersionedMemoryPoolId pool) + { Optional prunedOutputStage = queryInfo.getOutputStage().map(outputStage -> new StageInfo( outputStage.getStageId(), outputStage.getSelf(), Optional.empty(), // Remove the plan pruneStageExecutionInfo(outputStage.getLatestAttemptExecutionInfo()), ImmutableList.of(), // Remove failed attempts - ImmutableList.of(), - outputStage.isRuntimeOptimized())); // Remove the substages + ImmutableList.of(), // Remove the substages + outputStage.isRuntimeOptimized())); - QueryInfo prunedQueryInfo = new QueryInfo( + return new QueryInfo( queryInfo.getQueryId(), queryInfo.getSession(), queryInfo.getState(), - getMemoryPool().getId(), + pool.getId(), queryInfo.isScheduled(), queryInfo.getSelf(), queryInfo.getFieldNames(), @@ -1107,7 +1262,6 @@ public void pruneQueryInfo() ImmutableList.of(), ImmutableMap.of(), queryInfo.getPrestoSparkExecutionContext()); - finalQueryInfo.compareAndSet(finalInfo, Optional.of(prunedQueryInfo)); } private static StageExecutionInfo pruneStageExecutionInfo(StageExecutionInfo info) diff --git a/presto-main/src/main/java/com/facebook/presto/execution/QueryTracker.java b/presto-main/src/main/java/com/facebook/presto/execution/QueryTracker.java index e37d5c684ef61..e95a55d02f5cc 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/QueryTracker.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/QueryTracker.java @@ -197,7 +197,10 @@ public boolean addQuery(T execution) public void expireQuery(QueryId queryId) { tryGetQuery(queryId) - .ifPresent(expirationQueue::add); + .ifPresent(query -> { + query.pruneFinishedQueryInfo(); + expirationQueue.add(query); + }); } public long getRunningTaskCount() @@ -264,8 +267,8 @@ public int getTaskCount() } /** - * When cluster reaches max tasks limit and also a single query - * exceeds a threshold, kill this query + * When cluster reaches max tasks limit and also a single query + * exceeds a threshold, kill this query */ @VisibleForTesting void enforceTaskLimits() @@ -316,7 +319,7 @@ private void pruneExpiredQueries() if (expirationQueue.size() - count <= maxQueryHistory) { break; } - query.pruneInfo(); + query.pruneExpiredQueryInfo(); count++; } } @@ -409,6 +412,17 @@ public interface TrackedQuery void fail(Throwable cause); // XXX: This should be removed when the client protocol is improved, so that we don't need to hold onto so much query history - void pruneInfo(); + + /** + * Prune info from finished queries which are in the expiry queue and the queue length is + * greater than {@code query.max-history} + */ + void pruneExpiredQueryInfo(); + + /** + * Prune info from finished queries which should not be kept around at all after the query + * state machine has transitioned into a finished state + */ + void pruneFinishedQueryInfo(); } } diff --git a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java index b9470281fe4fa..cab667d73b832 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java @@ -88,6 +88,7 @@ import static com.facebook.presto.common.RuntimeMetricName.GET_CANONICAL_INFO_TIME_NANOS; import static com.facebook.presto.common.RuntimeMetricName.LOGICAL_PLANNER_TIME_NANOS; import static com.facebook.presto.common.RuntimeMetricName.OPTIMIZER_TIME_NANOS; +import static com.facebook.presto.execution.QueryStateMachine.pruneHistogramsFromStatsAndCosts; import static com.facebook.presto.execution.buffer.OutputBuffers.BROADCAST_PARTITION_ID; import static com.facebook.presto.execution.buffer.OutputBuffers.createInitialEmptyOutputBuffers; import static com.facebook.presto.execution.buffer.OutputBuffers.createSpoolingOutputBuffers; @@ -745,9 +746,21 @@ public void recordHeartbeat() } @Override - public void pruneInfo() + public void pruneExpiredQueryInfo() { - stateMachine.pruneQueryInfo(); + stateMachine.pruneQueryInfoExpired(); + } + + @Override + public void pruneFinishedQueryInfo() + { + queryPlan.getAndUpdate(plan -> new Plan( + plan.getRoot(), + plan.getTypes(), + pruneHistogramsFromStatsAndCosts(plan.getStatsAndCosts()))); + // drop the reference to the scheduler since execution is finished + queryScheduler.set(null); + stateMachine.pruneQueryInfoFinished(); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/execution/StateMachine.java b/presto-main/src/main/java/com/facebook/presto/execution/StateMachine.java index e9046bb854ef5..559e245720291 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/StateMachine.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/StateMachine.java @@ -283,6 +283,11 @@ boolean isTerminalState(T state) return terminalStates.contains(state); } + public void clearEventListeners() + { + stateChangeListeners.clear(); + } + @VisibleForTesting List> getStateChangeListeners() { diff --git a/presto-main/src/test/java/com/facebook/presto/execution/MockQueryExecution.java b/presto-main/src/test/java/com/facebook/presto/execution/MockQueryExecution.java index 139a260aa889f..6693c7ee2f19f 100644 --- a/presto-main/src/test/java/com/facebook/presto/execution/MockQueryExecution.java +++ b/presto-main/src/test/java/com/facebook/presto/execution/MockQueryExecution.java @@ -217,7 +217,11 @@ public Optional getFailureReason() } @Override - public void pruneInfo() + public void pruneExpiredQueryInfo() + { } + + @Override + public void pruneFinishedQueryInfo() { } @Override diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/statistics/ColumnStatistics.java b/presto-spi/src/main/java/com/facebook/presto/spi/statistics/ColumnStatistics.java index ec4ee420e46bd..bec6fb5c31299 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/statistics/ColumnStatistics.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/statistics/ColumnStatistics.java @@ -132,6 +132,16 @@ public static Builder builder() return new Builder(); } + public static Builder buildFrom(ColumnStatistics statistics) + { + return new Builder() + .setRange(statistics.getRange()) + .setDataSize(statistics.getDataSize()) + .setNullsFraction(statistics.getNullsFraction()) + .setDistinctValuesCount(statistics.getDistinctValuesCount()) + .setHistogram(statistics.getHistogram()); + } + /** * If one of the estimates below is unspecified, the default "unknown" estimate value * (represented by floating point NaN) may cause the resulting symbol statistics diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/statistics/TableStatistics.java b/presto-spi/src/main/java/com/facebook/presto/spi/statistics/TableStatistics.java index dd4060c785883..c6fece6d1a66c 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/statistics/TableStatistics.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/statistics/TableStatistics.java @@ -116,6 +116,15 @@ public static Builder builder() return new Builder(); } + public static Builder buildFrom(TableStatistics tableStatistics) + { + return new Builder() + .setRowCount(tableStatistics.getRowCount()) + .setTotalSize(tableStatistics.getTotalSize()) + .setConfidenceLevel(tableStatistics.getConfidence()) + .setColumnStatistics(tableStatistics.getColumnStatistics()); + } + public static final class Builder { private Estimate rowCount = Estimate.unknown(); @@ -154,6 +163,13 @@ public Builder setColumnStatistics(ColumnHandle columnHandle, ColumnStatistics c return this; } + public Builder setColumnStatistics(Map columnStatistics) + { + requireNonNull(columnStatistics, "columnStatistics can not be null"); + this.columnStatisticsMap.putAll(columnStatistics); + return this; + } + public Map getColumnStatistics() { return Collections.unmodifiableMap(columnStatisticsMap); diff --git a/presto-tests/src/test/java/com/facebook/presto/execution/TestEventListener.java b/presto-tests/src/test/java/com/facebook/presto/execution/TestEventListener.java index 85892ea69c1bd..8137940444408 100644 --- a/presto-tests/src/test/java/com/facebook/presto/execution/TestEventListener.java +++ b/presto-tests/src/test/java/com/facebook/presto/execution/TestEventListener.java @@ -335,7 +335,7 @@ public void testGraphvizQueryPlanOutput() assertEquals(queryCompletedEvent.getMetadata().getGraphvizPlan().get(), getOnlyElement(expected.getOnlyColumnAsSet())); } - static class EventsBuilder + public static class EventsBuilder { private ImmutableList.Builder queryCreatedEvents; private ImmutableList.Builder queryCompletedEvents; diff --git a/presto-tests/src/test/java/com/facebook/presto/execution/TestEventListenerPlugin.java b/presto-tests/src/test/java/com/facebook/presto/execution/TestEventListenerPlugin.java index bc57984352327..f6a36429d29bf 100644 --- a/presto-tests/src/test/java/com/facebook/presto/execution/TestEventListenerPlugin.java +++ b/presto-tests/src/test/java/com/facebook/presto/execution/TestEventListenerPlugin.java @@ -29,7 +29,7 @@ public class TestEventListenerPlugin { - static class TestingEventListenerPlugin + public static class TestingEventListenerPlugin implements Plugin { private final EventsBuilder eventsBuilder; @@ -46,7 +46,7 @@ public Iterable getEventListenerFactories() } } - private static class TestingEventListenerFactory + public static class TestingEventListenerFactory implements EventListenerFactory { private final EventsBuilder eventsBuilder; @@ -69,7 +69,7 @@ public EventListener create(Map config) } } - private static class TestingEventListener + public static class TestingEventListener implements EventListener { private final EventsBuilder eventsBuilder; diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestQueryManager.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestQueryManager.java index 3ef80e08fefc1..7e82c52e51bf2 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestQueryManager.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestQueryManager.java @@ -13,35 +13,58 @@ */ package com.facebook.presto.tests; +import com.facebook.presto.Session; +import com.facebook.presto.common.RuntimeStats; +import com.facebook.presto.cost.StatsAndCosts; import com.facebook.presto.dispatcher.DispatchManager; +import com.facebook.presto.execution.MockQueryExecution; import com.facebook.presto.execution.QueryInfo; import com.facebook.presto.execution.QueryManager; import com.facebook.presto.execution.QueryState; +import com.facebook.presto.execution.QueryStats; +import com.facebook.presto.execution.StateMachine; +import com.facebook.presto.execution.TestEventListener.EventsBuilder; +import com.facebook.presto.execution.TestEventListenerPlugin.TestingEventListenerPlugin; import com.facebook.presto.execution.TestingSessionContext; import com.facebook.presto.plugin.blackhole.BlackHolePlugin; import com.facebook.presto.resourceGroups.FileResourceGroupConfigurationManagerFactory; import com.facebook.presto.server.BasicQueryInfo; import com.facebook.presto.server.testing.TestingPrestoServer; import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.PrestoWarning; import com.facebook.presto.spi.QueryId; +import com.facebook.presto.spi.WarningCode; +import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; +import com.facebook.presto.spi.memory.MemoryPoolId; import com.google.common.base.Stopwatch; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.airlift.units.DataSize; +import io.airlift.units.Duration; +import org.intellij.lang.annotations.Language; +import org.joda.time.DateTime; import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import java.net.URI; +import java.util.ArrayList; import java.util.List; +import java.util.Optional; import static com.facebook.presto.SessionTestUtils.TEST_SESSION; import static com.facebook.presto.execution.QueryState.FAILED; +import static com.facebook.presto.execution.QueryState.FINISHED; import static com.facebook.presto.execution.QueryState.QUEUED; import static com.facebook.presto.execution.QueryState.RUNNING; import static com.facebook.presto.execution.TestQueryRunnerUtil.createQuery; import static com.facebook.presto.execution.TestQueryRunnerUtil.waitForQueryState; import static com.facebook.presto.execution.resourceGroups.db.H2TestUtil.getSimpleQueryRunner; +import static com.facebook.presto.operator.BlockedReason.WAITING_FOR_MEMORY; import static com.facebook.presto.spi.StandardErrorCode.EXCEEDED_CPU_LIMIT; +import static com.facebook.presto.spi.StandardErrorCode.EXCEEDED_GLOBAL_MEMORY_LIMIT; import static com.facebook.presto.spi.StandardErrorCode.EXCEEDED_OUTPUT_POSITIONS_LIMIT; import static com.facebook.presto.spi.StandardErrorCode.EXCEEDED_OUTPUT_SIZE_LIMIT; import static com.facebook.presto.spi.StandardErrorCode.EXCEEDED_SCAN_RAW_BYTES_READ_LIMIT; @@ -278,4 +301,211 @@ public void testQueryCountMetrics() Thread.sleep(1000); } } + + @Test + public void testQueryCompletedInfoNotPruned() + throws Exception + { + try (DistributedQueryRunner runner = DistributedQueryRunner.builder(TEST_SESSION) + .setNodeCount(0) + .build()) { + EventsBuilder eventsBuilder = new EventsBuilder(); + eventsBuilder.initialize(1); + TestingEventListenerPlugin testEventListenerPlugin = new TestingEventListenerPlugin(eventsBuilder); + runner.installPlugin(testEventListenerPlugin); + QueryManager manager = runner.getCoordinator().getQueryManager(); + QueryId id = runner.getCoordinator().getDispatchManager().createQueryId(); + @Language("SQL") String sql = "SELECT * FROM lineitem WHERE linenumber = 0 LIMIT 1"; + QueryInfo mockInfo = mockInfo(sql, id.toString(), FINISHED); + MockExecution exec = new MockExecution(eventsBuilder, mockInfo); + manager.createQuery(exec); + + // when the listener executes, we will verify that the query completed event exists + // when pruneInfo is called + exec.finalInfoListeners.forEach(item -> item.stateChanged(mockInfo)); + // verify we actually called pruneQueryFinished to assert that it was checked + assertEquals(exec.pruneFinishedCalls, 1); + } + } + + private static class MockExecution + extends MockQueryExecution + { + List> finalInfoListeners = new ArrayList<>(); + private final EventsBuilder eventsBuilder; + int pruneFinishedCalls; + int pruneExpiredCalls; + private final QueryInfo info; + + private MockExecution(EventsBuilder eventsBuilder, QueryInfo info) + { + this.eventsBuilder = eventsBuilder; + this.info = info; + } + + @Override + public DateTime getCreateTime() + { + return info.getQueryStats().getCreateTime(); + } + + @Override + public Duration getTotalCpuTime() + { + return info.getQueryStats().getTotalCpuTime(); + } + + @Override + public DataSize getRawInputDataSize() + { + return info.getQueryStats().getRawInputDataSize(); + } + + @Override + public DataSize getOutputDataSize() + { + return info.getQueryStats().getOutputDataSize(); + } + + @Override + public Session getSession() + { + return TEST_SESSION; + } + + @Override + public void addFinalQueryInfoListener(StateMachine.StateChangeListener stateChangeListener) + { + finalInfoListeners.add(stateChangeListener); + } + + @Override + public void pruneExpiredQueryInfo() + { + pruneExpiredCalls++; + Optional event = eventsBuilder.getQueryCompletedEvents().stream() + .filter(x -> x.getMetadata().getQueryId().equals(info.getQueryId().toString())) + .findFirst(); + // verify that the event listener was notified before prune was called + assertTrue(event.isPresent()); + } + + @Override + public void pruneFinishedQueryInfo() + { + pruneFinishedCalls++; + Optional event = eventsBuilder.getQueryCompletedEvents().stream() + .filter(x -> x.getMetadata().getQueryId().equals(info.getQueryId().toString())) + .findFirst(); + // verify that the event listener was notified before prune was called + assertTrue(event.isPresent()); + } + } + + private static QueryInfo mockInfo(String query, String queryId, QueryState state) + { + return new QueryInfo( + new QueryId(queryId), + TEST_SESSION.toSessionRepresentation(), + state, + new MemoryPoolId("reserved"), + true, + URI.create("1"), + ImmutableList.of("2", "3"), + query, + Optional.empty(), + Optional.empty(), + new QueryStats( + DateTime.parse("1991-09-06T05:00-05:30"), + DateTime.parse("1991-09-06T05:01-05:30"), + DateTime.parse("1991-09-06T05:02-05:30"), + DateTime.parse("1991-09-06T06:00-05:30"), + Duration.valueOf("8m"), + Duration.valueOf("5m"), + Duration.valueOf("7m"), + Duration.valueOf("34m"), + Duration.valueOf("5m"), + Duration.valueOf("6m"), + Duration.valueOf("35m"), + Duration.valueOf("44m"), + Duration.valueOf("9m"), + Duration.valueOf("10m"), + Duration.valueOf("11m"), + 13, + 14, + 15, + 16, + 100, + 17, + 18, + 34, + 19, + 20.0, + 43.0, + DataSize.valueOf("21GB"), + DataSize.valueOf("22GB"), + DataSize.valueOf("23GB"), + DataSize.valueOf("24GB"), + DataSize.valueOf("25GB"), + DataSize.valueOf("26GB"), + DataSize.valueOf("42GB"), + true, + Duration.valueOf("23m"), + Duration.valueOf("24m"), + Duration.valueOf("0m"), + Duration.valueOf("26m"), + true, + ImmutableSet.of(WAITING_FOR_MEMORY), + DataSize.valueOf("123MB"), + DataSize.valueOf("27GB"), + 28, + DataSize.valueOf("29GB"), + 30, + DataSize.valueOf("32GB"), + 40, + DataSize.valueOf("31GB"), + 32, + 33, + DataSize.valueOf("34GB"), + DataSize.valueOf("35GB"), + DataSize.valueOf("36GB"), + ImmutableList.of(), + ImmutableList.of(), + new RuntimeStats()), + Optional.empty(), + Optional.empty(), + ImmutableMap.of(), + ImmutableSet.of(), + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableSet.of(), + Optional.empty(), + false, + "33", + Optional.empty(), + null, + EXCEEDED_GLOBAL_MEMORY_LIMIT.toErrorCode(), + ImmutableList.of( + new PrestoWarning( + new WarningCode(123, "WARNING_123"), + "warning message")), + ImmutableSet.of(), + Optional.empty(), + false, + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), + ImmutableMap.of(), + ImmutableSet.of(), + StatsAndCosts.empty(), + ImmutableList.of(), + ImmutableList.of(), + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableList.of(), + ImmutableMap.of(), + Optional.empty()); + } } From fdd20597fba2ce9b6993a5b4e82e3c46f93d7e77 Mon Sep 17 00:00:00 2001 From: Pramod Satya Date: Wed, 20 Dec 2023 06:21:44 -0800 Subject: [PATCH 16/31] [native] Retrieve presto native session properties Co-authored-by: Abe Varghese Kodiyan Co-authored-by: Joe Abraham Co-authored-by: Deepthy Davis --- .../presto_cpp/main/CMakeLists.txt | 1 + .../presto_cpp/main/PrestoServer.cpp | 13 + .../presto_cpp/main/QueryContextManager.cpp | 76 ++---- .../presto_cpp/main/QueryContextManager.h | 9 + .../presto_cpp/main/SessionProperties.cpp | 243 ++++++++++++++++++ .../presto_cpp/main/SessionProperties.h | 167 ++++++++++++ .../presto_cpp/main/tests/CMakeLists.txt | 1 + .../main/tests/SessionPropertiesTest.cpp | 52 ++++ 8 files changed, 511 insertions(+), 51 deletions(-) create mode 100644 presto-native-execution/presto_cpp/main/SessionProperties.cpp create mode 100644 presto-native-execution/presto_cpp/main/SessionProperties.h create mode 100644 presto-native-execution/presto_cpp/main/tests/SessionPropertiesTest.cpp diff --git a/presto-native-execution/presto_cpp/main/CMakeLists.txt b/presto-native-execution/presto_cpp/main/CMakeLists.txt index e5cf1835dd824..14d288ebca647 100644 --- a/presto-native-execution/presto_cpp/main/CMakeLists.txt +++ b/presto-native-execution/presto_cpp/main/CMakeLists.txt @@ -30,6 +30,7 @@ add_library( ServerOperation.cpp SignalHandler.cpp SystemConnector.cpp + SessionProperties.cpp TaskManager.cpp TaskResource.cpp PeriodicHeartbeatManager.cpp diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.cpp b/presto-native-execution/presto_cpp/main/PrestoServer.cpp index 21384cf218b52..9c587fcc084ea 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.cpp +++ b/presto-native-execution/presto_cpp/main/PrestoServer.cpp @@ -466,6 +466,19 @@ void PrestoServer::run() { taskManager_ = std::make_unique( driverExecutor_.get(), httpSrvCpuExecutor_.get(), spillerExecutor_.get()); + if (systemConfig->prestoNativeSidecar()) { + httpServer_->registerGet( + "/v1/properties/session", + [this]( + proxygen::HTTPMessage* /*message*/, + const std::vector>& /*body*/, + proxygen::ResponseHandler* downstream) { + auto sessionProperties = + taskManager_->getQueryContextManager()->getSessionProperties(); + http::sendOkResponse(downstream, sessionProperties.serialize()); + }); + } + std::string taskUri; if (httpsPort.has_value()) { taskUri = fmt::format(kTaskUriFormat, kHttps, address_, httpsPort.value()); diff --git a/presto-native-execution/presto_cpp/main/QueryContextManager.cpp b/presto-native-execution/presto_cpp/main/QueryContextManager.cpp index eb41ce33c088f..e6c0d171f1838 100644 --- a/presto-native-execution/presto_cpp/main/QueryContextManager.cpp +++ b/presto-native-execution/presto_cpp/main/QueryContextManager.cpp @@ -28,36 +28,6 @@ using facebook::presto::protocol::TaskId; namespace facebook::presto { namespace { -// Utility function to translate a config name in Presto to its equivalent in -// Velox. Returns 'name' as is if there is no mapping. -std::string toVeloxConfig(const std::string& name) { - using velox::core::QueryConfig; - static const folly::F14FastMap - kPrestoToVeloxMapping = { - {"native_simplified_expression_evaluation_enabled", - QueryConfig::kExprEvalSimplified}, - {"native_max_spill_level", QueryConfig::kMaxSpillLevel}, - {"native_max_spill_file_size", QueryConfig::kMaxSpillFileSize}, - {"native_spill_compression_codec", - QueryConfig::kSpillCompressionKind}, - {"native_spill_write_buffer_size", - QueryConfig::kSpillWriteBufferSize}, - {"native_spill_file_create_config", - QueryConfig::kSpillFileCreateConfig}, - {"native_join_spill_enabled", QueryConfig::kJoinSpillEnabled}, - {"native_window_spill_enabled", QueryConfig::kWindowSpillEnabled}, - {"native_writer_spill_enabled", QueryConfig::kWriterSpillEnabled}, - {"native_row_number_spill_enabled", - QueryConfig::kRowNumberSpillEnabled}, - {"native_spiller_num_partition_bits", - QueryConfig::kSpillNumPartitionBits}, - {"native_topn_row_number_spill_enabled", - QueryConfig::kTopNRowNumberSpillEnabled}, - {"native_debug_validate_output_from_operators", - QueryConfig::kValidateOutputFromOperators}}; - auto it = kPrestoToVeloxMapping.find(name); - return it == kPrestoToVeloxMapping.end() ? name : it->second; -} // Update passed in query session configs with system configs. For any pairing // system/session configs if session config is present, it overrides system @@ -89,26 +59,6 @@ void updateFromSystemConfigs( } } -std::unordered_map toVeloxConfigs( - const protocol::SessionRepresentation& session) { - // Use base velox query config as the starting point and add Presto session - // properties on top of it. - auto configs = BaseVeloxQueryConfig::instance()->values(); - for (const auto& it : session.systemProperties) { - configs[toVeloxConfig(it.first)] = it.second; - } - - // If there's a timeZoneKey, convert to timezone name and add to the - // configs. Throws if timeZoneKey can't be resolved. - if (session.timeZoneKey != 0) { - configs.emplace( - velox::core::QueryConfig::kSessionTimezone, - velox::tz::getTimeZoneName(session.timeZoneKey)); - } - updateFromSystemConfigs(configs); - return configs; -} - std::unordered_map> toConnectorConfigs(const protocol::SessionRepresentation& session) { std::unordered_map> @@ -167,7 +117,9 @@ void updateVeloxConnectorConfigs( QueryContextManager::QueryContextManager( folly::Executor* driverExecutor, folly::Executor* spillerExecutor) - : driverExecutor_(driverExecutor), spillerExecutor_(spillerExecutor) {} + : driverExecutor_(driverExecutor), + spillerExecutor_(spillerExecutor), + sessionProperties_(SessionProperties()) {} std::shared_ptr QueryContextManager::findOrCreateQueryCtx( @@ -246,4 +198,26 @@ void QueryContextCache::testingClear() { queryIds_.clear(); } +std::unordered_map +QueryContextManager::toVeloxConfigs( + const protocol::SessionRepresentation& session) { + // Use base velox query config as the starting point and add Presto session + // properties on top of it. + auto configs = BaseVeloxQueryConfig::instance()->values(); + for (const auto& it : session.systemProperties) { + configs[sessionProperties_.toVeloxConfig(it.first)] = it.second; + sessionProperties_.updateVeloxConfig(it.first, it.second); + } + + // If there's a timeZoneKey, convert to timezone name and add to the + // configs. Throws if timeZoneKey can't be resolved. + if (session.timeZoneKey != 0) { + configs.emplace( + velox::core::QueryConfig::kSessionTimezone, + velox::tz::getTimeZoneName(session.timeZoneKey)); + } + updateFromSystemConfigs(configs); + return configs; +} + } // namespace facebook::presto diff --git a/presto-native-execution/presto_cpp/main/QueryContextManager.h b/presto-native-execution/presto_cpp/main/QueryContextManager.h index 12ac3fb29e30d..49ee7f2e072bd 100644 --- a/presto-native-execution/presto_cpp/main/QueryContextManager.h +++ b/presto-native-execution/presto_cpp/main/QueryContextManager.h @@ -19,6 +19,7 @@ #include #include +#include "presto_cpp/main/SessionProperties.h" #include "presto_cpp/presto_protocol/presto_protocol.h" #include "velox/core/QueryCtx.h" @@ -116,6 +117,10 @@ class QueryContextManager { /// Test method to clear the query context cache. void testingClearCache(); + const SessionProperties& getSessionProperties() const { + return sessionProperties_; + } + private: std::shared_ptr findOrCreateQueryCtx( const protocol::TaskId& taskId, @@ -125,10 +130,14 @@ class QueryContextManager { std::unordered_map>&& connectorConfigStrings); + std::unordered_map toVeloxConfigs( + const protocol::SessionRepresentation& session); + folly::Executor* const driverExecutor_{nullptr}; folly::Executor* const spillerExecutor_{nullptr}; folly::Synchronized queryContextCache_; + SessionProperties sessionProperties_; }; } // namespace facebook::presto diff --git a/presto-native-execution/presto_cpp/main/SessionProperties.cpp b/presto-native-execution/presto_cpp/main/SessionProperties.cpp new file mode 100644 index 0000000000000..8c0464669d190 --- /dev/null +++ b/presto-native-execution/presto_cpp/main/SessionProperties.cpp @@ -0,0 +1,243 @@ +/* + * 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. + */ +#include "presto_cpp/main/SessionProperties.h" +#include "velox/core/QueryConfig.h" + +using namespace facebook::velox; + +namespace facebook::presto { + +namespace { +const std::string boolToString(bool value) { + return value ? "true" : "false"; +} +} // namespace + +json SessionProperty::serialize() { + json j; + j["name"] = name_; + j["description"] = description_; + j["typeSignature"] = type_; + j["defaultValue"] = defaultValue_; + j["hidden"] = hidden_; + return j; +} + +void SessionProperties::addSessionProperty( + const std::string& name, + const std::string& description, + const TypePtr& type, + bool isHidden, + const std::string& veloxConfigName, + const std::string& veloxDefault) { + sessionProperties_[name] = std::make_shared( + name, + description, + type->toString(), + isHidden, + veloxConfigName, + veloxDefault); +} + +// List of native session properties is kept as the source of truth here. +SessionProperties::SessionProperties() { + using velox::core::QueryConfig; + // Use empty instance to get default property values. + QueryConfig c{{}}; + + addSessionProperty( + kExprEvalSimplified, + "Native Execution only. Enable simplified path in expression evaluation", + BOOLEAN(), + false, + QueryConfig::kExprEvalSimplified, + boolToString(c.exprEvalSimplified())); + + addSessionProperty( + kMaxSpillLevel, + "Native Execution only. The maximum allowed spilling level for hash join " + "build. 0 is the initial spilling level, -1 means unlimited.", + INTEGER(), + false, + QueryConfig::kMaxSpillLevel, + std::to_string(c.maxSpillLevel())); + + addSessionProperty( + kMaxSpillFileSize, + "The max allowed spill file size. If it is zero, then there is no limit.", + INTEGER(), + false, + QueryConfig::kMaxSpillFileSize, + std::to_string(c.maxSpillFileSize())); + + addSessionProperty( + kSpillCompressionCodec, + "Native Execution only. The compression algorithm type to compress the " + "spilled data.\n Supported compression codecs are: ZLIB, SNAPPY, LZO, " + "ZSTD, LZ4 and GZIP. NONE means no compression.", + VARCHAR(), + false, + QueryConfig::kSpillCompressionKind, + c.spillCompressionKind()); + + addSessionProperty( + kSpillWriteBufferSize, + "Native Execution only. The maximum size in bytes to buffer the serialized " + "spill data before writing to disk for IO efficiency. If set to zero, " + "buffering is disabled.", + BIGINT(), + false, + QueryConfig::kSpillWriteBufferSize, + std::to_string(c.spillWriteBufferSize())); + + addSessionProperty( + kSpillFileCreateConfig, + "Native Execution only. Config used to create spill files. This config is " + "provided to underlying file system and the config is free form. The form should be " + "defined by the underlying file system.", + VARCHAR(), + false, + QueryConfig::kSpillFileCreateConfig, + c.spillFileCreateConfig()); + + addSessionProperty( + kJoinSpillEnabled, + "Native Execution only. Enable join spilling on native engine", + BOOLEAN(), + false, + QueryConfig::kJoinSpillEnabled, + boolToString(c.joinSpillEnabled())); + + addSessionProperty( + kWindowSpillEnabled, + "Native Execution only. Enable window spilling on native engine", + BOOLEAN(), + false, + QueryConfig::kWindowSpillEnabled, + boolToString(c.windowSpillEnabled())); + + addSessionProperty( + kWriterSpillEnabled, + "Native Execution only. Enable writer spilling on native engine", + BOOLEAN(), + false, + QueryConfig::kWriterSpillEnabled, + boolToString(c.writerSpillEnabled())); + + addSessionProperty( + kRowNumberSpillEnabled, + "Native Execution only. Enable row number spilling on native engine", + BOOLEAN(), + false, + QueryConfig::kRowNumberSpillEnabled, + boolToString(c.rowNumberSpillEnabled())); + + addSessionProperty( + kJoinSpillPartitionBits, + "Native Execution only. The number of bits (N) used to calculate the " + "spilling partition number for hash join and RowNumber: 2 ^ N", + INTEGER(), + false, + QueryConfig::kJoinSpillPartitionBits, + std::to_string(c.rowNumberSpillEnabled())); + + addSessionProperty( + kNativeSpillerNumPartitionBits, + "none", + TINYINT(), + false, + QueryConfig::kSpillNumPartitionBits, + std::to_string(c.spillNumPartitionBits())), + + addSessionProperty( + kTopNRowNumberSpillEnabled, + "Native Execution only. Enable topN row number spilling on native engine", + BOOLEAN(), + false, + QueryConfig::kTopNRowNumberSpillEnabled, + boolToString(c.topNRowNumberSpillEnabled())); + + addSessionProperty( + kValidateOutputFromOperators, + "If set to true, then during execution of tasks, the output vectors of " + "every operator are validated for consistency. This is an expensive check " + "so should only be used for debugging. It can help debug issues where " + "malformed vector cause failures or crashes by helping identify which " + "operator is generating them.", + BOOLEAN(), + false, + QueryConfig::kValidateOutputFromOperators, + boolToString(c.validateOutputFromOperators())); + + // If `legacy_timestamp` is true, the coordinator expects timestamp + // conversions without a timezone to be converted to the user's + // session_timezone. + addSessionProperty( + kLegacyTimestamp, + "Native Execution only. Use legacy TIME & TIMESTAMP semantics. Warning: " + "this will be removed", + BOOLEAN(), + false, + QueryConfig::kAdjustTimestampToTimezone, + // Overrides velox default value. legacy_timestamp default value is true + // in the coordinator. + "true"); + + // TODO: remove this once cpu driver slicing config is turned on by default in + // Velox. + addSessionProperty( + kDriverCpuTimeSliceLimitMs, + "Native Execution only. The cpu time slice limit in ms that a driver thread. " + "If not zero, can continuously run without yielding. If it is zero, then " + "there is no limit.", + INTEGER(), + false, + QueryConfig::kDriverCpuTimeSliceLimitMs, + // Overrides velox default value. Set it to 1 second to be aligned with + // Presto Java. + std::to_string(1000)); +} + +const std::unordered_map>& +SessionProperties::getSessionProperties() { + return sessionProperties_; +} + +const std::string SessionProperties::toVeloxConfig(const std::string& name) { + auto it = sessionProperties_.find(name); + return it == sessionProperties_.end() ? name + : it->second->getVeloxConfigName(); +} + +void SessionProperties::updateVeloxConfig( + const std::string& name, + const std::string& value) { + auto it = sessionProperties_.find(name); + // Velox config value is updated only for presto session properties. + if (it == sessionProperties_.end()) { + return; + } + it->second->updateValue(value); +} + +json SessionProperties::serialize() { + json j = json::array(); + const auto sessionProperties = getSessionProperties(); + for (const auto& entry : sessionProperties) { + j.push_back(entry.second->serialize()); + } + return j; +} + +} // namespace facebook::presto diff --git a/presto-native-execution/presto_cpp/main/SessionProperties.h b/presto-native-execution/presto_cpp/main/SessionProperties.h new file mode 100644 index 0000000000000..09e2b0506b5ef --- /dev/null +++ b/presto-native-execution/presto_cpp/main/SessionProperties.h @@ -0,0 +1,167 @@ +/* + * 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. + */ +#pragma once + +#include "presto_cpp/external/json/nlohmann/json.hpp" +#include "velox/type/Type.h" + +using json = nlohmann::json; + +namespace facebook::presto { + +/// This is the interface of the session property. +/// Note: This interface should align with java coordinator. +class SessionProperty { + public: + SessionProperty( + const std::string& name, + const std::string& description, + const std::string& type, + bool hidden, + const std::string& veloxConfigName, + const std::string& defaultValue) + : name_(name), + description_(description), + type_(type), + hidden_(hidden), + veloxConfigName_(veloxConfigName), + defaultValue_(defaultValue), + value_(defaultValue) {} + + const std::string getVeloxConfigName() { + return veloxConfigName_; + } + + void updateValue(const std::string& value) { + value_ = value; + } + + bool operator==(const SessionProperty& other) const { + return name_ == other.name_ && description_ == other.description_ && + type_ == other.type_ && hidden_ == other.hidden_ && + veloxConfigName_ == other.veloxConfigName_ && + defaultValue_ == other.defaultValue_; + } + + json serialize(); + + private: + const std::string name_; + const std::string description_; + + // Datatype of presto native property. + const std::string type_; + const bool hidden_; + const std::string veloxConfigName_; + const std::string defaultValue_; + std::string value_; +}; + +/// Defines all system session properties supported by native worker to ensure +/// that they are the source of truth and to differentiate them from Java based +/// session properties. Also maps the native session properties to velox. +class SessionProperties { + public: + /// Enable simplified path in expression evaluation. + static constexpr const char* kExprEvalSimplified = + "native_simplified_expression_evaluation_enabled"; + + /// Enable join spilling on native engine. + static constexpr const char* kJoinSpillEnabled = "native_join_spill_enabled"; + + /// The maximum allowed spilling level for hash join build. + static constexpr const char* kMaxSpillLevel = "native_max_spill_level"; + + /// The maximum allowed spill file size. + static constexpr const char* kMaxSpillFileSize = "native_max_spill_file_size"; + + /// Enable row number spilling on native engine. + static constexpr const char* kRowNumberSpillEnabled = + "native_row_number_spill_enabled"; + + /// The compression algorithm type to compress the spilled data. + static constexpr const char* kSpillCompressionCodec = + "native_spill_compression_codec"; + + /// The maximum size in bytes to buffer the serialized spill data before + /// writing to disk for IO efficiency. + static constexpr const char* kSpillWriteBufferSize = + "native_spill_write_buffer_size"; + + /// Config used to create spill files. This config is provided to underlying + /// file system and the config is free form. + static constexpr const char* kSpillFileCreateConfig = + "native_spill_file_create_config"; + + /// Enable window spilling on native engine. + static constexpr const char* kWindowSpillEnabled = + "native_window_spill_enabled"; + + /// Enable writer spilling on native engine. + static constexpr const char* kWriterSpillEnabled = + "native_writer_spill_enabled"; + + /// The number of bits (N) used to calculate the spilling + /// partition number for hash join and RowNumber: 2 ^ N + static constexpr const char* kJoinSpillPartitionBits = + "native_join_spiller_partition_bits"; + + static constexpr const char* kNativeSpillerNumPartitionBits = + "native_spiller_num_partition_bits"; + + /// Enable topN row number spilling on native engine. + static constexpr const char* kTopNRowNumberSpillEnabled = + "native_topn_row_number_spill_enabled"; + + /// If set to true, then during execution of tasks, the output vectors of + /// every operator are validated for consistency. This is an expensive check + /// so should only be used for debugging. + static constexpr const char* kValidateOutputFromOperators = + "native_debug_validate_output_from_operators"; + + /// Enable timezone-less timestamp conversions. + static constexpr const char* kLegacyTimestamp = "legacy_timestamp"; + + /// Specifies the cpu time slice limit in ms that a driver thread + /// can continuously run without yielding. + static constexpr const char* kDriverCpuTimeSliceLimitMs = + "driver_cpu_time_slice_limit_ms"; + + SessionProperties(); + + const std::unordered_map>& + getSessionProperties(); + + /// Utility function to translate a config name in Presto to its equivalent in + /// Velox. Returns 'name' as is if there is no mapping. + const std::string toVeloxConfig(const std::string& name); + + void updateVeloxConfig(const std::string& name, const std::string& value); + + json serialize(); + + protected: + void addSessionProperty( + const std::string& name, + const std::string& description, + const velox::TypePtr& type, + bool isHidden, + const std::string& veloxConfigName, + const std::string& veloxDefault); + + std::unordered_map> + sessionProperties_; +}; + +} // namespace facebook::presto diff --git a/presto-native-execution/presto_cpp/main/tests/CMakeLists.txt b/presto-native-execution/presto_cpp/main/tests/CMakeLists.txt index 004c5929eed64..9635db335915b 100644 --- a/presto-native-execution/presto_cpp/main/tests/CMakeLists.txt +++ b/presto-native-execution/presto_cpp/main/tests/CMakeLists.txt @@ -20,6 +20,7 @@ add_executable( PrestoTaskTest.cpp QueryContextCacheTest.cpp ServerOperationTest.cpp + SessionPropertiesTest.cpp TaskManagerTest.cpp QueryContextManagerTest.cpp) diff --git a/presto-native-execution/presto_cpp/main/tests/SessionPropertiesTest.cpp b/presto-native-execution/presto_cpp/main/tests/SessionPropertiesTest.cpp new file mode 100644 index 0000000000000..5da79aac1081f --- /dev/null +++ b/presto-native-execution/presto_cpp/main/tests/SessionPropertiesTest.cpp @@ -0,0 +1,52 @@ +/* + * 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. + */ +#include + +#include "presto_cpp/main/SessionProperties.h" +#include "velox/core/QueryConfig.h" +#include "velox/type/Type.h" + +using namespace facebook::presto; +using namespace facebook::velox; + +class SessionPropertiesTest : public testing::Test {}; + +TEST_F(SessionPropertiesTest, validateMapping) { + const std::vector names = { + SessionProperties::kLegacyTimestamp, + SessionProperties::kDriverCpuTimeSliceLimitMs, + SessionProperties::kSpillCompressionCodec}; + const std::vector veloxConfigNames = { + core::QueryConfig::kAdjustTimestampToTimezone, + core::QueryConfig::kDriverCpuTimeSliceLimitMs, + core::QueryConfig::kSpillCompressionKind}; + auto sessionProperties = SessionProperties().getSessionProperties(); + const auto len = names.size(); + for (auto i = 0; i < len; i++) { + EXPECT_EQ( + veloxConfigNames[i], + sessionProperties.at(names[i])->getVeloxConfigName()); + } +} + +TEST_F(SessionPropertiesTest, serializeProperty) { + auto properties = SessionProperties(); + auto j = properties.serialize(); + for (const auto& property : j) { + auto name = property["name"]; + json expectedProperty = + properties.getSessionProperties().at(name)->serialize(); + EXPECT_EQ(property, expectedProperty); + } +} From a37fea2286f2e24a6c22362a9a2468eb2da542c5 Mon Sep 17 00:00:00 2001 From: Elliotte Rusty Harold Date: Mon, 22 Jul 2024 07:54:54 -0400 Subject: [PATCH 17/31] Prefer assertSame --- .../com/facebook/presto/sql/planner/plan/TestAssignments.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/plan/TestAssignments.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/plan/TestAssignments.java index a4789590a74f6..a0dbde1e00e17 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/plan/TestAssignments.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/plan/TestAssignments.java @@ -23,7 +23,7 @@ import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.expressions.LogicalRowExpressions.TRUE_CONSTANT; import static com.facebook.presto.sql.planner.iterative.rule.test.PlanBuilder.assignment; -import static org.testng.Assert.assertTrue; +import static org.testng.Assert.assertSame; public class TestAssignments { @@ -40,6 +40,6 @@ public void testOutputsImmutable() @Test public void testOutputsMemoized() { - assertTrue(assignments.getOutputs() == assignments.getOutputs()); + assertSame(assignments.getOutputs(), assignments.getOutputs()); } } From 61a01f4b1d0ca1217420f00073d8e5cf484c237a Mon Sep 17 00:00:00 2001 From: Matt Calder Date: Mon, 12 Aug 2024 16:30:15 -0700 Subject: [PATCH 18/31] initial commit for ip_prefix_collapse function docs more tests Update presto-docs/src/main/sphinx/functions/ip.rst Co-authored-by: Steve Burnett addressing reviewer comments reviewer feedback 2 retrigger checks renaming function to ip_prefix_collapse --- presto-docs/src/main/sphinx/functions/ip.rst | 11 + .../operator/scalar/IpPrefixFunctions.java | 221 ++++++++++++++++++ .../scalar/TestIpPrefixFunctions.java | 98 ++++++++ 3 files changed, 330 insertions(+) diff --git a/presto-docs/src/main/sphinx/functions/ip.rst b/presto-docs/src/main/sphinx/functions/ip.rst index dcf5d4923b7bf..b8792483b0c00 100644 --- a/presto-docs/src/main/sphinx/functions/ip.rst +++ b/presto-docs/src/main/sphinx/functions/ip.rst @@ -49,3 +49,14 @@ IP Functions SELECT is_subnet_of(IPPREFIX '192.168.3.131/26', IPPREFIX '192.168.3.144/30'); -- true SELECT is_subnet_of(IPPREFIX '64:ff9b::17/64', IPPREFIX '64:ffff::17/64'); -- false SELECT is_subnet_of(IPPREFIX '192.168.3.131/26', IPPREFIX '192.168.3.131/26'); -- true + +.. function:: ip_prefix_collapse(array(ip_prefix)) -> array(ip_prefix) + + Returns the minimal CIDR representation of the input ``IPPREFIX`` array. + Every ``IPPREFIX`` in the input array must be the same IP version (that is, only IPv4 or only IPv6) + or the query will fail and raise an error. :: + + SELECT IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '192.168.0.0/24', IPPREFIX '192.168.1.0/24']); -- [{192.168.0.0/23}] + SELECT IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '2620:10d:c090::/48', IPPREFIX '2620:10d:c091::/48']); -- [{2620:10d:c090::/47}] + SELECT IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '192.168.1.0/24', IPPREFIX '192.168.0.0/24', IPPREFIX '192.168.2.0/24', IPPREFIX '192.168.9.0/24']); -- [{192.168.0.0/23}, {192.168.2.0/24}, {192.168.9.0/24}] + diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/IpPrefixFunctions.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/IpPrefixFunctions.java index e40bb3c6e0bd3..c0c808508f4a4 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/IpPrefixFunctions.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/IpPrefixFunctions.java @@ -24,9 +24,14 @@ import com.google.common.net.InetAddresses; import io.airlift.slice.Slice; +import java.math.BigInteger; import java.net.InetAddress; import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import static com.facebook.presto.operator.scalar.ArraySortFunction.sort; import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; import static com.facebook.presto.type.IpAddressOperators.between; @@ -44,6 +49,8 @@ public final class IpPrefixFunctions { + private static final BigInteger TWO = BigInteger.valueOf(2); + private IpPrefixFunctions() {} @Description("IP prefix for a given IP address and subnet size") @@ -137,6 +144,220 @@ public static boolean isPrefixSubnetOf(@SqlType(StandardTypes.IPPREFIX) Slice fi return between(ipSubnetMin(second), ipSubnetMin(first), ipSubnetMax(first)) && between(ipSubnetMax(second), ipSubnetMin(first), ipSubnetMax(first)); } + @Description("Combines the input set of IP prefixes into the fewest contiguous CIDR ranges possible.") + @ScalarFunction("ip_prefix_collapse") + @SqlType("array(IPPREFIX)") + public static Block collapseIpPrefixes(@SqlType("array(IPPREFIX)") Block unsortedIpPrefixArray) + { + int inputPrefixCount = unsortedIpPrefixArray.getPositionCount(); + + // If we get an empty array or an array non-null single element, just return the original array. + if (inputPrefixCount == 0 || (inputPrefixCount == 1 && !unsortedIpPrefixArray.isNull(0))) { + return unsortedIpPrefixArray; + } + + // Sort prefixes. lessThanFunction is never used. NULLs are placed at the end. + // Prefixes are ordered by first IP and then prefix length. + // Example: + // Input: 10.0.0.0/8, 9.255.255.0/24, 10.0.0.0/7, 10.1.0.0/24, 10.10.0.0/16 + // Output: 9.255.255.0/24, 10.0.0.0/7, 10.0.0.0/8, 10.1.0.0/24, 10.10.0.0/16 + Block ipPrefixArray = sort(null, IPPREFIX, unsortedIpPrefixArray); + + // throw if anything is null + if (ipPrefixArray.isNull(0) || ipPrefixArray.isNull(inputPrefixCount - 1)) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "ip_prefix_collapse does not support null elements"); + } + + // check the first and last prefixes in the array to make sure their IP versions match. + Slice firstIpPrefix = IPPREFIX.getSlice(ipPrefixArray, 0); + boolean v4 = isIpv4(firstIpPrefix); + Slice lastIpPrefix = IPPREFIX.getSlice(ipPrefixArray, inputPrefixCount - 1); + if (isIpv4(lastIpPrefix) != v4) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "All IPPREFIX elements must be the same IP version."); + } + + List> outputIpPrefixes = new ArrayList<>(); + int outputPrefixCount = 0; + int ipMaxBitLength = v4 ? 32 : 128; + + List mergedIpRanges = mergeIpRanges(ipPrefixArray); + for (BigInteger[] ipRange : mergedIpRanges) { + List ipPrefixes = generateMinIpPrefixes(ipRange[0], ipRange[1], ipMaxBitLength); + outputIpPrefixes.add(ipPrefixes); + outputPrefixCount += ipPrefixes.size(); + } + + BlockBuilder blockBuilder = IPPREFIX.createBlockBuilder(null, outputPrefixCount); + for (List ipPrefixSlices : outputIpPrefixes) { + for (Slice ipPrefix : ipPrefixSlices) { + IPPREFIX.writeSlice(blockBuilder, ipPrefix); + } + } + + return blockBuilder.build(); + } + + private static List generateMinIpPrefixes(BigInteger firstIpAddress, BigInteger lastIpAddress, int ipVersionMaxBits) + { + List ipPrefixSlices = new ArrayList<>(); + + // i.e., while firstIpAddress <= lastIpAddress + while (firstIpAddress.compareTo(lastIpAddress) <= 0) { + long rangeBits = findRangeBits(firstIpAddress, lastIpAddress); // find the number of bits for the next prefix in the range + int prefixLength = (int) (ipVersionMaxBits - rangeBits); + + try { + InetAddress asInetAddress = bigIntegerToIpAddress(firstIpAddress); // convert firstIpAddress from BigInt to Slice + Slice ipPrefixAsSlice = castFromVarcharToIpPrefix(utf8Slice(InetAddresses.toAddrString(asInetAddress) + "/" + prefixLength)); + ipPrefixSlices.add(ipPrefixAsSlice); + } + catch (UnknownHostException ex) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, "Unable to convert " + firstIpAddress + " to IP prefix", ex); + } + + BigInteger ipCount = TWO.pow(ipVersionMaxBits - prefixLength); + firstIpAddress = firstIpAddress.add(ipCount); // move to the next prefix in the range + } + + return ipPrefixSlices; + } + + private static long findRangeBits(BigInteger firstIpAddress, BigInteger lastIpAddress) + { + // The number of IP addresses in the range + BigInteger ipCount = lastIpAddress.subtract(firstIpAddress).add(BigInteger.ONE); + + // We have two possibilities for determining the right prefix boundary + + // Case 1. Find the largest possible prefix that firstIpAddress can be. + // Say we have an input range of 192.168.0.0 to 192.184.0.0. + // The number of IP addresses in the range is 1048576 = 2^20, so we would need a /12 (32-20). + // to cover that many IP addresses but the largest valid prefix that can start from 192.168.0.0 is /13. + int firstAddressMaxBits = firstIpAddress.getLowestSetBit(); + + // Case 2. Find the largest prefix length to cover N IP addresses. + // The number of IP addresses within a valid prefix must be a power of 2 but the IP count + // in our IP ranges may not be a power of 2. If it isn't exactly a power of 2, we find the + // highest power of 2 that the doesn't overrun the ipCount. + + // If ipCount's bitLength is greater than the number of IP addresses (i.e., not a power of 2), then use 1 bit less. + int ipRangeMaxBits = (TWO.pow(ipCount.bitLength()).compareTo(ipCount) > 0) ? ipCount.bitLength() - 1 : ipCount.bitLength(); + + return min(firstAddressMaxBits, ipRangeMaxBits); + } + + private static List mergeIpRanges(Block ipPrefixArray) + { + List mergedRanges = new ArrayList<>(); + + Slice startingIpPrefix = IPPREFIX.getSlice(ipPrefixArray, 0); + BigInteger firstIpAddress = toBigInteger(ipSubnetMin(startingIpPrefix)); + BigInteger lastIpAddress = toBigInteger(ipSubnetMax(startingIpPrefix)); + + /* + There are four cases to cover for two IP ranges where range1.startIp <= range2.startIp + + 1. Could be equal/duplicates. + [-------] + [-------] + In this case, we just ignore the second one. + + 2. Second could be subnet/contained within first. + [-------] OR [-------] OR [-------] + [---] [----] [----] + In this case we ignore the second one. + + 3. Second could be adjacent/contiguous with the first. + [-------] + [-------] + In this case we extend the range to include the last IP address of the second one. + + 4. Second can be disjoint from the first. + [-------] + [-------] + In this case the first range is finalized, and the second range becomes the current one. + */ + + for (int i = 1; i < ipPrefixArray.getPositionCount(); i++) { + Slice ipPrefix = IPPREFIX.getSlice(ipPrefixArray, i); + BigInteger nextFirstIpAddress = toBigInteger(ipSubnetMin(ipPrefix)); + BigInteger nextLastIpAddress = toBigInteger(ipSubnetMax(ipPrefix)); + + // If nextFirstIpAddress <= lastIpAddress then there is overlap. + // However, based on the properties of the input sorted array, this will + // always mean that the next* range is a subnet of [firstIpAddress, lastIpAddress]. + // We just ignore these prefixes since they are already covered (case 1 and case 2). + if (lastIpAddress.compareTo(nextFirstIpAddress) < 0) { // i.e. nextFirstIpAddress > lastIpAddress -- the next range does not overlap the first + // If they are not contiguous (case 4), finalize the range. + // Otherwise, extend the current range (case 3). + if (lastIpAddress.add(BigInteger.ONE).compareTo(nextFirstIpAddress) != 0) { + BigInteger[] finalizedRange = {firstIpAddress, lastIpAddress}; + mergedRanges.add(finalizedRange); + firstIpAddress = nextFirstIpAddress; + } + lastIpAddress = nextLastIpAddress; + } + } + + // Add the last range + BigInteger[] finalizedRange = {firstIpAddress, lastIpAddress}; + mergedRanges.add(finalizedRange); + + return mergedRanges; + } + + private static byte[] bigIntegerToIPAddressBytes(BigInteger ipAddress) + { + byte[] ipAddressBytes = ipAddress.toByteArray(); + + // Covers IPv4 (4 bytes) and IPv6 (16 bytes) plus an additional 0-value byte for sign + if ((ipAddressBytes.length == 5 || ipAddressBytes.length == 17) && ipAddressBytes[0] == 0) { + ipAddressBytes = Arrays.copyOfRange(ipAddressBytes, 1, ipAddressBytes.length); // remove leading 0 + } + // Covers IPv4 and IPv6 cases when BigInteger needs less than 4 or 16 bytes to represent + // the integer value. E.g., 0.0.0.1 will be 1 byte and 15.1.99.212 will be 3 bytes + else if (ipAddressBytes.length <= 3 || (ipAddressBytes.length != 4 && ipAddressBytes.length <= 15)) { + // start with zero'd out byte array and fill in starting at position j + byte[] emptyRange = new byte[ipAddressBytes.length <= 3 ? 4 : 16]; + int j = emptyRange.length - ipAddressBytes.length; + for (int i = 0; i < ipAddressBytes.length; i++, j++) { + emptyRange[j] = ipAddressBytes[i]; + } + ipAddressBytes = emptyRange; + } + // else length is already 4 or 16 + return ipAddressBytes; + } + + private static InetAddress bigIntegerToIpAddress(BigInteger ipAddress) throws UnknownHostException + { + byte[] ipAddressBytes = bigIntegerToIPAddressBytes(ipAddress); + return InetAddress.getByAddress(ipAddressBytes); + } + + private static BigInteger toBigInteger(Slice ipAddress) + { + // first param sets values to always be positive + return new BigInteger(1, ipAddress.getBytes()); + } + + private static boolean isIpv4(Slice ipPrefix) + { + // IPADDRESS types are 16 bytes for IPv4 and IPv6. IPv4 is stored as IPv4-mapped IPv6 addresses specified in RFC 4291. + // The IPv4 address is encoded into the low-order 32 bits of the IPv6 address, and the high-order 96 bits + // hold the fixed prefix 0:0:0:0:0:FFFF. + // To check if this is an IPv4 address, we check if the first 10 bytes are 0 and that bytes 11 and 12 are 0xFF. + byte[] ipPartBytes = ipPrefix.getBytes(0, 2 * Long.BYTES); + + for (int i = 0; i <= 9; i++) { + if (ipPartBytes[i] != (byte) 0) { + return false; + } + } + + return ipPartBytes[10] == (byte) 0xff && ipPartBytes[11] == (byte) 0xff; + } + private static InetAddress toInetAddress(Slice ipAddress) { try { diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestIpPrefixFunctions.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestIpPrefixFunctions.java index f30361c859d46..d1ed2f711af53 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestIpPrefixFunctions.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestIpPrefixFunctions.java @@ -118,4 +118,102 @@ public void testIsSubnetOf() assertFunction("IS_SUBNET_OF(IPPREFIX '170.0.52.0/22', IPPREFIX '170.0.52.0/24')", BOOLEAN, true); assertFunction("IS_SUBNET_OF(IPPREFIX '170.0.52.0/24', IPPREFIX '170.0.52.0/22')", BOOLEAN, false); } + + @Test + public void testIpv4PrefixCollapse() + { + // simple + assertFunction("IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '192.168.0.0/24', IPPREFIX '192.168.1.0/24'])", + new ArrayType(IPPREFIX), + ImmutableList.of("192.168.0.0/23")); + + // unsorted input, 1 adjacent prefix that cannot be aggregated, and one disjoint. + assertFunction("IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '192.168.1.0/24', IPPREFIX '192.168.0.0/24', IPPREFIX '192.168.2.0/24', IPPREFIX '192.168.9.0/24'])", + new ArrayType(IPPREFIX), + ImmutableList.of("192.168.0.0/23", "192.168.2.0/24", "192.168.9.0/24")); + } + + @Test + public void testIpv6PrefixCollapse() + { + // simple + assertFunction("IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '2620:10d:c090::/48', IPPREFIX '2620:10d:c091::/48'])", + new ArrayType(IPPREFIX), + ImmutableList.of("2620:10d:c090::/47")); + + // unsorted input, 1 adjacent prefix that cannot be aggregated, and one disjoint. + assertFunction("IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '2804:13c:4d6:e200::/56', IPPREFIX '2804:13c:4d6:dd00::/56', IPPREFIX '2804:13c:4d6:dc00::/56', IPPREFIX '2804:13c:4d6:de00::/56'])", + new ArrayType(IPPREFIX), + ImmutableList.of("2804:13c:4d6:dc00::/55", "2804:13c:4d6:de00::/56", "2804:13c:4d6:e200::/56")); + } + + @Test + public void testIpPrefixCollapseIpv4SingleIPs() + { + assertFunction("IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '192.168.0.1/32', IPPREFIX '192.168.33.1/32'])", + new ArrayType(IPPREFIX), + ImmutableList.of("192.168.0.1/32", "192.168.33.1/32")); + } + + @Test + public void testIpPrefixCollapseIpv6SingleIPs() + { + assertFunction("IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '2620:10d:c090:400::5:a869/128', IPPREFIX '2620:10d:c091:400::5:a869/128'])", + new ArrayType(IPPREFIX), + ImmutableList.of("2620:10d:c090:400::5:a869/128", "2620:10d:c091:400::5:a869/128")); + } + + @Test + public void testIpPrefixCollapseSinglePrefixReturnsSamePrefix() + { + assertFunction("IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '192.168.0.0/22'])", + new ArrayType(IPPREFIX), + ImmutableList.of("192.168.0.0/22")); + } + + @Test + public void testIpPrefixCollapseOverlappingPrefixes() + { + assertFunction("IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '192.168.0.0/22', IPPREFIX '192.168.0.0/24'])", + new ArrayType(IPPREFIX), + ImmutableList.of("192.168.0.0/22")); + assertFunction("IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '192.168.0.0/22', IPPREFIX '192.168.2.0/24'])", + new ArrayType(IPPREFIX), + ImmutableList.of("192.168.0.0/22")); + assertFunction("IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '192.168.0.0/22', IPPREFIX '192.168.3.0/24'])", + new ArrayType(IPPREFIX), + ImmutableList.of("192.168.0.0/22")); + assertFunction("IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '10.0.64.0/18', IPPREFIX '10.2.0.0/15', IPPREFIX '10.0.0.0/8', IPPREFIX '11.0.0.0/8', IPPREFIX '172.168.32.0/20', IPPREFIX '172.168.0.0/18'])", + new ArrayType(IPPREFIX), + ImmutableList.of("10.0.0.0/7", "172.168.0.0/18")); + assertFunction("IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '10.0.0.0/8', IPPREFIX '10.0.0.0/7'])", + new ArrayType(IPPREFIX), + ImmutableList.of("10.0.0.0/7")); + } + + @Test + public void testIpPrefixCollapseEmptyArrayInput() + { + assertFunction("IP_PREFIX_COLLAPSE(CAST(ARRAY[] AS ARRAY(IPPREFIX)))", new ArrayType(IPPREFIX), ImmutableList.of()); + } + + @Test + public void testIpPrefixCollapseNullInput() + { + assertFunction("IP_PREFIX_COLLAPSE(CAST(NULL AS ARRAY(IPPREFIX)))", new ArrayType(IPPREFIX), null); + } + + @Test + public void testIpPrefixCollapseNoNullPrefixesError() + { + assertInvalidFunction("IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '192.168.0.0/22', CAST(NULL AS IPPREFIX)])", + "ip_prefix_collapse does not support null elements"); + } + + @Test + public void testIpPrefixCollapseMixedIpVersionError() + { + assertInvalidFunction("IP_PREFIX_COLLAPSE(ARRAY[IPPREFIX '192.168.0.0/22', IPPREFIX '2409:4043:251a:d200::/56'])", + "All IPPREFIX elements must be the same IP version."); + } } From dec6c145692ac37c1d8f51bc2bec4a8f80d86d65 Mon Sep 17 00:00:00 2001 From: Karteekmurthys Date: Mon, 1 Jul 2024 23:01:24 +0530 Subject: [PATCH 19/31] [native] Prestissimo worker metrics documentation --- .../src/main/sphinx/presto_cpp/features.rst | 49 ++++++++++++++++--- .../src/main/sphinx/presto_cpp/properties.rst | 7 +++ presto-native-execution/README.md | 13 +++++ 3 files changed, 61 insertions(+), 8 deletions(-) diff --git a/presto-docs/src/main/sphinx/presto_cpp/features.rst b/presto-docs/src/main/sphinx/presto_cpp/features.rst index 28c2b2ff77d87..5b660a393ba8e 100644 --- a/presto-docs/src/main/sphinx/presto_cpp/features.rst +++ b/presto-docs/src/main/sphinx/presto_cpp/features.rst @@ -22,15 +22,36 @@ HTTP endpoints related to tasks are registered to Proxygen in Other HTTP endpoints include: -* POST: v1/memory - * Reports memory, but no assignments are adjusted unlike in Java workers. -* GET: v1/info -* GET: v1/status +* POST: v1/memory: Reports memory, but no assignments are adjusted unlike in Java workers +* GET: v1/info/metrics: Returns worker level metrics in Prometheus Data format. Refer section `Worker Metrics Collection <#worker-metrics-collection>`_ for more info. Here is a sample Metrics data returned by this API. -The request/response flow of Presto C++ is identical to Java workers. The -tasks or new splits are registered via `TaskUpdateRequest`. Resource -utilization and query progress are sent to the coordinator via task endpoints. + .. code-block:: text + # TYPE presto_cpp_num_http_request counter + presto_cpp_num_http_request{cluster="testing",worker=""} 0 + # TYPE presto_cpp_num_http_request_error counter + presto_cpp_num_http_request_error{cluster="testing",worker=""} 0 + # TYPE presto_cpp_memory_pushback_count counter + presto_cpp_memory_pushback_count{cluster="testing",worker=""} 0 + # TYPE velox_driver_yield_count counter + velox_driver_yield_count{cluster="testing",worker=""} 0 + # TYPE velox_cache_shrink_count counter + velox_cache_shrink_count{cluster="testing",worker=""} 0 + # TYPE velox_memory_cache_num_stale_entries counter + velox_memory_cache_num_stale_entries{cluster="testing",worker=""} 0 + # TYPE velox_arbitrator_requests_count counter + velox_arbitrator_requests_count{cluster="testing",worker=""} 0 + + +* GET: v1/info: Returns basic information about the worker. Here is an example: + + .. code-block:: text + + {"coordinator":false,"environment":"testing","nodeVersion":{"version":"testversion"},"starting":false,"uptime":"49.00s"} + +* GET: v1/status: Returns memory pool information. + +The request/response flow of Presto C++ is identical to Java workers. The tasks or new splits are registered via `TaskUpdateRequest`. Resource utilization and query progress are sent to the coordinator via task endpoints. Remote Function Execution ------------------------- @@ -169,7 +190,7 @@ Size of the SSD cache when async data cache is enabled. * **Default value:** ``true`` * **Presto on Spark default value:** ``false`` -Enable periodic clean up of old tasks. The default value is ``true`` for Presto C++. +Enable periodic clean up of old tasks. The default value is ``true`` for Presto C++. For Presto on Spark this property defaults to ``false``, as zombie or stuck tasks are handled by Spark by speculative execution. @@ -185,6 +206,18 @@ Old task is defined as a PrestoTask which has not received heartbeat for at leas ``old-task-cleanup-ms``, or is not running and has an end time more than ``old-task-cleanup-ms`` ago. +Worker metrics collection +------------------------- + +Users can enable collection of worker level metrics by setting the property: + +``runtime-metrics-collection-enabled`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +* **Type:** ``boolean`` +* **Default value:** ``false`` + + When true, the default behavior is a no-op. There is a prior setup that must be done before enabling this flag. To enable + metrics collection in Prometheus Data Format refer `here `_. Session Properties ------------------ diff --git a/presto-docs/src/main/sphinx/presto_cpp/properties.rst b/presto-docs/src/main/sphinx/presto_cpp/properties.rst index ff4cae57e02b5..be38fef8f0d6c 100644 --- a/presto-docs/src/main/sphinx/presto_cpp/properties.rst +++ b/presto-docs/src/main/sphinx/presto_cpp/properties.rst @@ -124,6 +124,13 @@ The configuration properties of Presto C++ workers are described here, in alphab 1) the non-reserved space in ``query-memory-gb`` is used up; and 2) the amount it tries to get is less than ``memory-pool-reserved-capacity``. +``runtime-metrics-collection-enabled`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +* **Type:** ``boolean`` +* **Default value:** ``false`` + + Enables collection of worker level metrics. + ``system-memory-gb`` ^^^^^^^^^^^^^^^^^^^^ diff --git a/presto-native-execution/README.md b/presto-native-execution/README.md index a5346e7b38b6b..1f2f001872272 100644 --- a/presto-native-execution/README.md +++ b/presto-native-execution/README.md @@ -67,6 +67,7 @@ Compilers (and versions) not mentioned are known to not work or have not been tr | CentOS 9/RHEL 9 | `gcc12` | ### Build Prestissimo +#### Parquet and S3 Supprt To enable Parquet and S3 support, set `PRESTO_ENABLE_PARQUET = "ON"`, `PRESTO_ENABLE_S3 = "ON"` in the environment. @@ -76,6 +77,7 @@ This dependency can be installed by running the script below from the `./velox/scripts/setup-adapters.sh aws` +#### JWT Authentication To enable JWT authentication support, set `PRESTO_ENABLE_JWT = "ON"` in the environment. @@ -85,6 +87,17 @@ This dependency can be installed by running the script below from the `./scripts/setup-adapters.sh jwt` +#### Worker Metrics Collection + +To enable worker level metrics collection and to enable the REST API `v1/info/metrics` +follow these steps: + +*Pre-build setup:* `./scripts/setup-adapters.sh prometheus` + +*CMake flags:* `PRESTO_STATS_REPORTER_TYPE=PROMETHEUS` + +*Runtime configuration:* `runtime-metrics-collection-enabled=true` + * After installing the above dependencies, from the `presto/presto-native-execution` directory, run `make` * For development, use From 0e6d3ae2b5e1a5356182cb04ea1f449ba7e2f1dd Mon Sep 17 00:00:00 2001 From: Zuyu ZHANG Date: Mon, 19 Aug 2024 19:56:35 +0800 Subject: [PATCH 20/31] [native] Init UuidParse in TaskManager due to boost UUID non-POD changes --- presto-native-execution/presto_cpp/main/TaskManager.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-native-execution/presto_cpp/main/TaskManager.cpp b/presto-native-execution/presto_cpp/main/TaskManager.cpp index cfbb26b386b29..338908e935f71 100644 --- a/presto-native-execution/presto_cpp/main/TaskManager.cpp +++ b/presto-native-execution/presto_cpp/main/TaskManager.cpp @@ -1038,7 +1038,7 @@ std::shared_ptr TaskManager::findOrCreateTask( UuidSplit split; }; - UuidParse uuid; + UuidParse uuid{}; uuid.uuid = boost::uuids::random_generator()(); prestoTask->info.taskStatus.taskInstanceIdLeastSignificantBits = From a8684698417bc1421df610c723137f4204fc4d8d Mon Sep 17 00:00:00 2001 From: Zuyu ZHANG Date: Tue, 20 Aug 2024 10:12:43 +0800 Subject: [PATCH 21/31] Init w/ a random UUID --- presto-native-execution/presto_cpp/main/TaskManager.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/presto-native-execution/presto_cpp/main/TaskManager.cpp b/presto-native-execution/presto_cpp/main/TaskManager.cpp index 338908e935f71..fd90ba2b75926 100644 --- a/presto-native-execution/presto_cpp/main/TaskManager.cpp +++ b/presto-native-execution/presto_cpp/main/TaskManager.cpp @@ -1038,8 +1038,7 @@ std::shared_ptr TaskManager::findOrCreateTask( UuidSplit split; }; - UuidParse uuid{}; - uuid.uuid = boost::uuids::random_generator()(); + UuidParse uuid = {boost::uuids::random_generator()()}; prestoTask->info.taskStatus.taskInstanceIdLeastSignificantBits = uuid.split.lo; From 6530afc68a9f0dc36d0479b9f58872847ece74eb Mon Sep 17 00:00:00 2001 From: Zac Wen Date: Tue, 20 Aug 2024 19:44:40 -0700 Subject: [PATCH 22/31] [native] Fix Velox connector configs update The variable needs to be a reference instead of a copy. This is necessary to ensure that changes made to the config are reflected in the query context. --- presto-native-execution/presto_cpp/main/QueryContextManager.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-native-execution/presto_cpp/main/QueryContextManager.cpp b/presto-native-execution/presto_cpp/main/QueryContextManager.cpp index e6c0d171f1838..deda8bafdf459 100644 --- a/presto-native-execution/presto_cpp/main/QueryContextManager.cpp +++ b/presto-native-execution/presto_cpp/main/QueryContextManager.cpp @@ -101,7 +101,7 @@ void updateVeloxConnectorConfigs( const auto& systemConfig = SystemConfig::instance(); for (auto& entry : connectorConfigStrings) { - auto connectorConfig = entry.second; + auto& connectorConfig = entry.second; // Do not retain cache if `node_selection_strategy` is explicitly set to // `NO_PREFERENCE`. From 7cb9526e4b5ffa69871a6a94deefb57251efef1c Mon Sep 17 00:00:00 2001 From: Amit Dutta Date: Tue, 13 Aug 2024 16:35:32 -0700 Subject: [PATCH 23/31] [native] Add a sample plan validator and e2e tests. --- .../presto_cpp/main/PrestoServer.cpp | 4 +- .../presto_cpp/main/PrestoServer.h | 6 +-- .../presto_cpp/main/TaskResource.h | 6 +-- .../presto_cpp/main/common/Configs.cpp | 1 + .../presto_cpp/main/common/Configs.h | 2 + .../presto_cpp/main/tests/TaskManagerTest.cpp | 3 +- .../presto_cpp/main/types/CMakeLists.txt | 5 +- .../main/types/VeloxPlanValidator.cpp | 49 +++++++++++++++++++ ...oxPlanValidator.h => VeloxPlanValidator.h} | 6 +-- .../AbstractTestNativePlanValidation.java | 29 +++++++++++ .../PrestoNativeQueryRunnerUtils.java | 33 +++++++++---- .../TestNativePlanValidation.java} | 16 +++--- 12 files changed, 127 insertions(+), 33 deletions(-) create mode 100644 presto-native-execution/presto_cpp/main/types/VeloxPlanValidator.cpp rename presto-native-execution/presto_cpp/main/types/{PrestoToVeloxPlanValidator.h => VeloxPlanValidator.h} (82%) create mode 100644 presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativePlanValidation.java rename presto-native-execution/{presto_cpp/main/types/PrestoToVeloxPlanValidator.cpp => src/test/java/com/facebook/presto/nativeworker/TestNativePlanValidation.java} (62%) diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.cpp b/presto-native-execution/presto_cpp/main/PrestoServer.cpp index 9c587fcc084ea..ef69a9b664c9f 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.cpp +++ b/presto-native-execution/presto_cpp/main/PrestoServer.cpp @@ -1326,10 +1326,10 @@ void PrestoServer::enableWorkerStatsReporting() { void PrestoServer::initPrestoToVeloxPlanValidator() { VELOX_CHECK_NULL(planValidator_); - planValidator_ = std::make_shared(); + planValidator_ = std::make_shared(); } -PrestoToVeloxPlanValidator* PrestoServer::getPlanValidator() { +VeloxPlanValidator* PrestoServer::getPlanValidator() { return planValidator_.get(); } diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.h b/presto-native-execution/presto_cpp/main/PrestoServer.h index f74b7dc65e72c..f2e03056cd08e 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.h +++ b/presto-native-execution/presto_cpp/main/PrestoServer.h @@ -25,7 +25,7 @@ #include "presto_cpp/main/PeriodicHeartbeatManager.h" #include "presto_cpp/main/PrestoExchangeSource.h" #include "presto_cpp/main/PrestoServerOperations.h" -#include "presto_cpp/main/types/PrestoToVeloxPlanValidator.h" +#include "presto_cpp/main/types/VeloxPlanValidator.h" #include "velox/common/caching/AsyncDataCache.h" #include "velox/common/memory/MemoryAllocator.h" #if __has_include("filesystem") @@ -184,7 +184,7 @@ class PrestoServer { /// Invoked to initialize Presto to Velox plan validator. virtual void initPrestoToVeloxPlanValidator(); - PrestoToVeloxPlanValidator* getPlanValidator(); + VeloxPlanValidator* getPlanValidator(); /// Invoked to get the list of filters passed to the http server. std::vector> @@ -246,7 +246,7 @@ class PrestoServer { // Executor for spilling. std::shared_ptr spillerExecutor_; - std::shared_ptr planValidator_; + std::shared_ptr planValidator_; std::unique_ptr exchangeSourceConnectionPool_; diff --git a/presto-native-execution/presto_cpp/main/TaskResource.h b/presto-native-execution/presto_cpp/main/TaskResource.h index 2fbe74d4af369..048588ff8b3f8 100644 --- a/presto-native-execution/presto_cpp/main/TaskResource.h +++ b/presto-native-execution/presto_cpp/main/TaskResource.h @@ -15,7 +15,7 @@ #include "presto_cpp/main/TaskManager.h" #include "presto_cpp/main/http/HttpServer.h" -#include "presto_cpp/main/types/PrestoToVeloxPlanValidator.h" +#include "presto_cpp/main/types/VeloxPlanValidator.h" #include "velox/common/memory/Memory.h" namespace facebook::presto { @@ -25,7 +25,7 @@ class TaskResource { explicit TaskResource( velox::memory::MemoryPool* pool, folly::Executor* httpSrvCpuExecutor, - PrestoToVeloxPlanValidator* planValidator, + VeloxPlanValidator* planValidator, TaskManager& taskManager) : httpSrvCpuExecutor_(httpSrvCpuExecutor), pool_{pool}, @@ -100,7 +100,7 @@ class TaskResource { folly::Executor* const httpSrvCpuExecutor_; velox::memory::MemoryPool* const pool_; - PrestoToVeloxPlanValidator* const planValidator_; + VeloxPlanValidator* const planValidator_; TaskManager& taskManager_; }; diff --git a/presto-native-execution/presto_cpp/main/common/Configs.cpp b/presto-native-execution/presto_cpp/main/common/Configs.cpp index fc6b4e366eca4..50e1bb52cb291 100644 --- a/presto-native-execution/presto_cpp/main/common/Configs.cpp +++ b/presto-native-execution/presto_cpp/main/common/Configs.cpp @@ -234,6 +234,7 @@ SystemConfig::SystemConfig() { STR_PROP(kCacheVeloxTtlThreshold, "2d"), STR_PROP(kCacheVeloxTtlCheckInterval, "1h"), BOOL_PROP(kEnableRuntimeMetricsCollection, false), + BOOL_PROP(kPlanValidatorFailOnNestedLoopJoin, false), }; } diff --git a/presto-native-execution/presto_cpp/main/common/Configs.h b/presto-native-execution/presto_cpp/main/common/Configs.h index 74fa387ef6d1e..b4b233a987c49 100644 --- a/presto-native-execution/presto_cpp/main/common/Configs.h +++ b/presto-native-execution/presto_cpp/main/common/Configs.h @@ -639,6 +639,8 @@ class SystemConfig : public ConfigBase { static constexpr std::string_view kSinkMaxBufferSize{"sink.max-buffer-size"}; static constexpr std::string_view kDriverMaxPagePartitioningBufferSize{ "driver.max-page-partitioning-buffer-size"}; + static constexpr std::string_view kPlanValidatorFailOnNestedLoopJoin{ + "velox-plan-validator-fail-on-nested-loop-join"}; SystemConfig(); diff --git a/presto-native-execution/presto_cpp/main/tests/TaskManagerTest.cpp b/presto-native-execution/presto_cpp/main/tests/TaskManagerTest.cpp index d940fb033f35a..1da6d2d482fbd 100644 --- a/presto-native-execution/presto_cpp/main/tests/TaskManagerTest.cpp +++ b/presto-native-execution/presto_cpp/main/tests/TaskManagerTest.cpp @@ -247,8 +247,7 @@ class TaskManagerTest : public testing::Test { taskManager_ = std::make_unique( driverExecutor_.get(), httpSrvCpuExecutor_.get(), nullptr); - auto validator = - std::make_shared(); + auto validator = std::make_shared(); taskResource_ = std::make_unique( leafPool_.get(), httpSrvCpuExecutor_.get(), diff --git a/presto-native-execution/presto_cpp/main/types/CMakeLists.txt b/presto-native-execution/presto_cpp/main/types/CMakeLists.txt index 167d1c8258634..ecdd1bb30b7ff 100644 --- a/presto-native-execution/presto_cpp/main/types/CMakeLists.txt +++ b/presto-native-execution/presto_cpp/main/types/CMakeLists.txt @@ -16,9 +16,8 @@ target_link_libraries(presto_type_converter velox_type_parser) add_library( presto_types OBJECT - PrestoToVeloxQueryPlan.cpp PrestoToVeloxExpr.cpp - PrestoToVeloxPlanValidator.cpp PrestoToVeloxSplit.cpp - PrestoToVeloxConnector.cpp) + PrestoToVeloxQueryPlan.cpp PrestoToVeloxExpr.cpp VeloxPlanValidator.cpp + PrestoToVeloxSplit.cpp PrestoToVeloxConnector.cpp) add_dependencies(presto_types presto_operators presto_type_converter velox_type velox_type_fbhive velox_dwio_dwrf_proto) diff --git a/presto-native-execution/presto_cpp/main/types/VeloxPlanValidator.cpp b/presto-native-execution/presto_cpp/main/types/VeloxPlanValidator.cpp new file mode 100644 index 0000000000000..1adba985c1cbb --- /dev/null +++ b/presto-native-execution/presto_cpp/main/types/VeloxPlanValidator.cpp @@ -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. + */ + +#include "presto_cpp/main/types/VeloxPlanValidator.h" +#include "presto_cpp/main/common/Configs.h" + +namespace facebook::presto { +bool planHasNestedJoinLoop(const velox::core::PlanNodePtr planNode) { + if (auto joinNode = + std::dynamic_pointer_cast( + planNode)) { + return true; + } + + for (auto plan : planNode->sources()) { + if (planHasNestedJoinLoop(plan)) { + return true; + } + } + + return false; +} + +void VeloxPlanValidator::validatePlanFragment( + const velox::core::PlanFragment& fragment) { + const auto failOnNestedLoopJoin = + SystemConfig::instance() + ->optionalProperty( + SystemConfig::kPlanValidatorFailOnNestedLoopJoin) + .value_or(false); + if (failOnNestedLoopJoin) { + VELOX_CHECK( + !planHasNestedJoinLoop(fragment.planNode), + "Velox plan uses nested join loop which isn't supported."); + } +} + +} // namespace facebook::presto diff --git a/presto-native-execution/presto_cpp/main/types/PrestoToVeloxPlanValidator.h b/presto-native-execution/presto_cpp/main/types/VeloxPlanValidator.h similarity index 82% rename from presto-native-execution/presto_cpp/main/types/PrestoToVeloxPlanValidator.h rename to presto-native-execution/presto_cpp/main/types/VeloxPlanValidator.h index 3ee4f6a35d5aa..810dc9c565aac 100644 --- a/presto-native-execution/presto_cpp/main/types/PrestoToVeloxPlanValidator.h +++ b/presto-native-execution/presto_cpp/main/types/VeloxPlanValidator.h @@ -15,9 +15,9 @@ #include "velox/core/PlanFragment.h" namespace facebook::presto { -class PrestoToVeloxPlanValidator { +class VeloxPlanValidator { public: - virtual bool validatePlanFragment(const velox::core::PlanFragment& fragment); - virtual ~PrestoToVeloxPlanValidator() = default; + virtual void validatePlanFragment(const velox::core::PlanFragment& fragment); + virtual ~VeloxPlanValidator() = default; }; } // namespace facebook::presto diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativePlanValidation.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativePlanValidation.java new file mode 100644 index 0000000000000..f207385e9c3bd --- /dev/null +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativePlanValidation.java @@ -0,0 +1,29 @@ +/* + * 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 com.facebook.presto.nativeworker; + +import com.facebook.presto.tests.AbstractTestQueryFramework; +import org.testng.annotations.Test; + +public abstract class AbstractTestNativePlanValidation + extends AbstractTestQueryFramework +{ + @Test + public void testNestedLoopJoinPlainValidationFailure() + { + assertQueryFails( + "SELECT EXISTS(SELECT 1 WHERE l.orderkey > 0 OR l.orderkey != 3) " + + "FROM lineitem l LIMIT 1", ".*Plan uses nested join loop which isn't supported.*"); + } +} diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/PrestoNativeQueryRunnerUtils.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/PrestoNativeQueryRunnerUtils.java index b4b8dba60eca5..69bb2433b3f18 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/PrestoNativeQueryRunnerUtils.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/PrestoNativeQueryRunnerUtils.java @@ -88,7 +88,7 @@ public static QueryRunner createQueryRunner( defaultQueryRunner.close(); - return createNativeQueryRunner(dataDirectory.get().toString(), prestoServerPath.get(), workerCount, cacheMaxSize, true, Optional.empty(), storageFormat, addStorageFormatToPath); + return createNativeQueryRunner(dataDirectory.get().toString(), prestoServerPath.get(), workerCount, cacheMaxSize, true, Optional.empty(), storageFormat, addStorageFormatToPath, false); } public static QueryRunner createJavaQueryRunner() @@ -251,7 +251,7 @@ public static QueryRunner createNativeIcebergQueryRunner( false, false, OptionalInt.of(workerCount.orElse(4)), - getExternalWorkerLauncher("iceberg", prestoServerPath, cacheMaxSize, remoteFunctionServerUds), + getExternalWorkerLauncher("iceberg", prestoServerPath, cacheMaxSize, remoteFunctionServerUds, false), dataDirectory, addStorageFormatToPath); } @@ -264,7 +264,8 @@ public static QueryRunner createNativeQueryRunner( boolean useThrift, Optional remoteFunctionServerUds, String storageFormat, - boolean addStorageFormatToPath) + boolean addStorageFormatToPath, + Boolean failOnNestedLoopJoin) throws Exception { // The property "hive.allow-drop-table" needs to be set to true because security is always "legacy" in NativeQueryRunner. @@ -287,7 +288,7 @@ public static QueryRunner createNativeQueryRunner( hiveProperties, workerCount, Optional.of(Paths.get(addStorageFormatToPath ? dataDirectory + "/" + storageFormat : dataDirectory)), - getExternalWorkerLauncher("hive", prestoServerPath, cacheMaxSize, remoteFunctionServerUds)); + getExternalWorkerLauncher("hive", prestoServerPath, cacheMaxSize, remoteFunctionServerUds, failOnNestedLoopJoin)); } public static QueryRunner createNativeCteQueryRunner(boolean useThrift, String storageFormat) @@ -330,13 +331,13 @@ public static QueryRunner createNativeCteQueryRunner(boolean useThrift, String s hiveProperties, workerCount, Optional.of(Paths.get(addStorageFormatToPath ? dataDirectory + "/" + storageFormat : dataDirectory)), - getExternalWorkerLauncher("hive", prestoServerPath, cacheMaxSize, Optional.empty())); + getExternalWorkerLauncher("hive", prestoServerPath, cacheMaxSize, Optional.empty(), false)); } public static QueryRunner createNativeQueryRunner(String remoteFunctionServerUds) throws Exception { - return createNativeQueryRunner(false, DEFAULT_STORAGE_FORMAT, Optional.ofNullable(remoteFunctionServerUds)); + return createNativeQueryRunner(false, DEFAULT_STORAGE_FORMAT, Optional.ofNullable(remoteFunctionServerUds), false); } public static QueryRunner createNativeQueryRunner(boolean useThrift) @@ -345,13 +346,19 @@ public static QueryRunner createNativeQueryRunner(boolean useThrift) return createNativeQueryRunner(useThrift, DEFAULT_STORAGE_FORMAT); } + public static QueryRunner createNativeQueryRunner(boolean useThrift, boolean failOnNestedLoopJoin) + throws Exception + { + return createNativeQueryRunner(useThrift, DEFAULT_STORAGE_FORMAT, Optional.empty(), failOnNestedLoopJoin); + } + public static QueryRunner createNativeQueryRunner(boolean useThrift, String storageFormat) throws Exception { - return createNativeQueryRunner(useThrift, storageFormat, Optional.empty()); + return createNativeQueryRunner(useThrift, storageFormat, Optional.empty(), false); } - public static QueryRunner createNativeQueryRunner(boolean useThrift, String storageFormat, Optional remoteFunctionServerUds) + public static QueryRunner createNativeQueryRunner(boolean useThrift, String storageFormat, Optional remoteFunctionServerUds, Boolean failOnNestedLoopJoin) throws Exception { int cacheMaxSize = 0; @@ -364,7 +371,8 @@ public static QueryRunner createNativeQueryRunner(boolean useThrift, String stor useThrift, remoteFunctionServerUds, storageFormat, - true); + true, + failOnNestedLoopJoin); } // Start the remote function server. Return the UDS path used to communicate with it. @@ -411,7 +419,7 @@ public static NativeQueryRunnerParameters getNativeQueryRunnerParameters() return new NativeQueryRunnerParameters(prestoServerPath, dataDirectory, workerCount); } - public static Optional> getExternalWorkerLauncher(String catalogName, String prestoServerPath, int cacheMaxSize, Optional remoteFunctionServerUds) + public static Optional> getExternalWorkerLauncher(String catalogName, String prestoServerPath, int cacheMaxSize, Optional remoteFunctionServerUds, Boolean failOnNestedLoopJoin) { return Optional.of((workerIndex, discoveryUri) -> { @@ -436,6 +444,11 @@ public static Optional> getExternalWorkerLaunc "remote-function-server.serde=presto_page%n" + "remote-function-server.signature.files.directory.path=%s%n", configProperties, REMOTE_FUNCTION_CATALOG_NAME, remoteFunctionServerUds.get(), jsonSignaturesPath); } + + if (failOnNestedLoopJoin) { + configProperties = format("%s%n" + "velox-plan-validator-fail-on-nested-loop-join=true%n", configProperties); + } + Files.write(tempDirectoryPath.resolve("config.properties"), configProperties.getBytes()); Files.write(tempDirectoryPath.resolve("node.properties"), format("node.id=%s%n" + diff --git a/presto-native-execution/presto_cpp/main/types/PrestoToVeloxPlanValidator.cpp b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/TestNativePlanValidation.java similarity index 62% rename from presto-native-execution/presto_cpp/main/types/PrestoToVeloxPlanValidator.cpp rename to presto-native-execution/src/test/java/com/facebook/presto/nativeworker/TestNativePlanValidation.java index 4ef9a8179aefb..707d336caf595 100644 --- a/presto-native-execution/presto_cpp/main/types/PrestoToVeloxPlanValidator.cpp +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/TestNativePlanValidation.java @@ -11,12 +11,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package com.facebook.presto.nativeworker; +import com.facebook.presto.testing.QueryRunner; -#include "presto_cpp/main/types/PrestoToVeloxPlanValidator.h" - -namespace facebook::presto { -bool PrestoToVeloxPlanValidator::validatePlanFragment( - const velox::core::PlanFragment& fragment) { - return true; +public class TestNativePlanValidation + extends AbstractTestNativePlanValidation +{ + protected QueryRunner createQueryRunner() throws Exception + { + return PrestoNativeQueryRunnerUtils.createNativeQueryRunner(false, true); + } } -} // namespace facebook::presto From 14da6b7754aba39b5aae3203c94b1e0dabedfba3 Mon Sep 17 00:00:00 2001 From: Amit Dutta Date: Tue, 20 Aug 2024 23:52:58 -0700 Subject: [PATCH 24/31] [native] Advance velox. --- presto-native-execution/velox | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-native-execution/velox b/presto-native-execution/velox index 54b16e0416ada..55107b6f7438f 160000 --- a/presto-native-execution/velox +++ b/presto-native-execution/velox @@ -1 +1 @@ -Subproject commit 54b16e0416ada85dd2904b2d2bee7650a1d27b36 +Subproject commit 55107b6f7438f59789752e32f0327c0b6ba739ec From 73f58cc1a749d78045bc6739168b0c8c8ceee7fb Mon Sep 17 00:00:00 2001 From: Amit Dutta Date: Wed, 21 Aug 2024 10:29:15 -0700 Subject: [PATCH 25/31] [native] Minor renames and spell fixes. --- presto-native-execution/presto_cpp/main/PrestoServer.cpp | 8 ++++---- presto-native-execution/presto_cpp/main/PrestoServer.h | 4 ++-- .../nativeworker/AbstractTestNativeRemoteFunctions.java | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.cpp b/presto-native-execution/presto_cpp/main/PrestoServer.cpp index ef69a9b664c9f..0e663bbf0dfd1 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.cpp +++ b/presto-native-execution/presto_cpp/main/PrestoServer.cpp @@ -497,11 +497,11 @@ void PrestoServer::run() { << "Spilling root directory: " << baseSpillDirectory; } - initPrestoToVeloxPlanValidator(); + initVeloxPlanValidator(); taskResource_ = std::make_unique( pool_.get(), httpSrvCpuExecutor_.get(), - getPlanValidator(), + getVeloxPlanValidator(), *taskManager_); taskResource_->registerUris(*httpServer_); if (systemConfig->enableSerializedPageChecksum()) { @@ -1324,12 +1324,12 @@ void PrestoServer::enableWorkerStatsReporting() { registerStatsCounters(); } -void PrestoServer::initPrestoToVeloxPlanValidator() { +void PrestoServer::initVeloxPlanValidator() { VELOX_CHECK_NULL(planValidator_); planValidator_ = std::make_shared(); } -VeloxPlanValidator* PrestoServer::getPlanValidator() { +VeloxPlanValidator* PrestoServer::getVeloxPlanValidator() { return planValidator_.get(); } diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.h b/presto-native-execution/presto_cpp/main/PrestoServer.h index f2e03056cd08e..bee2d8d43391a 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.h +++ b/presto-native-execution/presto_cpp/main/PrestoServer.h @@ -182,9 +182,9 @@ class PrestoServer { virtual void enableWorkerStatsReporting(); /// Invoked to initialize Presto to Velox plan validator. - virtual void initPrestoToVeloxPlanValidator(); + virtual void initVeloxPlanValidator(); - VeloxPlanValidator* getPlanValidator(); + VeloxPlanValidator* getVeloxPlanValidator(); /// Invoked to get the list of filters passed to the http server. std::vector> diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeRemoteFunctions.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeRemoteFunctions.java index 121d47d57b109..7eff8b2954245 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeRemoteFunctions.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeRemoteFunctions.java @@ -29,7 +29,7 @@ public abstract class AbstractTestNativeRemoteFunctions extends AbstractTestQueryFramework { - // The unix domain socket (UDS) path to communicate with the remote fuction server. + // The unix domain socket (UDS) path to communicate with the remote function server. protected String remoteFunctionServerUds; // The path to the compiled remote function server binary. From 171e57de961ed865926f6b049dfb1dc10a9e9653 Mon Sep 17 00:00:00 2001 From: Reetika Agrawal Date: Wed, 21 Aug 2024 21:19:33 +0530 Subject: [PATCH 26/31] Fix sql block in Iceberg document --- presto-docs/src/main/sphinx/connector/iceberg.rst | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/presto-docs/src/main/sphinx/connector/iceberg.rst b/presto-docs/src/main/sphinx/connector/iceberg.rst index 831d961b0839a..5253bb3acc95b 100644 --- a/presto-docs/src/main/sphinx/connector/iceberg.rst +++ b/presto-docs/src/main/sphinx/connector/iceberg.rst @@ -1329,8 +1329,7 @@ Example Queries ^^^^^^^^^^^^^^^ Similar to the example queries in `SCHEMA EVOLUTION`_, create an Iceberg -table named `ctas_nation` from the TPCH `nation` table:: - +table named `ctas_nation` from the TPCH `nation` table: .. code-block:: sql From d69f1f8a09e643b5260dd28eb325ce545db8bddc Mon Sep 17 00:00:00 2001 From: Reetika Agrawal Date: Wed, 21 Aug 2024 22:02:17 +0530 Subject: [PATCH 27/31] Refactor dead links in the documents --- presto-docs/src/main/sphinx/connector/clickhouse.rst | 5 ++--- presto-docs/src/main/sphinx/connector/googlesheets.rst | 3 +-- presto-docs/src/main/sphinx/installation/jdbc.rst | 4 ++++ 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/presto-docs/src/main/sphinx/connector/clickhouse.rst b/presto-docs/src/main/sphinx/connector/clickhouse.rst index 6849578dfba9c..338d98836727b 100644 --- a/presto-docs/src/main/sphinx/connector/clickhouse.rst +++ b/presto-docs/src/main/sphinx/connector/clickhouse.rst @@ -143,9 +143,8 @@ SQL support ----------- The connector provides read and write access to data and metadata in -a ClickHouse catalog. In addition to the :ref:`globally available -` and :ref:`read operation ` -statements, the connector supports the following features: +a ClickHouse catalog. In addition to the globally available and +read operation statements, the connector supports the following features: * :doc:`/sql/insert` * :doc:`/sql/truncate` diff --git a/presto-docs/src/main/sphinx/connector/googlesheets.rst b/presto-docs/src/main/sphinx/connector/googlesheets.rst index 7e1fb515e58ff..94869d0640a1c 100644 --- a/presto-docs/src/main/sphinx/connector/googlesheets.rst +++ b/presto-docs/src/main/sphinx/connector/googlesheets.rst @@ -99,6 +99,5 @@ fetching the sheet data for every table, unless it is already cached. SQL support ----------- -The connector provides :ref:`globally available ` and -:ref:`read operation ` statements to access data and +The connector provides globally available and read operation statements to access data and metadata in Google Sheets. diff --git a/presto-docs/src/main/sphinx/installation/jdbc.rst b/presto-docs/src/main/sphinx/installation/jdbc.rst index a1393a264799f..5722d1a795e75 100644 --- a/presto-docs/src/main/sphinx/installation/jdbc.rst +++ b/presto-docs/src/main/sphinx/installation/jdbc.rst @@ -50,6 +50,8 @@ The above URL can be used as follows to create a connection: String url = "jdbc:presto://example.net:8080/hive/sales"; Connection connection = DriverManager.getConnection(url, "test", null); +.. _jdbc-java-connection: + Connection Parameters --------------------- @@ -75,6 +77,8 @@ These methods may be mixed; some parameters may be specified in the URL while others are specified using properties. However, the same parameter may not be specified using both methods. +.. _jdbc-parameter-reference: + Parameter Reference ------------------- From 81066131c0e6c9f4185fa4d22d3fbc10bd2b91bf Mon Sep 17 00:00:00 2001 From: wangd Date: Sun, 18 Aug 2024 21:55:12 +0800 Subject: [PATCH 28/31] Fix analyze table whose column name ends with `_` --- .../iceberg/IcebergDistributedTestBase.java | 24 +++++++++---------- .../planner/StatisticsAggregationPlanner.java | 16 +++++++++---- .../sql/relational/SqlFunctionUtils.java | 18 +++++++++++++- 3 files changed, 40 insertions(+), 18 deletions(-) diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java index 498e28aa15d59..a4e2b079b72c3 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java @@ -769,9 +769,9 @@ public void testStringFilters() @Test public void testReadWriteStats() { - assertUpdate("CREATE TABLE test_stats (col0 int, col1 varchar)"); + assertUpdate("CREATE TABLE test_stats (col0 int, col_1 varchar)"); assertTrue(getQueryRunner().tableExists(getSession(), "test_stats")); - assertTableColumnNames("test_stats", "col0", "col1"); + assertTableColumnNames("test_stats", "col0", "col_1"); // test that stats don't exist before analyze Function, Map> remapper = (input) -> input.entrySet().stream().collect(Collectors.toMap(e -> ((IcebergColumnHandle) e.getKey()).getName(), Map.Entry::getValue)); @@ -788,9 +788,9 @@ public void testReadWriteStats() ColumnStatistics columnStat = columnStats.get("col0"); assertEquals(columnStat.getDistinctValuesCount(), Estimate.of(3.0)); assertEquals(columnStat.getDataSize(), Estimate.unknown()); - columnStat = columnStats.get("col1"); + columnStat = columnStats.get("col_1"); assertEquals(columnStat.getDistinctValuesCount(), Estimate.of(3.0)); - double dataSize = (double) (long) getQueryRunner().execute("SELECT sum_data_size_for_stats(col1) FROM test_stats").getOnlyValue(); + double dataSize = (double) (long) getQueryRunner().execute("SELECT sum_data_size_for_stats(col_1) FROM test_stats").getOnlyValue(); assertEquals(columnStat.getDataSize().getValue(), dataSize); // test after inserting the same values, we still get the same estimate @@ -800,7 +800,7 @@ public void testReadWriteStats() columnStat = columnStats.get("col0"); assertEquals(columnStat.getDistinctValuesCount(), Estimate.of(3.0)); assertEquals(columnStat.getDataSize(), Estimate.unknown()); - columnStat = columnStats.get("col1"); + columnStat = columnStats.get("col_1"); assertEquals(columnStat.getDistinctValuesCount(), Estimate.of(3.0)); assertEquals(columnStat.getDataSize().getValue(), dataSize); @@ -811,9 +811,9 @@ public void testReadWriteStats() columnStat = columnStats.get("col0"); assertEquals(columnStat.getDistinctValuesCount(), Estimate.of(3.0)); assertEquals(columnStat.getDataSize(), Estimate.unknown()); - columnStat = columnStats.get("col1"); + columnStat = columnStats.get("col_1"); assertEquals(columnStat.getDistinctValuesCount(), Estimate.of(3.0)); - dataSize = (double) (long) getQueryRunner().execute("SELECT sum_data_size_for_stats(col1) FROM test_stats").getOnlyValue(); + dataSize = (double) (long) getQueryRunner().execute("SELECT sum_data_size_for_stats(col_1) FROM test_stats").getOnlyValue(); assertEquals(columnStat.getDataSize().getValue(), dataSize); // test after inserting a new value, but not analyzing, the estimate is the same. @@ -823,7 +823,7 @@ public void testReadWriteStats() columnStat = columnStats.get("col0"); assertEquals(columnStat.getDistinctValuesCount(), Estimate.of(3.0)); assertEquals(columnStat.getDataSize(), Estimate.unknown()); - columnStat = columnStats.get("col1"); + columnStat = columnStats.get("col_1"); assertEquals(columnStat.getDistinctValuesCount(), Estimate.of(3.0)); assertEquals(columnStat.getDataSize().getValue(), dataSize); @@ -834,9 +834,9 @@ public void testReadWriteStats() columnStat = columnStats.get("col0"); assertEquals(columnStat.getDistinctValuesCount(), Estimate.of(4.0)); assertEquals(columnStat.getDataSize(), Estimate.unknown()); - columnStat = columnStats.get("col1"); + columnStat = columnStats.get("col_1"); assertEquals(columnStat.getDistinctValuesCount(), Estimate.of(4.0)); - dataSize = (double) (long) getQueryRunner().execute("SELECT sum_data_size_for_stats(col1) FROM test_stats").getOnlyValue(); + dataSize = (double) (long) getQueryRunner().execute("SELECT sum_data_size_for_stats(col_1) FROM test_stats").getOnlyValue(); assertEquals(columnStat.getDataSize().getValue(), dataSize); // test adding a null value is successful, and analyze still runs successfully @@ -847,9 +847,9 @@ public void testReadWriteStats() columnStat = columnStats.get("col0"); assertEquals(columnStat.getDistinctValuesCount(), Estimate.of(4.0)); assertEquals(columnStat.getDataSize(), Estimate.unknown()); - columnStat = columnStats.get("col1"); + columnStat = columnStats.get("col_1"); assertEquals(columnStat.getDistinctValuesCount(), Estimate.of(4.0)); - dataSize = (double) (long) getQueryRunner().execute("SELECT sum_data_size_for_stats(col1) FROM test_stats").getOnlyValue(); + dataSize = (double) (long) getQueryRunner().execute("SELECT sum_data_size_for_stats(col_1) FROM test_stats").getOnlyValue(); assertEquals(columnStat.getDataSize().getValue(), dataSize); assertUpdate("DROP TABLE test_stats"); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java index f9722664ecebb..305ad85f77b0b 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java @@ -109,7 +109,8 @@ public TableStatisticAggregation createStatisticsAggregation(TableStatisticsMeta ColumnStatisticType statisticType = columnStatisticMetadata.getStatisticType(); VariableReferenceExpression inputVariable = columnToVariableMap.get(columnName); verify(inputVariable != null, "inputVariable is null"); - ColumnStatisticsAggregation aggregation = createColumnAggregation(columnStatisticMetadata, inputVariable); + ColumnStatisticsAggregation aggregation = createColumnAggregation(columnStatisticMetadata, inputVariable, + ImmutableMap.of(columnName, inputVariable.getName())); additionalVariables.putAll(aggregation.getInputProjections()); VariableReferenceExpression variable = variableAllocator.newVariable(statisticType + ":" + columnName, aggregation.getOutputType()); aggregations.put(variable, aggregation.getAggregation()); @@ -120,13 +121,17 @@ public TableStatisticAggregation createStatisticsAggregation(TableStatisticsMeta return new TableStatisticAggregation(aggregation, descriptor.build(), additionalVariables.build()); } - private ColumnStatisticsAggregation createColumnAggregationFromSqlFunction(String sqlFunction, VariableReferenceExpression input) + private ColumnStatisticsAggregation createColumnAggregationFromSqlFunction( + String sqlFunction, + VariableReferenceExpression input, + Map columnNameToInputVariableNameMap) { RowExpression expression = sqlFunctionToRowExpression( sqlFunction, ImmutableSet.of(input), functionAndTypeManager, - session); + session, + columnNameToInputVariableNameMap); verify(expression instanceof CallExpression, "column statistic SQL expressions must represent a function call"); CallExpression call = (CallExpression) expression; FunctionMetadata functionMeta = functionAndTypeResolver.getFunctionMetadata(call.getFunctionHandle()); @@ -186,10 +191,11 @@ private ColumnStatisticsAggregation createColumnAggregationFromFunctionName(Colu ImmutableMap.of()); } - private ColumnStatisticsAggregation createColumnAggregation(ColumnStatisticMetadata columnStatisticMetadata, VariableReferenceExpression input) + private ColumnStatisticsAggregation createColumnAggregation(ColumnStatisticMetadata columnStatisticMetadata, VariableReferenceExpression input, + Map columnNameToInputVariableNameMap) { if (columnStatisticMetadata.isSqlExpression()) { - return createColumnAggregationFromSqlFunction(columnStatisticMetadata.getFunction(), input); + return createColumnAggregationFromSqlFunction(columnStatisticMetadata.getFunction(), input, columnNameToInputVariableNameMap); } return createColumnAggregationFromFunctionName(columnStatisticMetadata, input); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlFunctionUtils.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlFunctionUtils.java index fdbcca5d44bbe..64796474cc7f5 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlFunctionUtils.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlFunctionUtils.java @@ -118,11 +118,27 @@ public static RowExpression getSqlFunctionRowExpression( public static RowExpression sqlFunctionToRowExpression(String functionBody, Set variables, FunctionAndTypeManager functionAndTypeManager, - Session session) + Session session, + Map columnNameToInputVariableNameMap) { Expression expression = parseSqlFunctionExpression( new SqlInvokedScalarFunctionImplementation(functionBody), session.getSqlFunctionProperties()); + + // Translate the parameter name in functionBody to input variable name + expression = ExpressionTreeRewriter.rewriteWith(new ExpressionRewriter>() + { + @Override + public Expression rewriteIdentifier(Identifier node, Map context, ExpressionTreeRewriter> treeRewriter) + { + String name = node.getValueLowerCase(); + if (context.containsKey(name)) { + return new Identifier(context.get(name)); + } + return node; + } + }, expression, columnNameToInputVariableNameMap); + return SqlToRowExpressionTranslator.translate( expression, analyzeSqlFunctionExpression( From 1ab72db57eb8ab439140c6a056b69b6635767625 Mon Sep 17 00:00:00 2001 From: wypb Date: Wed, 21 Aug 2024 19:29:16 +0800 Subject: [PATCH 29/31] [native] Fix docker-compose and add etc config into native worker docker image --- presto-native-execution/docker-compose.yml | 6 +++--- .../scripts/dockerfiles/prestissimo-runtime.dockerfile | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/presto-native-execution/docker-compose.yml b/presto-native-execution/docker-compose.yml index 248ba9dfdf19e..534e2a8d79a95 100644 --- a/presto-native-execution/docker-compose.yml +++ b/presto-native-execution/docker-compose.yml @@ -42,16 +42,16 @@ services: # Usage: # docker compose build centos-native-dependency # podman compose build centos-native-dependency - image: presto/prestissimo-dependency:centos8 + image: presto/prestissimo-dependency:centos9 build: context: . - dockerfile: scripts/dockerfiles/centos-8-stream-dependency.dockerfile + dockerfile: scripts/dockerfiles/centos-dependency.dockerfile centos-native-runtime: # Usage: # docker compose build centos-native-runtime # podman compose build centos-native-runtime - image: presto/prestissimo-runtime:centos8 + image: presto/prestissimo-runtime:centos9 build: args: # A few files in Velox require significant memory to compile and link. diff --git a/presto-native-execution/scripts/dockerfiles/prestissimo-runtime.dockerfile b/presto-native-execution/scripts/dockerfiles/prestissimo-runtime.dockerfile index 602c0f2fcb141..7ae303f21f27d 100644 --- a/presto-native-execution/scripts/dockerfiles/prestissimo-runtime.dockerfile +++ b/presto-native-execution/scripts/dockerfiles/prestissimo-runtime.dockerfile @@ -41,6 +41,7 @@ ENV BUILD_DIR="" COPY --chmod=0775 --from=prestissimo-image /prestissimo/${BUILD_BASE_DIR}/${BUILD_DIR}/presto_cpp/main/presto_server /usr/bin/ COPY --chmod=0775 --from=prestissimo-image /runtime-libraries/* /usr/lib64/prestissimo-libs/ +COPY --chmod=0755 ./etc /opt/presto-server/etc COPY --chmod=0775 ./entrypoint.sh /opt/entrypoint.sh RUN echo "/usr/lib64/prestissimo-libs" > /etc/ld.so.conf.d/prestissimo.conf && ldconfig From 67e423a7672cb011251cdff94789c0a580504c82 Mon Sep 17 00:00:00 2001 From: wangd Date: Thu, 22 Aug 2024 23:54:19 +0800 Subject: [PATCH 30/31] [Iceberg]Fix cleanup files behavior on expiring snapshots --- .../procedure/ExpireSnapshotsProcedure.java | 6 +++- .../org/apache/iceberg/IcebergLibUtils.java | 34 +++++++++++++++++++ .../iceberg/IcebergDistributedTestBase.java | 33 ++++++++++++++++++ .../nessie/TestIcebergDistributedNessie.java | 10 ++++++ 4 files changed, 82 insertions(+), 1 deletion(-) create mode 100644 presto-iceberg/src/main/java/org/apache/iceberg/IcebergLibUtils.java diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/procedure/ExpireSnapshotsProcedure.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/procedure/ExpireSnapshotsProcedure.java index 0487f3c2b6103..afbc4793b48bc 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/procedure/ExpireSnapshotsProcedure.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/procedure/ExpireSnapshotsProcedure.java @@ -36,6 +36,7 @@ import static com.facebook.presto.common.type.StandardTypes.TIMESTAMP; import static com.facebook.presto.common.type.StandardTypes.VARCHAR; import static java.util.Objects.requireNonNull; +import static org.apache.iceberg.IcebergLibUtils.withIncrementalCleanup; public class ExpireSnapshotsProcedure implements Provider @@ -85,7 +86,10 @@ private void doExpireSnapshots(ConnectorSession clientSession, String schema, St SchemaTableName schemaTableName = new SchemaTableName(schema, tableName); Table icebergTable = IcebergUtil.getIcebergTable(metadata, clientSession, schemaTableName); - ExpireSnapshots expireSnapshots = icebergTable.expireSnapshots(); + // Incremental clean up strategy has a bug when expire specified snapshots. + // So explicitly use reachable file cleanup strategy here. + // Referring to https://github.com/apache/iceberg/issues/10982 + ExpireSnapshots expireSnapshots = withIncrementalCleanup(icebergTable.expireSnapshots(), false); if (snapshotIds != null) { for (long id : snapshotIds) { diff --git a/presto-iceberg/src/main/java/org/apache/iceberg/IcebergLibUtils.java b/presto-iceberg/src/main/java/org/apache/iceberg/IcebergLibUtils.java new file mode 100644 index 0000000000000..ba0511e9d4e81 --- /dev/null +++ b/presto-iceberg/src/main/java/org/apache/iceberg/IcebergLibUtils.java @@ -0,0 +1,34 @@ +/* + * 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 org.apache.iceberg; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class IcebergLibUtils +{ + private IcebergLibUtils() + {} + + /** + * Call the method in Iceberg lib's protected class to set explicitly + * whether to use incremental cleanup when expiring snapshots + * */ + public static ExpireSnapshots withIncrementalCleanup(ExpireSnapshots expireSnapshots, boolean incrementalCleanup) + { + requireNonNull(expireSnapshots, "expireSnapshots is null"); + checkArgument(expireSnapshots instanceof RemoveSnapshots, "expireSnapshots is not an instance of RemoveSnapshots"); + return ((RemoveSnapshots) expireSnapshots).withIncrementalCleanup(incrementalCleanup); + } +} diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java index a4e2b079b72c3..61b9bde207fa8 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedTestBase.java @@ -1706,6 +1706,39 @@ public void testAllIcebergType() } } + @Test + public void testExpireSnapshotWithDeletedEntries() + { + try { + assertUpdate("create table test_expire_snapshot_with_deleted_entry (a int, b varchar) with (partitioning = ARRAY['a'])"); + assertUpdate("insert into test_expire_snapshot_with_deleted_entry values(1, '1001'), (1, '1002'), (2, '2001'), (2, '2002')", 4); + Table table = loadTable("test_expire_snapshot_with_deleted_entry"); + long snapshotId1 = table.currentSnapshot().snapshotId(); + + // Execute metadata deletion which delete whole files from table metadata + assertUpdate("delete from test_expire_snapshot_with_deleted_entry where a = 1", 2); + table = loadTable("test_expire_snapshot_with_deleted_entry"); + long snapshotId2 = table.currentSnapshot().snapshotId(); + + assertUpdate("insert into test_expire_snapshot_with_deleted_entry values(1, '1003'), (2, '2003'), (3, '3003')", 3); + table = loadTable("test_expire_snapshot_with_deleted_entry"); + long snapshotId3 = table.currentSnapshot().snapshotId(); + + assertQuery("select snapshot_id from \"test_expire_snapshot_with_deleted_entry$snapshots\"", "values " + snapshotId1 + ", " + snapshotId2 + ", " + snapshotId3); + + // Expire `snapshotId2` which contains a DELETED entry to delete a data file which is still referenced by `snapshotId1` + assertUpdate(format("call iceberg.system.expire_snapshots(schema => '%s', table_name => '%s', snapshot_ids => ARRAY[%d])", "tpch", "test_expire_snapshot_with_deleted_entry", snapshotId2)); + assertQuery("select snapshot_id from \"test_expire_snapshot_with_deleted_entry$snapshots\"", "values " + snapshotId1 + ", " + snapshotId3); + + // Execute time travel query successfully + assertQuery("select * from test_expire_snapshot_with_deleted_entry for version as of " + snapshotId1, "values(1, '1001'), (1, '1002'), (2, '2001'), (2, '2002')"); + assertQuery("select * from test_expire_snapshot_with_deleted_entry for version as of " + snapshotId3, "values(1, '1003'), (2, '2001'), (2, '2002'), (2, '2003'), (3, '3003')"); + } + finally { + assertUpdate("drop table if exists test_expire_snapshot_with_deleted_entry"); + } + } + private void testCheckDeleteFiles(Table icebergTable, int expectedSize, List expectedFileContent) { // check delete file list diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/nessie/TestIcebergDistributedNessie.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/nessie/TestIcebergDistributedNessie.java index d1b7c66fe0bf0..b97b791939442 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/nessie/TestIcebergDistributedNessie.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/nessie/TestIcebergDistributedNessie.java @@ -27,6 +27,7 @@ import static com.facebook.presto.iceberg.CatalogType.NESSIE; import static com.facebook.presto.iceberg.nessie.NessieTestUtil.nessieConnectorProperties; +import static org.assertj.core.api.Assertions.assertThatThrownBy; @Test public class TestIcebergDistributedNessie @@ -76,4 +77,13 @@ protected QueryRunner createQueryRunner() { return IcebergQueryRunner.createIcebergQueryRunner(ImmutableMap.of(), nessieConnectorProperties(nessieContainer.getRestApiUri())); } + + @Override + public void testExpireSnapshotWithDeletedEntries() + { + // Nessie do not support expire snapshots as it set table property `gc.enabled` to `false` by default + assertThatThrownBy(() -> super.testExpireSnapshotWithDeletedEntries()) + .isInstanceOf(RuntimeException.class) + .hasMessageMatching("Cannot expire snapshots: GC is disabled .*"); + } } From 797019c5d05fd8ab752135a7b08544b64036d161 Mon Sep 17 00:00:00 2001 From: Ke Date: Tue, 20 Aug 2024 15:43:10 -0700 Subject: [PATCH 31/31] [native] Refactor TaskManagerTest to not create common::WriterOptions instance After velox refactoring, we should not create dwio::common::WriterOptions instance anymore. Update TaskManagerTest.cpp --- .../presto_cpp/main/tests/TaskManagerTest.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/presto-native-execution/presto_cpp/main/tests/TaskManagerTest.cpp b/presto-native-execution/presto_cpp/main/tests/TaskManagerTest.cpp index 1da6d2d482fbd..e7f68a7c7782d 100644 --- a/presto-native-execution/presto_cpp/main/tests/TaskManagerTest.cpp +++ b/presto-native-execution/presto_cpp/main/tests/TaskManagerTest.cpp @@ -29,6 +29,7 @@ #include "velox/dwio/common/FileSink.h" #include "velox/dwio/common/WriterFactory.h" #include "velox/dwio/common/tests/utils/BatchMaker.h" +#include "velox/dwio/dwrf/writer/Writer.h" #include "velox/exec/Exchange.h" #include "velox/exec/Values.h" #include "velox/exec/tests/utils/PlanBuilder.h" @@ -269,6 +270,8 @@ class TaskManagerTest : public testing::Test { "http://{}:{}", serverAddress.getAddressStr(), serverAddress.getPort())); + writerFactory_ = + dwio::common::getWriterFactory(dwio::common::FileFormat::DWRF); } void TearDown() override { @@ -298,13 +301,13 @@ class TaskManagerTest : public testing::Test { void writeToFile( const std::string& filePath, const std::vector& vectors) { - auto options = std::make_shared(); + auto options = writerFactory_->createWriterOptions(); options->schema = rowType_; options->memoryPool = rootPool_.get(); auto sink = std::make_unique( filePath, dwio::common::FileSink::Options{}); - auto writer = dwio::common::getWriterFactory(dwio::common::FileFormat::DWRF) - ->createWriter(std::move(sink), options); + auto writer = + writerFactory_->createWriter(std::move(sink), std::move(options)); for (size_t i = 0; i < vectors.size(); ++i) { writer->write(vectors[i]); @@ -673,6 +676,7 @@ class TaskManagerTest : public testing::Test { long splitSequenceId_{0}; std::shared_ptr connPool_ = std::make_shared(); + std::shared_ptr writerFactory_; }; // Runs "select * from t where c0 % 5 = 0" query.