From d2364ad0d5d4aa03052d4d0e52470920b70e2788 Mon Sep 17 00:00:00 2001 From: Pratik Joseph Dabre Date: Tue, 22 Oct 2024 13:00:37 -0700 Subject: [PATCH 1/8] Allow workers to act as coordinator sidecars Co-authored-by: Tim Meehan --- .../presto/connector/ConnectorAwareNodeManager.java | 9 +++------ .../facebook/presto/execution/ClusterSizeMonitor.java | 10 ++-------- .../facebook/presto/nodeManager/PluginNodeManager.java | 9 +++------ .../presto/execution/TestClusterSizeMonitor.java | 6 +++--- 4 files changed, 11 insertions(+), 23 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/connector/ConnectorAwareNodeManager.java b/presto-main/src/main/java/com/facebook/presto/connector/ConnectorAwareNodeManager.java index 3506d55393f2..b0b092b4a306 100644 --- a/presto-main/src/main/java/com/facebook/presto/connector/ConnectorAwareNodeManager.java +++ b/presto-main/src/main/java/com/facebook/presto/connector/ConnectorAwareNodeManager.java @@ -20,12 +20,12 @@ import com.facebook.presto.spi.NodeManager; import com.facebook.presto.spi.PrestoException; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import java.util.Random; import java.util.Set; import static com.facebook.presto.spi.StandardErrorCode.NO_CPP_SIDECARS; -import static com.facebook.presto.spi.StandardErrorCode.TOO_MANY_SIDECARS; -import static com.google.common.collect.Iterables.getOnlyElement; import static java.util.Objects.requireNonNull; public class ConnectorAwareNodeManager @@ -70,10 +70,7 @@ public Node getSidecarNode() if (coordinatorSidecars.isEmpty()) { throw new PrestoException(NO_CPP_SIDECARS, "Expected exactly one coordinator sidecar, but found none"); } - if (coordinatorSidecars.size() > 1) { - throw new PrestoException(TOO_MANY_SIDECARS, "Expected exactly one coordinator sidecar, but found " + coordinatorSidecars.size()); - } - return getOnlyElement(coordinatorSidecars); + return Iterables.get(coordinatorSidecars, new Random().nextInt(coordinatorSidecars.size())); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/execution/ClusterSizeMonitor.java b/presto-main/src/main/java/com/facebook/presto/execution/ClusterSizeMonitor.java index 396d6d575f2d..6833912e31c1 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/ClusterSizeMonitor.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/ClusterSizeMonitor.java @@ -40,7 +40,6 @@ import static com.facebook.airlift.concurrent.Threads.threadsNamed; import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INSUFFICIENT_RESOURCES; import static com.facebook.presto.spi.StandardErrorCode.NO_CPP_SIDECARS; -import static com.facebook.presto.spi.StandardErrorCode.TOO_MANY_SIDECARS; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.util.concurrent.Futures.immediateFuture; import static java.lang.String.format; @@ -183,11 +182,7 @@ public boolean hasRequiredCoordinators() */ public boolean hasRequiredCoordinatorSidecars() { - if (currentCoordinatorSidecarCount > 1) { - throw new PrestoException(TOO_MANY_SIDECARS, - format("Expected a single active coordinator sidecar. Found %s active coordinator sidecars", currentCoordinatorSidecarCount)); - } - return currentCoordinatorSidecarCount == 1; + return currentCoordinatorSidecarCount > 0; } /** @@ -257,7 +252,7 @@ public synchronized ListenableFuture waitForMinimumCoordinators() public synchronized ListenableFuture waitForMinimumCoordinatorSidecars() { - if (currentCoordinatorSidecarCount == 1 || !isCoordinatorSidecarEnabled) { + if (currentCoordinatorSidecarCount > 0 || !isCoordinatorSidecarEnabled) { return immediateFuture(null); } @@ -309,7 +304,6 @@ private synchronized void updateAllNodes(AllNodes allNodes) Set activeNodes = new HashSet<>(allNodes.getActiveNodes()); activeNodes.removeAll(allNodes.getActiveCoordinators()); activeNodes.removeAll(allNodes.getActiveResourceManagers()); - activeNodes.removeAll(allNodes.getActiveCoordinatorSidecars()); currentWorkerCount = activeNodes.size(); } currentCoordinatorCount = allNodes.getActiveCoordinators().size(); diff --git a/presto-main/src/main/java/com/facebook/presto/nodeManager/PluginNodeManager.java b/presto-main/src/main/java/com/facebook/presto/nodeManager/PluginNodeManager.java index 34c2a1a3ccaa..ffa4b37a7726 100644 --- a/presto-main/src/main/java/com/facebook/presto/nodeManager/PluginNodeManager.java +++ b/presto-main/src/main/java/com/facebook/presto/nodeManager/PluginNodeManager.java @@ -19,14 +19,14 @@ import com.facebook.presto.spi.NodeManager; import com.facebook.presto.spi.PrestoException; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import com.google.inject.Inject; +import java.util.Random; import java.util.Set; import static com.facebook.presto.spi.StandardErrorCode.NO_CPP_SIDECARS; -import static com.facebook.presto.spi.StandardErrorCode.TOO_MANY_SIDECARS; import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static com.google.common.collect.Iterables.getOnlyElement; import static java.util.Objects.requireNonNull; /** @@ -83,10 +83,7 @@ public Node getSidecarNode() if (coordinatorSidecars.isEmpty()) { throw new PrestoException(NO_CPP_SIDECARS, "Expected exactly one coordinator sidecar, but found none"); } - if (coordinatorSidecars.size() > 1) { - throw new PrestoException(TOO_MANY_SIDECARS, "Expected exactly one coordinator sidecar, but found " + coordinatorSidecars.size()); - } - return getOnlyElement(coordinatorSidecars); + return Iterables.get(coordinatorSidecars, new Random().nextInt(coordinatorSidecars.size())); } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/execution/TestClusterSizeMonitor.java b/presto-main/src/test/java/com/facebook/presto/execution/TestClusterSizeMonitor.java index de0ce3e05fd1..9ef10e188e0d 100644 --- a/presto-main/src/test/java/com/facebook/presto/execution/TestClusterSizeMonitor.java +++ b/presto-main/src/test/java/com/facebook/presto/execution/TestClusterSizeMonitor.java @@ -17,7 +17,6 @@ import com.facebook.presto.metadata.InMemoryNodeManager; import com.facebook.presto.metadata.InternalNode; import com.facebook.presto.spi.ConnectorId; -import com.facebook.presto.spi.PrestoException; import com.google.common.util.concurrent.ListenableFuture; import io.airlift.units.Duration; import org.testng.annotations.AfterMethod; @@ -170,7 +169,7 @@ public void testHasRequiredCoordinatorSidecars() assertTrue(monitor.hasRequiredCoordinatorSidecars()); } - @Test(expectedExceptions = PrestoException.class, expectedExceptionsMessageRegExp = "Expected a single active coordinator sidecar. Found 2 active coordinator sidecars") + @Test public void testHasRequiredCoordinatorSidecarsMoreThanOne() throws InterruptedException { @@ -178,7 +177,7 @@ public void testHasRequiredCoordinatorSidecarsMoreThanOne() for (int i = numCoordinatorSidecars.get(); i < DESIRED_COORDINATOR_SIDECAR_COUNT + 1; i++) { addCoordinatorSidecar(nodeManager); } - assertFalse(monitor.hasRequiredCoordinatorSidecars()); + assertTrue(monitor.hasRequiredCoordinatorSidecars()); } @Test @@ -223,6 +222,7 @@ private ListenableFuture waitForMinimumCoordinatorSidecars() addSuccessCallback(coordinatorSidecarsFuture, () -> { assertFalse(coordinatorSidecarsTimeout.get()); minCoordinatorSidecarsLatch.countDown(); + minCoordinatorSidecarsLatch.countDown(); }); addExceptionCallback(coordinatorSidecarsFuture, () -> { assertTrue(coordinatorSidecarsTimeout.compareAndSet(false, true)); From 730fbd088bde998e5ac8818a0867e3db59f9f304 Mon Sep 17 00:00:00 2001 From: Pratik Joseph Dabre Date: Tue, 22 Oct 2024 13:28:47 -0700 Subject: [PATCH 2/8] Separate out worker session properties and introduce JavaFeaturesConfig Co-authored-by: Joe Abraham --- .../presto/druid/TestDruidQueryBase.java | 4 +- .../presto/iceberg/TestIcebergFileWriter.java | 3 +- .../presto/SystemSessionProperties.java | 361 +----------------- .../system/SystemConnectorSessionUtil.java | 5 +- .../presto/dispatcher/DispatchManager.java | 4 +- .../presto/metadata/MetadataManager.java | 3 +- .../metadata/SessionPropertyManager.java | 121 +++++- .../GenericAccumulatorFactory.java | 6 +- .../facebook/presto/server/PrestoServer.java | 2 + .../presto/server/ServerMainModule.java | 20 + .../JavaWorkerSessionPropertyProvider.java | 205 ++++++++++ .../NativeWorkerSessionPropertyProvider.java | 236 ++++++++++++ .../TempStorageSingleStreamSpiller.java | 2 +- .../TempStorageStandaloneSpillerFactory.java | 2 +- .../presto/sql/analyzer/FeaturesConfig.java | 152 -------- .../sql/analyzer/JavaFeaturesConfig.java | 176 +++++++++ .../sql/planner/LocalExecutionPlanner.java | 19 +- .../presto/testing/LocalQueryRunner.java | 11 +- .../presto/testing/TestingSession.java | 3 +- .../dispatcher/TestLocalDispatchQuery.java | 6 +- .../TestAdaptivePhasedExecutionPolicy.java | 4 +- ...stResourceManagerClusterStateProvider.java | 24 +- .../server/TestQuerySessionSupplier.java | 6 +- .../server/TestSessionPropertyDefaults.java | 14 +- .../presto/sql/analyzer/TestAnalyzer.java | 6 +- .../sql/analyzer/TestFeaturesConfig.java | 33 -- .../sql/analyzer/TestJavaFeaturesConfig.java | 76 ++++ .../iterative/rule/test/RuleTester.java | 3 +- .../sanity/TestValidateStreamingJoins.java | 4 +- .../AbstractTestNativeGeneralQueries.java | 43 ++- .../TestPrestoSparkNativeGeneralQueries.java | 8 + .../presto/pinot/TestPinotQueryBase.java | 4 +- .../presto/spark/PrestoSparkModule.java | 2 + ...toSparkSessionPropertyManagerProvider.java | 7 +- ...arkPhysicalResourceAllocationStrategy.java | 6 +- .../spi/session/SessionPropertyContext.java | 43 +++ .../WorkerSessionPropertyProvider.java | 22 ++ .../WorkerSessionPropertyProviderFactory.java | 21 + .../presto/tests/AbstractTestQueries.java | 34 ++ .../tests/TestDistributedSpilledQueries.java | 6 +- ...tributedSpilledQueriesWithTempStorage.java | 3 +- .../presto/tests/TestLocalQueries.java | 5 + .../tests/TestQueryPlanDeterminism.java | 5 + ...gationWithHighMemoryRevokingThreshold.java | 6 +- ...regationWithLargeBlockSpillingEnabled.java | 10 +- ...edAggregationWithPreprocessingEnabled.java | 6 +- .../presto/tests/TestSpilledAggregations.java | 6 +- .../tests/TestSpilledOrderByQueries.java | 2 +- .../presto/tests/TestSpilledTopNQueries.java | 5 +- .../tests/TestSpilledWindowQueries.java | 2 +- .../tests/TestVerboseOptimizerInfo.java | 6 + 51 files changed, 1136 insertions(+), 627 deletions(-) create mode 100644 presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/JavaWorkerSessionPropertyProvider.java create mode 100644 presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java create mode 100644 presto-main/src/main/java/com/facebook/presto/sql/analyzer/JavaFeaturesConfig.java create mode 100644 presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestJavaFeaturesConfig.java create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/session/SessionPropertyContext.java create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/session/WorkerSessionPropertyProvider.java create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/session/WorkerSessionPropertyProviderFactory.java diff --git a/presto-druid/src/test/java/com/facebook/presto/druid/TestDruidQueryBase.java b/presto-druid/src/test/java/com/facebook/presto/druid/TestDruidQueryBase.java index f1fabf17c495..428d6d7bf300 100644 --- a/presto-druid/src/test/java/com/facebook/presto/druid/TestDruidQueryBase.java +++ b/presto-druid/src/test/java/com/facebook/presto/druid/TestDruidQueryBase.java @@ -20,7 +20,6 @@ import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.MetadataManager; -import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorId; import com.facebook.presto.spi.ConnectorSession; @@ -62,6 +61,7 @@ import static com.facebook.presto.druid.DruidQueryGeneratorContext.Origin.DERIVED; import static com.facebook.presto.druid.DruidQueryGeneratorContext.Origin.TABLE_COLUMN; import static com.facebook.presto.metadata.FunctionAndTypeManager.createTestFunctionAndTypeManager; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.spi.plan.LimitNode.Step.FINAL; import static com.facebook.presto.sql.analyzer.ExpressionAnalyzer.getExpressionTypes; import static com.facebook.presto.testing.TestingConnectorSession.SESSION; @@ -109,7 +109,7 @@ protected static class SessionHolder public SessionHolder() { connectorSession = SESSION; - session = TestingSession.testSessionBuilder(new SessionPropertyManager(new SystemSessionProperties().getSessionProperties())).build(); + session = TestingSession.testSessionBuilder(createTestingSessionPropertyManager(new SystemSessionProperties().getSessionProperties())).build(); } public ConnectorSession getConnectorSession() diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergFileWriter.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergFileWriter.java index e96b0a501d78..734d47172ded 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergFileWriter.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergFileWriter.java @@ -63,6 +63,7 @@ import static com.facebook.presto.iceberg.IcebergAbstractMetadata.toIcebergSchema; import static com.facebook.presto.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static com.facebook.presto.iceberg.IcebergSessionProperties.dataSizeSessionProperty; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static com.google.common.io.Files.createTempDir; import static org.apache.iceberg.parquet.ParquetSchemaUtil.convert; @@ -78,7 +79,7 @@ public class TestIcebergFileWriter public void setup() throws Exception { ConnectorId connectorId = new ConnectorId("iceberg"); - SessionPropertyManager sessionPropertyManager = new SessionPropertyManager(); + SessionPropertyManager sessionPropertyManager = createTestingSessionPropertyManager(); sessionPropertyManager.addConnectorSessionProperties( connectorId, diff --git a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java index c22c7e80f06a..dd0070fba695 100644 --- a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java +++ b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java @@ -152,19 +152,7 @@ public final class SystemSessionProperties public static final String TREAT_LOW_CONFIDENCE_ZERO_ESTIMATION_AS_UNKNOWN_ENABLED = "treat_low_confidence_zero_estimation_unknown_enabled"; public static final String SPILL_ENABLED = "spill_enabled"; public static final String JOIN_SPILL_ENABLED = "join_spill_enabled"; - public static final String AGGREGATION_SPILL_ENABLED = "aggregation_spill_enabled"; - public static final String TOPN_SPILL_ENABLED = "topn_spill_enabled"; - public static final String DISTINCT_AGGREGATION_SPILL_ENABLED = "distinct_aggregation_spill_enabled"; - public static final String DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED = "dedup_based_distinct_aggregation_spill_enabled"; - public static final String DISTINCT_AGGREGATION_LARGE_BLOCK_SPILL_ENABLED = "distinct_aggregation_large_block_spill_enabled"; - public static final String DISTINCT_AGGREGATION_LARGE_BLOCK_SIZE_THRESHOLD = "distinct_aggregation_large_block_size_threshold"; - public static final String ORDER_BY_AGGREGATION_SPILL_ENABLED = "order_by_aggregation_spill_enabled"; - public static final String WINDOW_SPILL_ENABLED = "window_spill_enabled"; - public static final String ORDER_BY_SPILL_ENABLED = "order_by_spill_enabled"; - public static final String AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT = "aggregation_operator_unspill_memory_limit"; - public static final String TOPN_OPERATOR_UNSPILL_MEMORY_LIMIT = "topn_operator_unspill_memory_limit"; public static final String QUERY_MAX_REVOCABLE_MEMORY_PER_NODE = "query_max_revocable_memory_per_node"; - public static final String TEMP_STORAGE_SPILLER_BUFFER_SIZE = "temp_storage_spiller_buffer_size"; public static final String OPTIMIZE_DISTINCT_AGGREGATIONS = "optimize_mixed_distinct_aggregations"; public static final String LEGACY_ROW_FIELD_ORDINAL_ACCESS = "legacy_row_field_ordinal_access"; public static final String LEGACY_MAP_SUBSCRIPT = "do_not_use_legacy_map_subscript"; @@ -332,48 +320,18 @@ public final class SystemSessionProperties public static final String PRINT_ESTIMATED_STATS_FROM_CACHE = "print_estimated_stats_from_cache"; public static final String REMOVE_CROSS_JOIN_WITH_CONSTANT_SINGLE_ROW_INPUT = "remove_cross_join_with_constant_single_row_input"; public static final String EAGER_PLAN_VALIDATION_ENABLED = "eager_plan_validation_enabled"; + public static final String DEFAULT_VIEW_SECURITY_MODE = "default_view_security_mode"; + public static final String JOIN_PREFILTER_BUILD_SIDE = "join_prefilter_build_side"; + public static final String OPTIMIZER_USE_HISTOGRAMS = "optimizer_use_histograms"; + public static final String WARN_ON_COMMON_NAN_PATTERNS = "warn_on_common_nan_patterns"; + public static final String INLINE_PROJECTIONS_ON_VALUES = "inline_projections_on_values"; // TODO: Native execution related session properties that are temporarily put here. They will be relocated in the future. - public static final String NATIVE_SIMPLIFIED_EXPRESSION_EVALUATION_ENABLED = "native_simplified_expression_evaluation_enabled"; - public static final String NATIVE_EXPRESSION_MAX_ARRAY_SIZE_IN_REDUCE = "native_expression_max_array_size_in_reduce"; public static final String NATIVE_AGGREGATION_SPILL_ALL = "native_aggregation_spill_all"; - public static final String NATIVE_MAX_SPILL_LEVEL = "native_max_spill_level"; - public static final String NATIVE_MAX_SPILL_FILE_SIZE = "native_max_spill_file_size"; - public static final String NATIVE_SPILL_COMPRESSION_CODEC = "native_spill_compression_codec"; - public static final String NATIVE_SPILL_WRITE_BUFFER_SIZE = "native_spill_write_buffer_size"; - public static final String NATIVE_SPILL_FILE_CREATE_CONFIG = "native_spill_file_create_config"; - public static final String NATIVE_JOIN_SPILL_ENABLED = "native_join_spill_enabled"; - public static final String NATIVE_WINDOW_SPILL_ENABLED = "native_window_spill_enabled"; - public static final String NATIVE_WRITER_SPILL_ENABLED = "native_writer_spill_enabled"; - public static final String NATIVE_ROW_NUMBER_SPILL_ENABLED = "native_row_number_spill_enabled"; - public static final String NATIVE_TOPN_ROW_NUMBER_SPILL_ENABLED = "native_topn_row_number_spill_enabled"; - public static final String NATIVE_SPILLER_NUM_PARTITION_BITS = "native_spiller_num_partition_bits"; private static final String NATIVE_EXECUTION_ENABLED = "native_execution_enabled"; private static final String NATIVE_EXECUTION_EXECUTABLE_PATH = "native_execution_executable_path"; private static final String NATIVE_EXECUTION_PROGRAM_ARGUMENTS = "native_execution_program_arguments"; public static final String NATIVE_EXECUTION_PROCESS_REUSE_ENABLED = "native_execution_process_reuse_enabled"; - public static final String NATIVE_DEBUG_VALIDATE_OUTPUT_FROM_OPERATORS = "native_debug_validate_output_from_operators"; - public static final String NATIVE_DEBUG_DISABLE_EXPRESSION_WITH_PEELING = "native_debug_disable_expression_with_peeling"; - public static final String NATIVE_DEBUG_DISABLE_COMMON_SUB_EXPRESSION = "native_debug_disable_common_sub_expressions"; - public static final String NATIVE_DEBUG_DISABLE_EXPRESSION_WITH_MEMOIZATION = "native_debug_disable_expression_with_memoization"; - public static final String NATIVE_DEBUG_DISABLE_EXPRESSION_WITH_LAZY_INPUTS = "native_debug_disable_expression_with_lazy_inputs"; - public static final String NATIVE_SELECTIVE_NIMBLE_READER_ENABLED = "native_selective_nimble_reader_enabled"; - public static final String NATIVE_MAX_PARTIAL_AGGREGATION_MEMORY = "native_max_partial_aggregation_memory"; - public static final String NATIVE_MAX_EXTENDED_PARTIAL_AGGREGATION_MEMORY = "native_max_extended_partial_aggregation_memory"; - public static final String NATIVE_MAX_SPILL_BYTES = "native_max_spill_bytes"; - public static final String NATIVE_QUERY_TRACE_ENABLED = "native_query_trace_enabled"; - public static final String NATIVE_QUERY_TRACE_DIR = "native_query_trace_dir"; - public static final String NATIVE_QUERY_TRACE_NODE_IDS = "native_query_trace_node_ids"; - public static final String NATIVE_QUERY_TRACE_MAX_BYTES = "native_query_trace_max_bytes"; - public static final String NATIVE_QUERY_TRACE_REG_EXP = "native_query_trace_task_reg_exp"; - public static final String NATIVE_MAX_PAGE_PARTITIONING_BUFFER_SIZE = "native_max_page_partitioning_buffer_size"; - public static final String NATIVE_MAX_OUTPUT_BUFFER_SIZE = "native_max_output_buffer_size"; - - public static final String DEFAULT_VIEW_SECURITY_MODE = "default_view_security_mode"; - public static final String JOIN_PREFILTER_BUILD_SIDE = "join_prefilter_build_side"; - public static final String OPTIMIZER_USE_HISTOGRAMS = "optimizer_use_histograms"; - public static final String WARN_ON_COMMON_NAN_PATTERNS = "warn_on_common_nan_patterns"; - public static final String INLINE_PROJECTIONS_ON_VALUES = "inline_projections_on_values"; private final List> sessionProperties; @@ -822,73 +780,6 @@ public SystemSessionProperties( "Enable join spilling", featuresConfig.isJoinSpillingEnabled(), false), - booleanProperty( - AGGREGATION_SPILL_ENABLED, - "Enable aggregate spilling if spill_enabled", - featuresConfig.isAggregationSpillEnabled(), - false), - booleanProperty( - TOPN_SPILL_ENABLED, - "Enable topN spilling if spill_enabled", - featuresConfig.isTopNSpillEnabled(), - false), - booleanProperty( - DISTINCT_AGGREGATION_SPILL_ENABLED, - "Enable spill for distinct aggregations if spill_enabled and aggregation_spill_enabled", - featuresConfig.isDistinctAggregationSpillEnabled(), - false), - booleanProperty( - DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED, - "Perform deduplication of input data for distinct aggregates before spilling", - featuresConfig.isDedupBasedDistinctAggregationSpillEnabled(), - false), - booleanProperty( - DISTINCT_AGGREGATION_LARGE_BLOCK_SPILL_ENABLED, - "Spill large block to a separate spill file", - featuresConfig.isDistinctAggregationLargeBlockSpillEnabled(), - false), - new PropertyMetadata<>( - DISTINCT_AGGREGATION_LARGE_BLOCK_SIZE_THRESHOLD, - "Block size threshold beyond which it will be spilled into a separate spill file", - VARCHAR, - DataSize.class, - featuresConfig.getDistinctAggregationLargeBlockSizeThreshold(), - false, - value -> DataSize.valueOf((String) value), - DataSize::toString), - booleanProperty( - ORDER_BY_AGGREGATION_SPILL_ENABLED, - "Enable spill for order-by aggregations if spill_enabled and aggregation_spill_enabled", - featuresConfig.isOrderByAggregationSpillEnabled(), - false), - booleanProperty( - WINDOW_SPILL_ENABLED, - "Enable window spilling if spill_enabled", - featuresConfig.isWindowSpillEnabled(), - false), - booleanProperty( - ORDER_BY_SPILL_ENABLED, - "Enable order by spilling if spill_enabled", - featuresConfig.isOrderBySpillEnabled(), - false), - new PropertyMetadata<>( - AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT, - "Experimental: How much memory can should be allocated per aggregation operator in unspilling process", - VARCHAR, - DataSize.class, - featuresConfig.getAggregationOperatorUnspillMemoryLimit(), - false, - value -> DataSize.valueOf((String) value), - DataSize::toString), - new PropertyMetadata<>( - TOPN_OPERATOR_UNSPILL_MEMORY_LIMIT, - "How much memory can should be allocated per topN operator in unspilling process", - VARCHAR, - DataSize.class, - featuresConfig.getTopNOperatorUnspillMemoryLimit(), - false, - value -> DataSize.valueOf((String) value), - DataSize::toString), new PropertyMetadata<>( QUERY_MAX_REVOCABLE_MEMORY_PER_NODE, "Maximum amount of revocable memory a query can use", @@ -898,15 +789,6 @@ public SystemSessionProperties( true, value -> DataSize.valueOf((String) value), DataSize::toString), - new PropertyMetadata<>( - TEMP_STORAGE_SPILLER_BUFFER_SIZE, - "Experimental: Buffer size used by TempStorageSingleStreamSpiller", - VARCHAR, - DataSize.class, - nodeSpillConfig.getTempStorageBufferSize(), - false, - value -> DataSize.valueOf((String) value), - DataSize::toString), booleanProperty( OPTIMIZE_DISTINCT_AGGREGATIONS, "Optimize mixed non-distinct and distinct aggregations", @@ -1646,16 +1528,6 @@ public SystemSessionProperties( "Combine individual approx_percentile calls on individual field to evaluation on an array", featuresConfig.isOptimizeMultipleApproxPercentileOnSameFieldEnabled(), false), - booleanProperty( - NATIVE_SIMPLIFIED_EXPRESSION_EVALUATION_ENABLED, - "Native Execution only. Enable simplified path in expression evaluation", - false, - false), - integerProperty( - NATIVE_EXPRESSION_MAX_ARRAY_SIZE_IN_REDUCE, - "Native Execution only. Reduce() function will throw an error if it encounters an array of size greater than this value.", - 100000, - false), booleanProperty( NATIVE_AGGREGATION_SPILL_ALL, "Native Execution only. If true and spilling has been triggered during the input " + @@ -1664,160 +1536,16 @@ public SystemSessionProperties( "output processing stage.", true, false), - integerProperty( - NATIVE_MAX_SPILL_LEVEL, - "Native Execution only. The maximum allowed spilling level for hash join build.\n" + - "0 is the initial spilling level, -1 means unlimited.", - 4, - false), - integerProperty( - NATIVE_MAX_SPILL_FILE_SIZE, - "The max allowed spill file size. If it is zero, then there is no limit.", - 0, - false), - stringProperty( - NATIVE_SPILL_COMPRESSION_CODEC, - "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.", - "zstd", - false), - longProperty( - NATIVE_SPILL_WRITE_BUFFER_SIZE, - "Native Execution only. The maximum size in bytes to buffer the serialized spill data before writing to disk for IO efficiency.\n" + - "If set to zero, buffering is disabled.", - 1024L * 1024L, - false), - stringProperty( - NATIVE_SPILL_FILE_CREATE_CONFIG, - "Native Execution only. Config used to create spill files. This config is \n" + - "provided to underlying file system and the config is free form. The form should be\n" + - "defined by the underlying file system.", - "", - false), - booleanProperty( - NATIVE_JOIN_SPILL_ENABLED, - "Native Execution only. Enable join spilling on native engine", - false, - false), - booleanProperty( - NATIVE_WINDOW_SPILL_ENABLED, - "Native Execution only. Enable window spilling on native engine", - false, - false), - booleanProperty( - NATIVE_WRITER_SPILL_ENABLED, - "Native Execution only. Enable writer spilling on native engine", - false, - false), - booleanProperty( - NATIVE_ROW_NUMBER_SPILL_ENABLED, - "Native Execution only. Enable row number spilling on native engine", - false, - false), booleanProperty( - NATIVE_TOPN_ROW_NUMBER_SPILL_ENABLED, - "Native Execution only. Enable topN row number spilling on native engine", - false, - false), - integerProperty( - NATIVE_SPILLER_NUM_PARTITION_BITS, - "Native Execution only. The number of bits (N) used to calculate the " + - "spilling partition number for hash join and RowNumber: 2 ^ N", - 3, - false), + NATIVE_EXECUTION_ENABLED, + "Enable execution on native engine", + featuresConfig.isNativeExecutionEnabled(), + true), booleanProperty( NATIVE_EXECUTION_PROCESS_REUSE_ENABLED, "Enable reuse the native process within the same JVM", true, false), - booleanProperty( - NATIVE_DEBUG_VALIDATE_OUTPUT_FROM_OPERATORS, - "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.", - false, - true), - booleanProperty( - NATIVE_DEBUG_DISABLE_EXPRESSION_WITH_PEELING, - "If set to true, disables optimization in expression evaluation to peel common " + - "dictionary layer from inputs. Should only be used for debugging.", - false, - true), - booleanProperty( - NATIVE_DEBUG_DISABLE_COMMON_SUB_EXPRESSION, - "If set to true, disables optimization in expression evaluation to reuse cached " + - "results for common sub-expressions. Should only be used for debugging.", - false, - true), - booleanProperty( - NATIVE_DEBUG_DISABLE_EXPRESSION_WITH_MEMOIZATION, - "If set to true, disables optimization in expression evaluation to reuse cached " + - "results between subsequent input batches that are dictionary encoded and " + - "have the same alphabet(underlying flat vector). Should only be used for " + - "debugging.", - false, - true), - booleanProperty( - NATIVE_DEBUG_DISABLE_EXPRESSION_WITH_LAZY_INPUTS, - "If set to true, disables optimization in expression evaluation to delay loading " + - "of lazy inputs unless required. Should only be used for debugging.", - false, - true), - booleanProperty( - NATIVE_SELECTIVE_NIMBLE_READER_ENABLED, - "Temporary flag to control whether selective Nimble reader should be " + - "used in this query or not. Will be removed after the selective Nimble " + - "reader is fully rolled out.", - false, - true), - longProperty( - NATIVE_MAX_PARTIAL_AGGREGATION_MEMORY, - "The max partial aggregation memory when data reduction is not optimal.", - 1L << 24, - false), - longProperty( - NATIVE_MAX_EXTENDED_PARTIAL_AGGREGATION_MEMORY, - "The max partial aggregation memory when data reduction is optimal.", - 1L << 26, - false), - longProperty( - NATIVE_MAX_SPILL_BYTES, - "The max allowed spill bytes", - 100L << 30, - false), - booleanProperty(NATIVE_QUERY_TRACE_ENABLED, - "Enables query tracing.", - false, - false), - stringProperty(NATIVE_QUERY_TRACE_DIR, - "Base dir of a query to store tracing data.", - "", - false), - stringProperty(NATIVE_QUERY_TRACE_NODE_IDS, - "A comma-separated list of plan node ids whose input data will be traced. Empty string if only want to trace the query metadata.", - "", - false), - longProperty(NATIVE_QUERY_TRACE_MAX_BYTES, - "The max trace bytes limit. Tracing is disabled if zero.", - 0L, - false), - stringProperty(NATIVE_QUERY_TRACE_REG_EXP, - "The regexp of traced task id. We only enable trace on a task if its id matches.", - "", - false), - longProperty(NATIVE_MAX_OUTPUT_BUFFER_SIZE, - "The maximum size in bytes for the task's buffered output. The buffer is shared among all drivers.", - 200L << 20, - false), - longProperty(NATIVE_MAX_PAGE_PARTITIONING_BUFFER_SIZE, - "The maximum bytes to buffer per PartitionedOutput operator to avoid creating tiny " + - "SerializedPages. For PartitionedOutputNode::Kind::kPartitioned, PartitionedOutput operator " + - "would buffer up to that number of bytes / number of destinations for each destination before " + - "producing a SerializedPage.", - 24L << 20, - false), booleanProperty( RANDOMIZE_OUTER_JOIN_NULL_KEY, "(Deprecated) Randomize null join key for outer join", @@ -2472,77 +2200,11 @@ public static boolean isJoinSpillingEnabled(Session session) return session.getSystemProperty(JOIN_SPILL_ENABLED, Boolean.class) && isSpillEnabled(session); } - public static boolean isAggregationSpillEnabled(Session session) - { - return session.getSystemProperty(AGGREGATION_SPILL_ENABLED, Boolean.class) && isSpillEnabled(session); - } - - public static boolean isTopNSpillEnabled(Session session) - { - return session.getSystemProperty(TOPN_SPILL_ENABLED, Boolean.class) && isSpillEnabled(session); - } - - public static boolean isDistinctAggregationSpillEnabled(Session session) - { - return session.getSystemProperty(DISTINCT_AGGREGATION_SPILL_ENABLED, Boolean.class) && isAggregationSpillEnabled(session); - } - - public static boolean isDedupBasedDistinctAggregationSpillEnabled(Session session) - { - return session.getSystemProperty(DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED, Boolean.class); - } - - public static boolean isDistinctAggregationLargeBlockSpillEnabled(Session session) - { - return session.getSystemProperty(DISTINCT_AGGREGATION_LARGE_BLOCK_SPILL_ENABLED, Boolean.class); - } - - public static DataSize getDistinctAggregationLargeBlockSizeThreshold(Session session) - { - return session.getSystemProperty(DISTINCT_AGGREGATION_LARGE_BLOCK_SIZE_THRESHOLD, DataSize.class); - } - - public static boolean isOrderByAggregationSpillEnabled(Session session) - { - return session.getSystemProperty(ORDER_BY_AGGREGATION_SPILL_ENABLED, Boolean.class) && isAggregationSpillEnabled(session); - } - - public static boolean isWindowSpillEnabled(Session session) - { - return session.getSystemProperty(WINDOW_SPILL_ENABLED, Boolean.class) && isSpillEnabled(session); - } - - public static boolean isOrderBySpillEnabled(Session session) - { - return session.getSystemProperty(ORDER_BY_SPILL_ENABLED, Boolean.class) && isSpillEnabled(session); - } - - public static DataSize getAggregationOperatorUnspillMemoryLimit(Session session) - { - DataSize memoryLimitForMerge = session.getSystemProperty(AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT, DataSize.class); - checkArgument(memoryLimitForMerge.toBytes() >= 0, "%s must be non-negative", AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT); - return memoryLimitForMerge; - } - - public static DataSize getTopNOperatorUnspillMemoryLimit(Session session) - { - DataSize unspillMemoryLimit = session.getSystemProperty(TOPN_OPERATOR_UNSPILL_MEMORY_LIMIT, DataSize.class); - checkArgument(unspillMemoryLimit.toBytes() >= 0, "%s must be non-negative", TOPN_OPERATOR_UNSPILL_MEMORY_LIMIT); - return unspillMemoryLimit; - } - public static DataSize getQueryMaxRevocableMemoryPerNode(Session session) { return session.getSystemProperty(QUERY_MAX_REVOCABLE_MEMORY_PER_NODE, DataSize.class); } - public static DataSize getTempStorageSpillerBufferSize(Session session) - { - DataSize tempStorageSpillerBufferSize = session.getSystemProperty(TEMP_STORAGE_SPILLER_BUFFER_SIZE, DataSize.class); - checkArgument(tempStorageSpillerBufferSize.toBytes() >= 0, "%s must be non-negative", TEMP_STORAGE_SPILLER_BUFFER_SIZE); - return tempStorageSpillerBufferSize; - } - public static boolean isOptimizeDistinctAggregationEnabled(Session session) { return session.getSystemProperty(OPTIMIZE_DISTINCT_AGGREGATIONS, Boolean.class); @@ -2604,6 +2266,11 @@ public static boolean shouldPushAggregationThroughJoin(Session session) return session.getSystemProperty(PUSH_AGGREGATION_THROUGH_JOIN, Boolean.class); } + public static boolean isNativeExecutionEnabled(Session session) + { + return session.getSystemProperty(NATIVE_EXECUTION_ENABLED, Boolean.class); + } + public static boolean isPushAggregationThroughJoin(Session session) { return session.getSystemProperty(PUSH_PARTIAL_AGGREGATION_THROUGH_JOIN, Boolean.class); diff --git a/presto-main/src/main/java/com/facebook/presto/connector/system/SystemConnectorSessionUtil.java b/presto-main/src/main/java/com/facebook/presto/connector/system/SystemConnectorSessionUtil.java index 3fe9b2601ed2..19cc00b0a706 100644 --- a/presto-main/src/main/java/com/facebook/presto/connector/system/SystemConnectorSessionUtil.java +++ b/presto-main/src/main/java/com/facebook/presto/connector/system/SystemConnectorSessionUtil.java @@ -16,13 +16,14 @@ import com.facebook.presto.Session; import com.facebook.presto.SystemSessionProperties; import com.facebook.presto.common.transaction.TransactionId; -import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.QueryId; import com.facebook.presto.spi.connector.ConnectorTransactionHandle; import com.facebook.presto.spi.security.ConnectorIdentity; import com.facebook.presto.spi.security.Identity; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; + public final class SystemConnectorSessionUtil { private static final SystemSessionProperties SYSTEM_SESSION_PROPERTIES = new SystemSessionProperties(); @@ -35,7 +36,7 @@ public static Session toSession(ConnectorTransactionHandle transactionHandle, Co TransactionId transactionId = ((GlobalSystemTransactionHandle) transactionHandle).getTransactionId(); ConnectorIdentity connectorIdentity = session.getIdentity(); Identity identity = new Identity(connectorIdentity.getUser(), connectorIdentity.getPrincipal(), connectorIdentity.getExtraCredentials()); - return Session.builder(new SessionPropertyManager(SYSTEM_SESSION_PROPERTIES)) + return Session.builder(createTestingSessionPropertyManager(SYSTEM_SESSION_PROPERTIES)) .setQueryId(new QueryId(session.getQueryId())) .setTransactionId(transactionId) .setCatalog("catalog") 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 2639529ba1ab..1b57af2b2588 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 @@ -24,7 +24,6 @@ import com.facebook.presto.execution.QueryTracker; import com.facebook.presto.execution.resourceGroups.ResourceGroupManager; import com.facebook.presto.execution.warnings.WarningCollectorFactory; -import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.resourcemanager.ClusterQueryTrackerService; import com.facebook.presto.resourcemanager.ClusterStatusSender; import com.facebook.presto.server.BasicQueryInfo; @@ -56,6 +55,7 @@ import static com.facebook.presto.Session.SessionBuilder; import static com.facebook.presto.SystemSessionProperties.getAnalyzerType; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.spi.StandardErrorCode.QUERY_TEXT_TOO_LARGE; import static com.facebook.presto.util.AnalyzerUtil.createAnalyzerOptions; import static com.google.common.base.Preconditions.checkArgument; @@ -330,7 +330,7 @@ private void createQueryInternal(QueryId queryId, String slug, int retryCoun catch (Throwable throwable) { // creation must never fail, so register a failed query in this case if (session == null) { - session = Session.builder(new SessionPropertyManager()) + session = Session.builder(createTestingSessionPropertyManager()) .setQueryId(queryId) .setIdentity(sessionContext.getIdentity()) .setSource(sessionContext.getSource()) diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/MetadataManager.java b/presto-main/src/main/java/com/facebook/presto/metadata/MetadataManager.java index 24703303c9a2..c9b86244de8c 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/MetadataManager.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/MetadataManager.java @@ -119,6 +119,7 @@ import static com.facebook.presto.metadata.MetadataUtil.getOptionalCatalogMetadata; import static com.facebook.presto.metadata.MetadataUtil.getOptionalTableHandle; import static com.facebook.presto.metadata.MetadataUtil.toSchemaTableName; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.metadata.TableLayout.fromConnectorLayout; import static com.facebook.presto.spi.Constraint.alwaysTrue; import static com.facebook.presto.spi.StandardErrorCode.INVALID_VIEW; @@ -240,7 +241,7 @@ public static MetadataManager createTestMetadataManager(TransactionManager trans return new MetadataManager( new FunctionAndTypeManager(transactionManager, blockEncodingManager, featuresConfig, functionsConfig, new HandleResolver(), ImmutableSet.of()), blockEncodingManager, - new SessionPropertyManager(), + createTestingSessionPropertyManager(), new SchemaPropertyManager(), new TablePropertyManager(), new ColumnPropertyManager(), diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/SessionPropertyManager.java b/presto-main/src/main/java/com/facebook/presto/metadata/SessionPropertyManager.java index 4e6fed01d547..4372102c211d 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/SessionPropertyManager.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/SessionPropertyManager.java @@ -25,16 +25,27 @@ import com.facebook.presto.common.type.IntegerType; import com.facebook.presto.common.type.MapType; import com.facebook.presto.common.type.Type; +import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.VarcharType; +import com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider; import com.facebook.presto.spi.ConnectorId; +import com.facebook.presto.spi.NodeManager; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.session.PropertyMetadata; +import com.facebook.presto.spi.session.SessionPropertyContext; +import com.facebook.presto.spi.session.WorkerSessionPropertyProvider; +import com.facebook.presto.spi.session.WorkerSessionPropertyProviderFactory; +import com.facebook.presto.spiller.NodeSpillConfig; +import com.facebook.presto.sql.analyzer.FeaturesConfig; +import com.facebook.presto.sql.analyzer.JavaFeaturesConfig; import com.facebook.presto.sql.planner.ParameterRewriter; import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.ExpressionTreeRewriter; import com.facebook.presto.sql.tree.NodeRef; import com.facebook.presto.sql.tree.Parameter; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import javax.annotation.Nullable; @@ -47,35 +58,100 @@ import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.function.Supplier; import static com.facebook.presto.common.type.TypeUtils.writeNativeValue; import static com.facebook.presto.spi.StandardErrorCode.INVALID_SESSION_PROPERTY; import static com.facebook.presto.sql.planner.ExpressionInterpreter.evaluateConstantExpression; import static com.google.common.base.MoreObjects.firstNonNull; import static com.google.common.base.Preconditions.checkState; +import static com.google.common.collect.ImmutableList.toImmutableList; import static java.lang.String.format; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.HOURS; public final class SessionPropertyManager { private static final JsonCodecFactory JSON_CODEC_FACTORY = new JsonCodecFactory(); private final ConcurrentMap> systemSessionProperties = new ConcurrentHashMap<>(); private final ConcurrentMap>> connectorSessionProperties = new ConcurrentHashMap<>(); + private final Map workerSessionPropertyProviders; + private final Map workerSessionPropertyProviderFactories = new ConcurrentHashMap<>(); + private final Supplier>> memoizedWorkerSessionProperties; + private final Optional nodeManager; + private final Optional functionAndTypeManager; - public SessionPropertyManager() + @Inject + public SessionPropertyManager( + SystemSessionProperties systemSessionProperties, + Map workerSessionPropertyProviders, + FunctionAndTypeManager functionAndTypeManager, + NodeManager nodeManager) { - this(new SystemSessionProperties()); + this(systemSessionProperties.getSessionProperties(), workerSessionPropertyProviders, Optional.ofNullable(functionAndTypeManager), Optional.ofNullable(nodeManager)); } - @Inject - public SessionPropertyManager(SystemSessionProperties systemSessionProperties) + public SessionPropertyManager( + List> sessionProperties, + Map workerSessionPropertyProviders, + Optional functionAndTypeManager, + Optional nodeManager) + { + this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionAndTypeManager is null"); + this.memoizedWorkerSessionProperties = Suppliers.memoizeWithExpiration(this::getWorkerSessionProperties, + 1, HOURS); + this.workerSessionPropertyProviders = new ConcurrentHashMap<>(workerSessionPropertyProviders); + addSystemSessionProperties(sessionProperties); + } + + public static SessionPropertyManager createTestingSessionPropertyManager() + { + return createTestingSessionPropertyManager(new SystemSessionProperties().getSessionProperties(), new JavaFeaturesConfig(), new NodeSpillConfig()); + } + + public static SessionPropertyManager createTestingSessionPropertyManager(SystemSessionProperties systemSessionProperties) + { + return createTestingSessionPropertyManager(systemSessionProperties.getSessionProperties(), new JavaFeaturesConfig(), new NodeSpillConfig()); + } + + public static SessionPropertyManager createTestingSessionPropertyManager(List> sessionProperties) + { + return createTestingSessionPropertyManager(sessionProperties, new JavaFeaturesConfig(), new NodeSpillConfig()); + } + + public static SessionPropertyManager createTestingSessionPropertyManager( + List> sessionProperties, + JavaFeaturesConfig javaFeaturesConfig, + NodeSpillConfig nodeSpillConfig) { - this(systemSessionProperties.getSessionProperties()); + return new SessionPropertyManager( + sessionProperties, + ImmutableMap.of( + "java-worker", + new JavaWorkerSessionPropertyProvider( + new FeaturesConfig(), + javaFeaturesConfig, + nodeSpillConfig)), + Optional.empty(), + Optional.empty()); + } + + public void loadSessionPropertyProvider(String sessionPropertyProviderName) + { + WorkerSessionPropertyProviderFactory factory = workerSessionPropertyProviderFactories.get(sessionPropertyProviderName); + checkState(factory != null, "No factory for session property provider : " + sessionPropertyProviderName); + WorkerSessionPropertyProvider sessionPropertyProvider = factory.create(new SessionPropertyContext(functionAndTypeManager, nodeManager)); + if (workerSessionPropertyProviders.putIfAbsent(sessionPropertyProviderName, sessionPropertyProvider) != null) { + throw new IllegalArgumentException("System session property provider is already registered for property provider : " + sessionPropertyProviderName); + } } - public SessionPropertyManager(List> systemSessionProperties) + public void loadSessionPropertyProviders() { - addSystemSessionProperties(systemSessionProperties); + for (String sessionPropertyProviderName : workerSessionPropertyProviderFactories.keySet()) { + loadSessionPropertyProvider(sessionPropertyProviderName); + } } public void addSystemSessionProperties(List> systemSessionProperties) @@ -108,7 +184,9 @@ public void removeConnectorSessionProperties(ConnectorId connectorId) public Optional> getSystemSessionPropertyMetadata(String name) { requireNonNull(name, "name is null"); - + if (systemSessionProperties.get(name) == null) { + return Optional.ofNullable(memoizedWorkerSessionProperties.get().get(name)); + } return Optional.ofNullable(systemSessionProperties.get(name)); } @@ -124,6 +202,20 @@ public Optional> getConnectorSessionPropertyMetadata(Connect return Optional.ofNullable(properties.get(propertyName)); } + private Map> getWorkerSessionProperties() + { + List> workerSessionPropertiesList = workerSessionPropertyProviders.values().stream() + .flatMap(manager -> manager.getSessionProperties().stream()) + .collect(toImmutableList()); + Map> workerSessionProperties = new ConcurrentHashMap<>(); + workerSessionPropertiesList.forEach(sessionProperty -> { + requireNonNull(sessionProperty, "sessionProperty is null"); + // TODO: Implement fail fast in case of duplicate entries. + workerSessionProperties.put(sessionProperty.getName(), sessionProperty); + }); + return workerSessionProperties; + } + public List getAllSessionProperties(Session session, Map catalogs) { requireNonNull(session, "session is null"); @@ -165,6 +257,19 @@ public List getAllSessionProperties(Session session, Map property : new TreeMap<>(memoizedWorkerSessionProperties.get()).values()) { + String defaultValue = firstNonNull(property.getDefaultValue(), "").toString(); + String value = systemProperties.getOrDefault(property.getName(), defaultValue); + sessionPropertyValues.add(new SessionPropertyValue( + value, + defaultValue, + property.getName(), + Optional.empty(), + property.getName(), + property.getDescription(), + property.getSqlType().getDisplayName(), + property.isHidden())); + } return sessionPropertyValues.build(); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/GenericAccumulatorFactory.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/GenericAccumulatorFactory.java index b922f98448f2..d83db731c358 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/GenericAccumulatorFactory.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/GenericAccumulatorFactory.java @@ -67,15 +67,15 @@ import java.util.Optional; import java.util.stream.Collectors; -import static com.facebook.presto.SystemSessionProperties.getDistinctAggregationLargeBlockSizeThreshold; -import static com.facebook.presto.SystemSessionProperties.isDedupBasedDistinctAggregationSpillEnabled; -import static com.facebook.presto.SystemSessionProperties.isDistinctAggregationLargeBlockSpillEnabled; import static com.facebook.presto.common.Page.wrapBlocksWithoutCopy; import static com.facebook.presto.common.block.ColumnarArray.toColumnarArray; import static com.facebook.presto.common.block.ColumnarRow.toColumnarRow; import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.BooleanType.BOOLEAN; import static com.facebook.presto.common.type.VarcharType.VARCHAR; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.getDistinctAggregationLargeBlockSizeThreshold; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.isDedupBasedDistinctAggregationSpillEnabled; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.isDistinctAggregationLargeBlockSpillEnabled; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.ImmutableList.toImmutableList; diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java index 0f3ba3b14df0..5d681d829ffa 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java @@ -39,6 +39,7 @@ import com.facebook.presto.execution.warnings.WarningCollectorModule; import com.facebook.presto.metadata.Catalog; import com.facebook.presto.metadata.CatalogManager; +import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.metadata.StaticCatalogStore; import com.facebook.presto.metadata.StaticFunctionNamespaceStore; import com.facebook.presto.security.AccessControlManager; @@ -179,6 +180,7 @@ public void run() injector.getInstance(NodeStatusNotificationManager.class).loadNodeStatusNotificationProvider(); injector.getInstance(GracefulShutdownHandler.class).loadNodeStatusNotification(); injector.getInstance(PlanCheckerProviderManager.class).loadPlanCheckerProviders(); + injector.getInstance(SessionPropertyManager.class).loadSessionPropertyProviders(); startAssociatedProcesses(injector); injector.getInstance(Announcer.class).start(); 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 85fd54dfb767..808b17e0dbc8 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 @@ -105,6 +105,7 @@ import com.facebook.presto.metadata.StaticFunctionNamespaceStore; import com.facebook.presto.metadata.StaticFunctionNamespaceStoreConfig; import com.facebook.presto.metadata.TablePropertyManager; +import com.facebook.presto.nodeManager.PluginNodeManager; import com.facebook.presto.operator.ExchangeClientConfig; import com.facebook.presto.operator.ExchangeClientFactory; import com.facebook.presto.operator.ExchangeClientSupplier; @@ -139,9 +140,12 @@ import com.facebook.presto.server.thrift.ThriftServerInfoService; import com.facebook.presto.server.thrift.ThriftTaskClient; import com.facebook.presto.server.thrift.ThriftTaskService; +import com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider; +import com.facebook.presto.sessionpropertyproviders.NativeWorkerSessionPropertyProvider; import com.facebook.presto.spi.ConnectorMetadataUpdateHandle; import com.facebook.presto.spi.ConnectorSplit; import com.facebook.presto.spi.ConnectorTypeSerde; +import com.facebook.presto.spi.NodeManager; import com.facebook.presto.spi.PageIndexerFactory; import com.facebook.presto.spi.PageSorter; import com.facebook.presto.spi.analyzer.ViewDefinition; @@ -152,6 +156,7 @@ import com.facebook.presto.spi.relation.DomainTranslator; import com.facebook.presto.spi.relation.PredicateCompiler; import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.spi.session.WorkerSessionPropertyProvider; import com.facebook.presto.spiller.FileSingleStreamSpillerFactory; import com.facebook.presto.spiller.GenericPartitioningSpillerFactory; import com.facebook.presto.spiller.GenericSpillerFactory; @@ -184,6 +189,7 @@ import com.facebook.presto.sql.analyzer.FeaturesConfig.SingleStreamSpillerChoice; import com.facebook.presto.sql.analyzer.ForMetadataExtractor; import com.facebook.presto.sql.analyzer.FunctionsConfig; +import com.facebook.presto.sql.analyzer.JavaFeaturesConfig; import com.facebook.presto.sql.analyzer.MetadataExtractor; import com.facebook.presto.sql.analyzer.MetadataExtractorMBean; import com.facebook.presto.sql.analyzer.QueryExplainer; @@ -227,6 +233,7 @@ import com.google.inject.Provides; import com.google.inject.Scopes; import com.google.inject.TypeLiteral; +import com.google.inject.multibindings.MapBinder; import io.airlift.slice.Slice; import io.airlift.units.DataSize; import io.airlift.units.Duration; @@ -310,6 +317,7 @@ else if (serverConfig.isCoordinator()) { configBinder(binder).bindConfig(FeaturesConfig.class); configBinder(binder).bindConfig(FunctionsConfig.class); + configBinder(binder).bindConfig(JavaFeaturesConfig.class); binder.bind(PlanChecker.class).in(Scopes.SINGLETON); @@ -793,6 +801,18 @@ public ListeningExecutorService createResourceManagerExecutor(ResourceManagerCon binder.bind(NodeStatusNotificationManager.class).in(Scopes.SINGLETON); binder.bind(PlanCheckerProviderManager.class).in(Scopes.SINGLETON); + + // Worker session property providers + MapBinder mapBinder = + newMapBinder(binder, String.class, WorkerSessionPropertyProvider.class); + mapBinder.addBinding("java-worker").to(JavaWorkerSessionPropertyProvider.class).in(Scopes.SINGLETON); + if (!serverConfig.isCoordinatorSidecarEnabled()) { + mapBinder.addBinding("native-worker").to(NativeWorkerSessionPropertyProvider.class).in(Scopes.SINGLETON); + } + + // Node manager binding + binder.bind(PluginNodeManager.class).in(Scopes.SINGLETON); + binder.bind(NodeManager.class).to(PluginNodeManager.class).in(Scopes.SINGLETON); } @Provides diff --git a/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/JavaWorkerSessionPropertyProvider.java b/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/JavaWorkerSessionPropertyProvider.java new file mode 100644 index 000000000000..d53339750ed3 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/JavaWorkerSessionPropertyProvider.java @@ -0,0 +1,205 @@ +/* + * 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.sessionpropertyproviders; + +import com.facebook.presto.Session; +import com.facebook.presto.spi.session.PropertyMetadata; +import com.facebook.presto.spi.session.WorkerSessionPropertyProvider; +import com.facebook.presto.spiller.NodeSpillConfig; +import com.facebook.presto.sql.analyzer.FeaturesConfig; +import com.facebook.presto.sql.analyzer.JavaFeaturesConfig; +import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; +import io.airlift.units.DataSize; + +import java.util.List; + +import static com.facebook.presto.SystemSessionProperties.isSpillEnabled; +import static com.facebook.presto.common.type.VarcharType.VARCHAR; +import static com.facebook.presto.spi.session.PropertyMetadata.booleanProperty; +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class JavaWorkerSessionPropertyProvider + implements WorkerSessionPropertyProvider +{ + public static final String AGGREGATION_SPILL_ENABLED = "aggregation_spill_enabled"; + public static final String TOPN_SPILL_ENABLED = "topn_spill_enabled"; + public static final String DISTINCT_AGGREGATION_SPILL_ENABLED = "distinct_aggregation_spill_enabled"; + public static final String DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED = "dedup_based_distinct_aggregation_spill_enabled"; + public static final String DISTINCT_AGGREGATION_LARGE_BLOCK_SPILL_ENABLED = "distinct_aggregation_large_block_spill_enabled"; + public static final String DISTINCT_AGGREGATION_LARGE_BLOCK_SIZE_THRESHOLD = "distinct_aggregation_large_block_size_threshold"; + public static final String ORDER_BY_AGGREGATION_SPILL_ENABLED = "order_by_aggregation_spill_enabled"; + public static final String WINDOW_SPILL_ENABLED = "window_spill_enabled"; + public static final String ORDER_BY_SPILL_ENABLED = "order_by_spill_enabled"; + public static final String AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT = "aggregation_operator_unspill_memory_limit"; + public static final String TOPN_OPERATOR_UNSPILL_MEMORY_LIMIT = "topn_operator_unspill_memory_limit"; + public static final String TEMP_STORAGE_SPILLER_BUFFER_SIZE = "temp_storage_spiller_buffer_size"; + private final List> sessionProperties; + + @Inject + public JavaWorkerSessionPropertyProvider(FeaturesConfig featuresConfig, JavaFeaturesConfig javaFeaturesConfig, NodeSpillConfig nodeSpillConfig) + { + boolean nativeExecution = requireNonNull(featuresConfig, "featuresConfig is null").isNativeExecutionEnabled(); + sessionProperties = ImmutableList.of( + booleanProperty( + TOPN_SPILL_ENABLED, + "Enable topN spilling if spill_enabled", + javaFeaturesConfig.isTopNSpillEnabled(), + nativeExecution), + booleanProperty( + AGGREGATION_SPILL_ENABLED, + "Enable aggregate spilling if spill_enabled", + javaFeaturesConfig.isAggregationSpillEnabled(), + nativeExecution), + booleanProperty( + DISTINCT_AGGREGATION_SPILL_ENABLED, + "Enable spill for distinct aggregations if spill_enabled and aggregation_spill_enabled", + javaFeaturesConfig.isDistinctAggregationSpillEnabled(), + nativeExecution), + booleanProperty( + DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED, + "Perform deduplication of input data for distinct aggregates before spilling", + javaFeaturesConfig.isDedupBasedDistinctAggregationSpillEnabled(), + nativeExecution), + booleanProperty( + DISTINCT_AGGREGATION_LARGE_BLOCK_SPILL_ENABLED, + "Spill large block to a separate spill file", + javaFeaturesConfig.isDistinctAggregationLargeBlockSpillEnabled(), + nativeExecution), + new PropertyMetadata<>( + DISTINCT_AGGREGATION_LARGE_BLOCK_SIZE_THRESHOLD, + "Block size threshold beyond which it will be spilled into a separate spill file", + VARCHAR, + DataSize.class, + javaFeaturesConfig.getDistinctAggregationLargeBlockSizeThreshold(), + nativeExecution, + value -> DataSize.valueOf((String) value), + DataSize::toString), + booleanProperty( + ORDER_BY_AGGREGATION_SPILL_ENABLED, + "Enable spill for order-by aggregations if spill_enabled and aggregation_spill_enabled", + javaFeaturesConfig.isOrderByAggregationSpillEnabled(), + nativeExecution), + booleanProperty( + WINDOW_SPILL_ENABLED, + "Enable window spilling if spill_enabled", + javaFeaturesConfig.isWindowSpillEnabled(), + nativeExecution), + booleanProperty( + ORDER_BY_SPILL_ENABLED, + "Enable order by spilling if spill_enabled", + javaFeaturesConfig.isOrderBySpillEnabled(), + nativeExecution), + new PropertyMetadata<>( + AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT, + "Experimental: How much memory can should be allocated per aggregation operator in unspilling process", + VARCHAR, + DataSize.class, + javaFeaturesConfig.getAggregationOperatorUnspillMemoryLimit(), + nativeExecution, + value -> DataSize.valueOf((String) value), + DataSize::toString), + new PropertyMetadata<>( + TOPN_OPERATOR_UNSPILL_MEMORY_LIMIT, + "How much memory can should be allocated per topN operator in unspilling process", + VARCHAR, + DataSize.class, + javaFeaturesConfig.getTopNOperatorUnspillMemoryLimit(), + nativeExecution, + value -> DataSize.valueOf((String) value), + DataSize::toString), + new PropertyMetadata<>( + TEMP_STORAGE_SPILLER_BUFFER_SIZE, + "Experimental: Buffer size used by TempStorageSingleStreamSpiller", + VARCHAR, + DataSize.class, + nodeSpillConfig.getTempStorageBufferSize(), + nativeExecution, + value -> DataSize.valueOf((String) value), + DataSize::toString)); + } + + @Override + public List> getSessionProperties() + { + return sessionProperties; + } + + public static boolean isTopNSpillEnabled(Session session) + { + return session.getSystemProperty(TOPN_SPILL_ENABLED, Boolean.class) && isSpillEnabled(session); + } + + public static boolean isAggregationSpillEnabled(Session session) + { + return session.getSystemProperty(AGGREGATION_SPILL_ENABLED, Boolean.class) && isSpillEnabled(session); + } + + public static boolean isDistinctAggregationSpillEnabled(Session session) + { + return session.getSystemProperty(DISTINCT_AGGREGATION_SPILL_ENABLED, Boolean.class) && isAggregationSpillEnabled(session); + } + + public static boolean isDedupBasedDistinctAggregationSpillEnabled(Session session) + { + return session.getSystemProperty(DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED, Boolean.class); + } + + public static boolean isDistinctAggregationLargeBlockSpillEnabled(Session session) + { + return session.getSystemProperty(DISTINCT_AGGREGATION_LARGE_BLOCK_SPILL_ENABLED, Boolean.class); + } + + public static DataSize getDistinctAggregationLargeBlockSizeThreshold(Session session) + { + return session.getSystemProperty(DISTINCT_AGGREGATION_LARGE_BLOCK_SIZE_THRESHOLD, DataSize.class); + } + + public static boolean isOrderByAggregationSpillEnabled(Session session) + { + return session.getSystemProperty(ORDER_BY_AGGREGATION_SPILL_ENABLED, Boolean.class) && isAggregationSpillEnabled(session); + } + + public static boolean isWindowSpillEnabled(Session session) + { + return session.getSystemProperty(WINDOW_SPILL_ENABLED, Boolean.class) && isSpillEnabled(session); + } + + public static boolean isOrderBySpillEnabled(Session session) + { + return session.getSystemProperty(ORDER_BY_SPILL_ENABLED, Boolean.class) && isSpillEnabled(session); + } + + public static DataSize getAggregationOperatorUnspillMemoryLimit(Session session) + { + DataSize memoryLimitForMerge = session.getSystemProperty(AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT, DataSize.class); + checkArgument(memoryLimitForMerge.toBytes() >= 0, "%s must be positive", AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT); + return memoryLimitForMerge; + } + + public static DataSize getTopNOperatorUnspillMemoryLimit(Session session) + { + DataSize unspillMemoryLimit = session.getSystemProperty(TOPN_OPERATOR_UNSPILL_MEMORY_LIMIT, DataSize.class); + checkArgument(unspillMemoryLimit.toBytes() >= 0, "%s must be positive", TOPN_OPERATOR_UNSPILL_MEMORY_LIMIT); + return unspillMemoryLimit; + } + + public static DataSize getTempStorageSpillerBufferSize(Session session) + { + DataSize tempStorageSpillerBufferSize = session.getSystemProperty(TEMP_STORAGE_SPILLER_BUFFER_SIZE, DataSize.class); + checkArgument(tempStorageSpillerBufferSize.toBytes() >= 0, "%s must be positive", TEMP_STORAGE_SPILLER_BUFFER_SIZE); + return tempStorageSpillerBufferSize; + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java b/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java new file mode 100644 index 000000000000..ebe2c798587c --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sessionpropertyproviders/NativeWorkerSessionPropertyProvider.java @@ -0,0 +1,236 @@ +/* + * 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.sessionpropertyproviders; + +import com.facebook.presto.spi.session.PropertyMetadata; +import com.facebook.presto.spi.session.WorkerSessionPropertyProvider; +import com.facebook.presto.sql.analyzer.FeaturesConfig; +import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; + +import java.util.List; + +import static com.facebook.presto.spi.session.PropertyMetadata.booleanProperty; +import static com.facebook.presto.spi.session.PropertyMetadata.integerProperty; +import static com.facebook.presto.spi.session.PropertyMetadata.longProperty; +import static com.facebook.presto.spi.session.PropertyMetadata.stringProperty; +import static java.util.Objects.requireNonNull; + +@Deprecated +public class NativeWorkerSessionPropertyProvider + implements WorkerSessionPropertyProvider +{ + public static final String NATIVE_SIMPLIFIED_EXPRESSION_EVALUATION_ENABLED = "native_simplified_expression_evaluation_enabled"; + public static final String NATIVE_EXPRESSION_MAX_ARRAY_SIZE_IN_REDUCE = "native_expression_max_array_size_in_reduce"; + public static final String NATIVE_MAX_SPILL_LEVEL = "native_max_spill_level"; + public static final String NATIVE_MAX_SPILL_FILE_SIZE = "native_max_spill_file_size"; + public static final String NATIVE_SPILL_COMPRESSION_CODEC = "native_spill_compression_codec"; + public static final String NATIVE_SPILL_WRITE_BUFFER_SIZE = "native_spill_write_buffer_size"; + public static final String NATIVE_SPILL_FILE_CREATE_CONFIG = "native_spill_file_create_config"; + public static final String NATIVE_JOIN_SPILL_ENABLED = "native_join_spill_enabled"; + public static final String NATIVE_WINDOW_SPILL_ENABLED = "native_window_spill_enabled"; + public static final String NATIVE_WRITER_SPILL_ENABLED = "native_writer_spill_enabled"; + public static final String NATIVE_ROW_NUMBER_SPILL_ENABLED = "native_row_number_spill_enabled"; + public static final String NATIVE_TOPN_ROW_NUMBER_SPILL_ENABLED = "native_topn_row_number_spill_enabled"; + public static final String NATIVE_SPILLER_NUM_PARTITION_BITS = "native_spiller_num_partition_bits"; + public static final String NATIVE_DEBUG_VALIDATE_OUTPUT_FROM_OPERATORS = "native_debug_validate_output_from_operators"; + public static final String NATIVE_DEBUG_DISABLE_EXPRESSION_WITH_PEELING = "native_debug_disable_expression_with_peeling"; + public static final String NATIVE_DEBUG_DISABLE_COMMON_SUB_EXPRESSION = "native_debug_disable_common_sub_expressions"; + public static final String NATIVE_DEBUG_DISABLE_EXPRESSION_WITH_MEMOIZATION = "native_debug_disable_expression_with_memoization"; + public static final String NATIVE_DEBUG_DISABLE_EXPRESSION_WITH_LAZY_INPUTS = "native_debug_disable_expression_with_lazy_inputs"; + public static final String NATIVE_SELECTIVE_NIMBLE_READER_ENABLED = "native_selective_nimble_reader_enabled"; + public static final String NATIVE_MAX_PARTIAL_AGGREGATION_MEMORY = "native_max_partial_aggregation_memory"; + public static final String NATIVE_MAX_EXTENDED_PARTIAL_AGGREGATION_MEMORY = "native_max_extended_partial_aggregation_memory"; + public static final String NATIVE_MAX_SPILL_BYTES = "native_max_spill_bytes"; + public static final String NATIVE_MAX_PAGE_PARTITIONING_BUFFER_SIZE = "native_max_page_partitioning_buffer_size"; + public static final String NATIVE_MAX_OUTPUT_BUFFER_SIZE = "native_max_output_buffer_size"; + public static final String NATIVE_QUERY_TRACE_ENABLED = "native_query_trace_enabled"; + public static final String NATIVE_QUERY_TRACE_DIR = "native_query_trace_dir"; + public static final String NATIVE_QUERY_TRACE_NODE_IDS = "native_query_trace_node_ids"; + public static final String NATIVE_QUERY_TRACE_MAX_BYTES = "native_query_trace_max_bytes"; + public static final String NATIVE_QUERY_TRACE_REG_EXP = "native_query_trace_task_reg_exp"; + private final List> sessionProperties; + + @Inject + public NativeWorkerSessionPropertyProvider(FeaturesConfig featuresConfig) + { + boolean nativeExecution = requireNonNull(featuresConfig, "featuresConfig is null").isNativeExecutionEnabled(); + sessionProperties = ImmutableList.of( + booleanProperty( + NATIVE_SIMPLIFIED_EXPRESSION_EVALUATION_ENABLED, + "Native Execution only. Enable simplified path in expression evaluation", + false, + !nativeExecution), + integerProperty( + NATIVE_EXPRESSION_MAX_ARRAY_SIZE_IN_REDUCE, + "Native Execution only. Reduce() function will throw an error if it encounters an array of size greater than this value.", + 100000, + !nativeExecution), + integerProperty( + NATIVE_MAX_SPILL_LEVEL, + "Native Execution only. The maximum allowed spilling level for hash join build.\n" + + "0 is the initial spilling level, -1 means unlimited.", + 4, + !nativeExecution), + integerProperty( + NATIVE_MAX_SPILL_FILE_SIZE, + "The max allowed spill file size. If it is zero, then there is no limit.", + 0, + !nativeExecution), + stringProperty( + NATIVE_SPILL_COMPRESSION_CODEC, + "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.", + "zstd", + !nativeExecution), + longProperty( + NATIVE_SPILL_WRITE_BUFFER_SIZE, + "Native Execution only. The maximum size in bytes to buffer the serialized spill data before writing to disk for IO efficiency.\n" + + "If set to zero, buffering is disabled.", + 1024L * 1024L, + !nativeExecution), + stringProperty( + NATIVE_SPILL_FILE_CREATE_CONFIG, + "Native Execution only. Config used to create spill files. This config is \n" + + "provided to underlying file system and the config is free form. The form should be\n" + + "defined by the underlying file system.", + "", + !nativeExecution), + booleanProperty( + NATIVE_JOIN_SPILL_ENABLED, + "Native Execution only. Enable join spilling on native engine", + false, + !nativeExecution), + booleanProperty( + NATIVE_WINDOW_SPILL_ENABLED, + "Native Execution only. Enable window spilling on native engine", + false, + !nativeExecution), + booleanProperty( + NATIVE_WRITER_SPILL_ENABLED, + "Native Execution only. Enable writer spilling on native engine", + false, + !nativeExecution), + booleanProperty( + NATIVE_ROW_NUMBER_SPILL_ENABLED, + "Native Execution only. Enable row number spilling on native engine", + false, + !nativeExecution), + booleanProperty( + NATIVE_TOPN_ROW_NUMBER_SPILL_ENABLED, + "Native Execution only. Enable topN row number spilling on native engine", + false, + !nativeExecution), + integerProperty( + NATIVE_SPILLER_NUM_PARTITION_BITS, + "Native Execution only. The number of bits (N) used to calculate the " + + "spilling partition number for hash join and RowNumber: 2 ^ N", + 3, + !nativeExecution), + booleanProperty( + NATIVE_DEBUG_VALIDATE_OUTPUT_FROM_OPERATORS, + "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.", + false, + true), + booleanProperty( + NATIVE_DEBUG_DISABLE_EXPRESSION_WITH_PEELING, + "If set to true, disables optimization in expression evaluation to peel common " + + "dictionary layer from inputs. Should only be used for debugging.", + false, + true), + booleanProperty( + NATIVE_DEBUG_DISABLE_COMMON_SUB_EXPRESSION, + "If set to true, disables optimization in expression evaluation to reuse cached " + + "results for common sub-expressions. Should only be used for debugging.", + false, + true), + booleanProperty( + NATIVE_DEBUG_DISABLE_EXPRESSION_WITH_MEMOIZATION, + "If set to true, disables optimization in expression evaluation to reuse cached " + + "results between subsequent input batches that are dictionary encoded and " + + "have the same alphabet(underlying flat vector). Should only be used for " + + "debugging.", + false, + true), + booleanProperty( + NATIVE_DEBUG_DISABLE_EXPRESSION_WITH_LAZY_INPUTS, + "If set to true, disables optimization in expression evaluation to delay loading " + + "of lazy inputs unless required. Should only be used for debugging.", + false, + true), + booleanProperty( + NATIVE_SELECTIVE_NIMBLE_READER_ENABLED, + "Temporary flag to control whether selective Nimble reader should be " + + "used in this query or not. Will be removed after the selective Nimble " + + "reader is fully rolled out.", + false, + !nativeExecution), + longProperty( + NATIVE_MAX_PARTIAL_AGGREGATION_MEMORY, + "The max partial aggregation memory when data reduction is not optimal.", + 1L << 24, + !nativeExecution), + longProperty( + NATIVE_MAX_EXTENDED_PARTIAL_AGGREGATION_MEMORY, + "The max partial aggregation memory when data reduction is optimal.", + 1L << 26, + !nativeExecution), + longProperty( + NATIVE_MAX_SPILL_BYTES, + "The max allowed spill bytes", + 100L << 30, + !nativeExecution), + booleanProperty(NATIVE_QUERY_TRACE_ENABLED, + "Enables query tracing.", + false, + !nativeExecution), + stringProperty(NATIVE_QUERY_TRACE_DIR, + "Base dir of a query to store tracing data.", + "", + !nativeExecution), + stringProperty(NATIVE_QUERY_TRACE_NODE_IDS, + "A comma-separated list of plan node ids whose input data will be traced. Empty string if only want to trace the query metadata.", + "", + !nativeExecution), + longProperty(NATIVE_QUERY_TRACE_MAX_BYTES, + "The max trace bytes limit. Tracing is disabled if zero.", + 0L, + !nativeExecution), + stringProperty(NATIVE_QUERY_TRACE_REG_EXP, + "The regexp of traced task id. We only enable trace on a task if its id matches.", + "", + !nativeExecution), + longProperty(NATIVE_MAX_OUTPUT_BUFFER_SIZE, + "The maximum size in bytes for the task's buffered output. The buffer is shared among all drivers.", + 200L << 20, + !nativeExecution), + longProperty(NATIVE_MAX_PAGE_PARTITIONING_BUFFER_SIZE, + "The maximum bytes to buffer per PartitionedOutput operator to avoid creating tiny " + + "SerializedPages. For PartitionedOutputNode::Kind::kPartitioned, PartitionedOutput operator " + + "would buffer up to that number of bytes / number of destinations for each destination before " + + "producing a SerializedPage.", + 24L << 20, + !nativeExecution)); + } + + @Override + public List> getSessionProperties() + { + return sessionProperties; + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/spiller/TempStorageSingleStreamSpiller.java b/presto-main/src/main/java/com/facebook/presto/spiller/TempStorageSingleStreamSpiller.java index 36b80f5c2532..49634921a69f 100644 --- a/presto-main/src/main/java/com/facebook/presto/spiller/TempStorageSingleStreamSpiller.java +++ b/presto-main/src/main/java/com/facebook/presto/spiller/TempStorageSingleStreamSpiller.java @@ -46,9 +46,9 @@ import java.util.List; import java.util.Optional; -import static com.facebook.presto.SystemSessionProperties.getTempStorageSpillerBufferSize; import static com.facebook.presto.common.block.PageBuilderStatus.DEFAULT_MAX_PAGE_SIZE_IN_BYTES; import static com.facebook.presto.execution.buffer.PageSplitterUtil.splitPage; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.getTempStorageSpillerBufferSize; import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.Iterators.transform; diff --git a/presto-main/src/main/java/com/facebook/presto/spiller/TempStorageStandaloneSpillerFactory.java b/presto-main/src/main/java/com/facebook/presto/spiller/TempStorageStandaloneSpillerFactory.java index a27e1f6e0e7b..ad25975e916a 100644 --- a/presto-main/src/main/java/com/facebook/presto/spiller/TempStorageStandaloneSpillerFactory.java +++ b/presto-main/src/main/java/com/facebook/presto/spiller/TempStorageStandaloneSpillerFactory.java @@ -25,7 +25,7 @@ import java.util.Optional; -import static com.facebook.presto.SystemSessionProperties.getTempStorageSpillerBufferSize; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.getTempStorageSpillerBufferSize; import static java.lang.Math.toIntExact; import static java.util.Objects.requireNonNull; diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java index 88d1b3248cd7..938bc146dae0 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java @@ -127,17 +127,6 @@ public class FeaturesConfig private boolean dictionaryAggregation; private boolean spillEnabled; private boolean joinSpillingEnabled = true; - private boolean aggregationSpillEnabled = true; - private boolean topNSpillEnabled = true; - private boolean distinctAggregationSpillEnabled = true; - private boolean dedupBasedDistinctAggregationSpillEnabled; - private boolean distinctAggregationLargeBlockSpillEnabled; - private DataSize distinctAggregationLargeBlockSizeThreshold = new DataSize(50, MEGABYTE); - private boolean orderByAggregationSpillEnabled = true; - private boolean windowSpillEnabled = true; - private boolean orderBySpillEnabled = true; - private DataSize aggregationOperatorUnspillMemoryLimit = new DataSize(4, MEGABYTE); - private DataSize topNOperatorUnspillMemoryLimit = new DataSize(4, MEGABYTE); private List spillerSpillPaths = ImmutableList.of(); private int spillerThreads = 4; private double spillMaxUsedSpaceThreshold = 0.9; @@ -1186,123 +1175,6 @@ public FeaturesConfig setJoinSpillingEnabled(boolean joinSpillingEnabled) return this; } - @Config("experimental.aggregation-spill-enabled") - @ConfigDescription("Spill aggregations if spill is enabled") - public FeaturesConfig setAggregationSpillEnabled(boolean aggregationSpillEnabled) - { - this.aggregationSpillEnabled = aggregationSpillEnabled; - return this; - } - - public boolean isAggregationSpillEnabled() - { - return aggregationSpillEnabled; - } - - @Config("experimental.topn-spill-enabled") - @ConfigDescription("Spill TopN if spill is enabled") - public FeaturesConfig setTopNSpillEnabled(boolean topNSpillEnabled) - { - this.topNSpillEnabled = topNSpillEnabled; - return this; - } - - public boolean isTopNSpillEnabled() - { - return topNSpillEnabled; - } - - @Config("experimental.distinct-aggregation-spill-enabled") - @ConfigDescription("Spill distinct aggregations if aggregation spill is enabled") - public FeaturesConfig setDistinctAggregationSpillEnabled(boolean distinctAggregationSpillEnabled) - { - this.distinctAggregationSpillEnabled = distinctAggregationSpillEnabled; - return this; - } - - public boolean isDistinctAggregationSpillEnabled() - { - return distinctAggregationSpillEnabled; - } - - @Config("experimental.dedup-based-distinct-aggregation-spill-enabled") - @ConfigDescription("Dedup input data for Distinct Aggregates before spilling") - public FeaturesConfig setDedupBasedDistinctAggregationSpillEnabled(boolean dedupBasedDistinctAggregationSpillEnabled) - { - this.dedupBasedDistinctAggregationSpillEnabled = dedupBasedDistinctAggregationSpillEnabled; - return this; - } - - public boolean isDedupBasedDistinctAggregationSpillEnabled() - { - return dedupBasedDistinctAggregationSpillEnabled; - } - - @Config("experimental.distinct-aggregation-large-block-spill-enabled") - @ConfigDescription("Spill large block to a separate spill file") - public FeaturesConfig setDistinctAggregationLargeBlockSpillEnabled(boolean distinctAggregationLargeBlockSpillEnabled) - { - this.distinctAggregationLargeBlockSpillEnabled = distinctAggregationLargeBlockSpillEnabled; - return this; - } - - public boolean isDistinctAggregationLargeBlockSpillEnabled() - { - return distinctAggregationLargeBlockSpillEnabled; - } - - @Config("experimental.distinct-aggregation-large-block-size-threshold") - @ConfigDescription("Block size threshold beyond which it will be spilled into a separate spill file") - public FeaturesConfig setDistinctAggregationLargeBlockSizeThreshold(DataSize distinctAggregationLargeBlockSizeThreshold) - { - this.distinctAggregationLargeBlockSizeThreshold = distinctAggregationLargeBlockSizeThreshold; - return this; - } - - public DataSize getDistinctAggregationLargeBlockSizeThreshold() - { - return distinctAggregationLargeBlockSizeThreshold; - } - - @Config("experimental.order-by-aggregation-spill-enabled") - @ConfigDescription("Spill order-by aggregations if aggregation spill is enabled") - public FeaturesConfig setOrderByAggregationSpillEnabled(boolean orderByAggregationSpillEnabled) - { - this.orderByAggregationSpillEnabled = orderByAggregationSpillEnabled; - return this; - } - - public boolean isOrderByAggregationSpillEnabled() - { - return orderByAggregationSpillEnabled; - } - - @Config("experimental.window-spill-enabled") - @ConfigDescription("Enable Window Operator Spilling if spill is enabled") - public FeaturesConfig setWindowSpillEnabled(boolean windowSpillEnabled) - { - this.windowSpillEnabled = windowSpillEnabled; - return this; - } - - public boolean isWindowSpillEnabled() - { - return windowSpillEnabled; - } - - @Config("experimental.order-by-spill-enabled") - @ConfigDescription("Enable Order-by Operator Spilling if spill is enabled") - public FeaturesConfig setOrderBySpillEnabled(boolean orderBySpillEnabled) - { - this.orderBySpillEnabled = orderBySpillEnabled; - return this; - } - - public boolean isOrderBySpillEnabled() - { - return orderBySpillEnabled; - } - public boolean isIterativeOptimizerEnabled() { return iterativeOptimizerEnabled; @@ -1441,30 +1313,6 @@ public double getDefaultWriterReplicationCoefficient() return defaultWriterReplicationCoefficient; } - public DataSize getTopNOperatorUnspillMemoryLimit() - { - return topNOperatorUnspillMemoryLimit; - } - - @Config("experimental.topn-operator-unspill-memory-limit") - public FeaturesConfig setTopNOperatorUnspillMemoryLimit(DataSize aggregationOperatorUnspillMemoryLimit) - { - this.topNOperatorUnspillMemoryLimit = aggregationOperatorUnspillMemoryLimit; - return this; - } - - public DataSize getAggregationOperatorUnspillMemoryLimit() - { - return aggregationOperatorUnspillMemoryLimit; - } - - @Config("experimental.aggregation-operator-unspill-memory-limit") - public FeaturesConfig setAggregationOperatorUnspillMemoryLimit(DataSize aggregationOperatorUnspillMemoryLimit) - { - this.aggregationOperatorUnspillMemoryLimit = aggregationOperatorUnspillMemoryLimit; - return this; - } - public List getSpillerSpillPaths() { return spillerSpillPaths; diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/JavaFeaturesConfig.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/JavaFeaturesConfig.java new file mode 100644 index 000000000000..ae5e78377565 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/JavaFeaturesConfig.java @@ -0,0 +1,176 @@ +/* + * 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.airlift.configuration.Config; +import com.facebook.airlift.configuration.ConfigDescription; +import io.airlift.units.DataSize; + +import static io.airlift.units.DataSize.Unit.MEGABYTE; + +public class JavaFeaturesConfig +{ + private boolean topNSpillEnabled = true; + private boolean aggregationSpillEnabled = true; + private boolean distinctAggregationSpillEnabled = true; + private boolean dedupBasedDistinctAggregationSpillEnabled; + private boolean distinctAggregationLargeBlockSpillEnabled; + private boolean orderByAggregationSpillEnabled = true; + private boolean orderBySpillEnabled = true; + private boolean windowSpillEnabled = true; + private DataSize distinctAggregationLargeBlockSizeThreshold = new DataSize(50, MEGABYTE); + private DataSize topNOperatorUnspillMemoryLimit = new DataSize(4, MEGABYTE); + private DataSize aggregationOperatorUnspillMemoryLimit = new DataSize(4, MEGABYTE); + + public boolean isOrderBySpillEnabled() + { + return orderBySpillEnabled; + } + + @Config("experimental.order-by-spill-enabled") + @ConfigDescription("Enable Order-by Operator Spilling if spill is enabled") + public JavaFeaturesConfig setOrderBySpillEnabled(boolean orderBySpillEnabled) + { + this.orderBySpillEnabled = orderBySpillEnabled; + return this; + } + + public boolean isWindowSpillEnabled() + { + return windowSpillEnabled; + } + + @Config("experimental.window-spill-enabled") + @ConfigDescription("Enable Window Operator Spilling if spill is enabled") + public JavaFeaturesConfig setWindowSpillEnabled(boolean windowSpillEnabled) + { + this.windowSpillEnabled = windowSpillEnabled; + return this; + } + + public boolean isOrderByAggregationSpillEnabled() + { + return orderByAggregationSpillEnabled; + } + + @Config("experimental.order-by-aggregation-spill-enabled") + @ConfigDescription("Spill order-by aggregations if aggregation spill is enabled") + public JavaFeaturesConfig setOrderByAggregationSpillEnabled(boolean orderByAggregationSpillEnabled) + { + this.orderByAggregationSpillEnabled = orderByAggregationSpillEnabled; + return this; + } + + public DataSize getDistinctAggregationLargeBlockSizeThreshold() + { + return distinctAggregationLargeBlockSizeThreshold; + } + + @Config("experimental.distinct-aggregation-large-block-size-threshold") + @ConfigDescription("Block size threshold beyond which it will be spilled into a separate spill file") + public JavaFeaturesConfig setDistinctAggregationLargeBlockSizeThreshold(DataSize distinctAggregationLargeBlockSizeThreshold) + { + this.distinctAggregationLargeBlockSizeThreshold = distinctAggregationLargeBlockSizeThreshold; + return this; + } + + public boolean isDistinctAggregationLargeBlockSpillEnabled() + { + return distinctAggregationLargeBlockSpillEnabled; + } + + @Config("experimental.distinct-aggregation-large-block-spill-enabled") + @ConfigDescription("Spill large block to a separate spill file") + public JavaFeaturesConfig setDistinctAggregationLargeBlockSpillEnabled(boolean distinctAggregationLargeBlockSpillEnabled) + { + this.distinctAggregationLargeBlockSpillEnabled = distinctAggregationLargeBlockSpillEnabled; + return this; + } + + public boolean isDedupBasedDistinctAggregationSpillEnabled() + { + return dedupBasedDistinctAggregationSpillEnabled; + } + + @Config("experimental.dedup-based-distinct-aggregation-spill-enabled") + @ConfigDescription("Dedup input data for Distinct Aggregates before spilling") + public JavaFeaturesConfig setDedupBasedDistinctAggregationSpillEnabled(boolean dedupBasedDistinctAggregationSpillEnabled) + { + this.dedupBasedDistinctAggregationSpillEnabled = dedupBasedDistinctAggregationSpillEnabled; + return this; + } + + public boolean isAggregationSpillEnabled() + { + return aggregationSpillEnabled; + } + + @Config("experimental.aggregation-spill-enabled") + @ConfigDescription("Spill aggregations if spill is enabled") + public JavaFeaturesConfig setAggregationSpillEnabled(boolean aggregationSpillEnabled) + { + this.aggregationSpillEnabled = aggregationSpillEnabled; + return this; + } + + @Config("experimental.topn-spill-enabled") + @ConfigDescription("Spill TopN if spill is enabled") + public JavaFeaturesConfig setTopNSpillEnabled(boolean topNSpillEnabled) + { + this.topNSpillEnabled = topNSpillEnabled; + return this; + } + + public boolean isTopNSpillEnabled() + { + return topNSpillEnabled; + } + + public boolean isDistinctAggregationSpillEnabled() + { + return distinctAggregationSpillEnabled; + } + + @Config("experimental.distinct-aggregation-spill-enabled") + @ConfigDescription("Spill distinct aggregations if aggregation spill is enabled") + public JavaFeaturesConfig setDistinctAggregationSpillEnabled(boolean distinctAggregationSpillEnabled) + { + this.distinctAggregationSpillEnabled = distinctAggregationSpillEnabled; + return this; + } + + @Config("experimental.aggregation-operator-unspill-memory-limit") + public JavaFeaturesConfig setAggregationOperatorUnspillMemoryLimit(DataSize aggregationOperatorUnspillMemoryLimit) + { + this.aggregationOperatorUnspillMemoryLimit = aggregationOperatorUnspillMemoryLimit; + return this; + } + + public DataSize getAggregationOperatorUnspillMemoryLimit() + { + return aggregationOperatorUnspillMemoryLimit; + } + + public DataSize getTopNOperatorUnspillMemoryLimit() + { + return topNOperatorUnspillMemoryLimit; + } + + @Config("experimental.topn-operator-unspill-memory-limit") + public JavaFeaturesConfig setTopNOperatorUnspillMemoryLimit(DataSize aggregationOperatorUnspillMemoryLimit) + { + this.topNOperatorUnspillMemoryLimit = aggregationOperatorUnspillMemoryLimit; + return this; + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java index eca1428de810..f232d9a7c528 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java @@ -255,7 +255,6 @@ import static com.facebook.airlift.concurrent.MoreFutures.addSuccessCallback; import static com.facebook.presto.SystemSessionProperties.getAdaptivePartialAggregationRowsReductionRatioThreshold; -import static com.facebook.presto.SystemSessionProperties.getAggregationOperatorUnspillMemoryLimit; import static com.facebook.presto.SystemSessionProperties.getDynamicFilteringMaxPerDriverRowCount; import static com.facebook.presto.SystemSessionProperties.getDynamicFilteringMaxPerDriverSize; import static com.facebook.presto.SystemSessionProperties.getDynamicFilteringRangeRowLimitPerDriver; @@ -265,23 +264,17 @@ import static com.facebook.presto.SystemSessionProperties.getTaskConcurrency; import static com.facebook.presto.SystemSessionProperties.getTaskPartitionedWriterCount; import static com.facebook.presto.SystemSessionProperties.getTaskWriterCount; -import static com.facebook.presto.SystemSessionProperties.getTopNOperatorUnspillMemoryLimit; import static com.facebook.presto.SystemSessionProperties.isAdaptivePartialAggregationEnabled; -import static com.facebook.presto.SystemSessionProperties.isAggregationSpillEnabled; -import static com.facebook.presto.SystemSessionProperties.isDistinctAggregationSpillEnabled; import static com.facebook.presto.SystemSessionProperties.isEnableDynamicFiltering; import static com.facebook.presto.SystemSessionProperties.isExchangeChecksumEnabled; import static com.facebook.presto.SystemSessionProperties.isExchangeCompressionEnabled; import static com.facebook.presto.SystemSessionProperties.isJoinSpillingEnabled; +import static com.facebook.presto.SystemSessionProperties.isNativeExecutionEnabled; import static com.facebook.presto.SystemSessionProperties.isOptimizeCommonSubExpressions; import static com.facebook.presto.SystemSessionProperties.isOptimizeJoinProbeForEmptyBuildRuntimeEnabled; import static com.facebook.presto.SystemSessionProperties.isOptimizedRepartitioningEnabled; -import static com.facebook.presto.SystemSessionProperties.isOrderByAggregationSpillEnabled; -import static com.facebook.presto.SystemSessionProperties.isOrderBySpillEnabled; import static com.facebook.presto.SystemSessionProperties.isQuickDistinctLimitEnabled; import static com.facebook.presto.SystemSessionProperties.isSpillEnabled; -import static com.facebook.presto.SystemSessionProperties.isTopNSpillEnabled; -import static com.facebook.presto.SystemSessionProperties.isWindowSpillEnabled; import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; import static com.facebook.presto.common.type.TypeUtils.writeNativeValue; @@ -307,6 +300,14 @@ import static com.facebook.presto.operator.WindowFunctionDefinition.window; import static com.facebook.presto.operator.aggregation.GenericAccumulatorFactory.generateAccumulatorFactory; import static com.facebook.presto.operator.unnest.UnnestOperator.UnnestOperatorFactory; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.getAggregationOperatorUnspillMemoryLimit; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.getTopNOperatorUnspillMemoryLimit; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.isAggregationSpillEnabled; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.isDistinctAggregationSpillEnabled; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.isOrderByAggregationSpillEnabled; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.isOrderBySpillEnabled; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.isTopNSpillEnabled; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.isWindowSpillEnabled; import static com.facebook.presto.spi.StandardErrorCode.COMPILER_ERROR; import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; @@ -1103,7 +1104,7 @@ public PhysicalOperation visitTopNRowNumber(TopNRowNumberNode node, LocalExecuti unspillMemoryLimit.toBytes(), joinCompiler, spillerFactory, - isTopNSpillEnabled(session)); + !isNativeExecutionEnabled(session) && isTopNSpillEnabled(session)); return new PhysicalOperation(operatorFactory, makeLayout(node), context, source); } 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 96c4c61beff7..1be93e222005 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 @@ -103,7 +103,6 @@ import com.facebook.presto.metadata.MetadataUtil; import com.facebook.presto.metadata.QualifiedTablePrefix; import com.facebook.presto.metadata.SchemaPropertyManager; -import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.metadata.Split; import com.facebook.presto.metadata.TablePropertyManager; import com.facebook.presto.operator.Driver; @@ -165,6 +164,7 @@ import com.facebook.presto.sql.analyzer.BuiltInQueryPreparerProvider; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.sql.analyzer.FunctionsConfig; +import com.facebook.presto.sql.analyzer.JavaFeaturesConfig; import com.facebook.presto.sql.analyzer.QueryExplainer; import com.facebook.presto.sql.analyzer.QueryPreparerProviderManager; import com.facebook.presto.sql.gen.ExpressionCompiler; @@ -265,6 +265,7 @@ import static com.facebook.presto.cost.StatsCalculatorModule.createNewStatsCalculator; import static com.facebook.presto.execution.scheduler.StreamingPlanSection.extractStreamingSections; import static com.facebook.presto.execution.scheduler.TableWriteInfo.createTableWriteInfo; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.GROUPED_SCHEDULING; import static com.facebook.presto.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.REWINDABLE_GROUPED_SCHEDULING; import static com.facebook.presto.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING; @@ -416,7 +417,7 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig, this.metadata = new MetadataManager( new FunctionAndTypeManager(transactionManager, blockEncodingManager, featuresConfig, functionsConfig, new HandleResolver(), ImmutableSet.of()), blockEncodingManager, - new SessionPropertyManager( + createTestingSessionPropertyManager( new SystemSessionProperties( new QueryManagerConfig(), new TaskManagerConfig(), @@ -429,7 +430,9 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig, new NodeSpillConfig(), new TracingConfig(), new CompilerConfig(), - new HistoryBasedOptimizationConfig())), + new HistoryBasedOptimizationConfig()).getSessionProperties(), + new JavaFeaturesConfig(), + nodeSpillConfig), new SchemaPropertyManager(), new TablePropertyManager(), new ColumnPropertyManager(), @@ -445,7 +448,7 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig, this.statsNormalizer = new StatsNormalizer(); this.scalarStatsCalculator = new ScalarStatsCalculator(metadata); this.filterStatsCalculator = new FilterStatsCalculator(metadata, scalarStatsCalculator, statsNormalizer); - this.historyBasedPlanStatisticsManager = new HistoryBasedPlanStatisticsManager(objectMapper, new SessionPropertyManager(), metadata, new HistoryBasedOptimizationConfig(), featuresConfig, new NodeVersion("1")); + this.historyBasedPlanStatisticsManager = new HistoryBasedPlanStatisticsManager(objectMapper, createTestingSessionPropertyManager(), metadata, new HistoryBasedOptimizationConfig(), featuresConfig, new NodeVersion("1")); this.fragmentStatsProvider = new FragmentStatsProvider(); this.statsCalculator = createNewStatsCalculator(metadata, scalarStatsCalculator, statsNormalizer, filterStatsCalculator, historyBasedPlanStatisticsManager, fragmentStatsProvider); this.taskCountEstimator = new TaskCountEstimator(() -> nodeCountForStats); diff --git a/presto-main/src/main/java/com/facebook/presto/testing/TestingSession.java b/presto-main/src/main/java/com/facebook/presto/testing/TestingSession.java index 35d5af930feb..f7fdf91411c0 100644 --- a/presto-main/src/main/java/com/facebook/presto/testing/TestingSession.java +++ b/presto-main/src/main/java/com/facebook/presto/testing/TestingSession.java @@ -32,6 +32,7 @@ import java.util.Optional; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.spi.ConnectorId.createInformationSchemaConnectorId; import static com.facebook.presto.spi.ConnectorId.createSystemTablesConnectorId; import static java.util.Locale.ENGLISH; @@ -54,7 +55,7 @@ private TestingSession() {} public static SessionBuilder testSessionBuilder() { - return testSessionBuilder(new SessionPropertyManager()); + return testSessionBuilder(createTestingSessionPropertyManager()); } public static SessionBuilder testSessionBuilder(SessionPropertyManager sessionPropertyManager) diff --git a/presto-main/src/test/java/com/facebook/presto/dispatcher/TestLocalDispatchQuery.java b/presto-main/src/test/java/com/facebook/presto/dispatcher/TestLocalDispatchQuery.java index eb632eb160f2..8d850679f8e3 100644 --- a/presto-main/src/test/java/com/facebook/presto/dispatcher/TestLocalDispatchQuery.java +++ b/presto-main/src/test/java/com/facebook/presto/dispatcher/TestLocalDispatchQuery.java @@ -30,7 +30,6 @@ import com.facebook.presto.execution.resourceGroups.QueryQueueFullException; import com.facebook.presto.metadata.InMemoryNodeManager; import com.facebook.presto.metadata.MetadataManager; -import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.operator.OperatorInfo; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.QueryId; @@ -67,6 +66,7 @@ import static com.facebook.presto.execution.QueryState.QUEUED; import static com.facebook.presto.execution.QueryState.WAITING_FOR_PREREQUISITES; import static com.facebook.presto.execution.TaskTestUtils.createQueryStateMachine; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.spi.StandardErrorCode.ABANDONED_QUERY; import static com.facebook.presto.spi.StandardErrorCode.ABANDONED_TASK; import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INSUFFICIENT_RESOURCES; @@ -455,10 +455,10 @@ private QueryMonitor createQueryMonitor(CountingEventListener eventListener) eventListenerManager, new NodeInfo("test"), UNKNOWN, - new SessionPropertyManager(), + createTestingSessionPropertyManager(), metadata, new QueryMonitorConfig(), - new HistoryBasedPlanStatisticsManager(new ObjectMapper(), new SessionPropertyManager(), metadata, new HistoryBasedOptimizationConfig(), new FeaturesConfig(), new NodeVersion("1")), + new HistoryBasedPlanStatisticsManager(new ObjectMapper(), createTestingSessionPropertyManager(), metadata, new HistoryBasedOptimizationConfig(), new FeaturesConfig(), new NodeVersion("1")), new FeaturesConfig()); } diff --git a/presto-main/src/test/java/com/facebook/presto/execution/scheduler/TestAdaptivePhasedExecutionPolicy.java b/presto-main/src/test/java/com/facebook/presto/execution/scheduler/TestAdaptivePhasedExecutionPolicy.java index 774500557501..ff7754e1a995 100644 --- a/presto-main/src/test/java/com/facebook/presto/execution/scheduler/TestAdaptivePhasedExecutionPolicy.java +++ b/presto-main/src/test/java/com/facebook/presto/execution/scheduler/TestAdaptivePhasedExecutionPolicy.java @@ -28,7 +28,6 @@ import com.facebook.presto.failureDetector.NoOpFailureDetector; import com.facebook.presto.memory.MemoryManagerConfig; import com.facebook.presto.memory.NodeMemoryConfig; -import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.spi.ConnectorId; import com.facebook.presto.spi.QueryId; import com.facebook.presto.spi.TableHandle; @@ -66,6 +65,7 @@ import static com.facebook.presto.SessionTestUtils.TEST_SESSION; import static com.facebook.presto.common.type.VarcharType.VARCHAR; import static com.facebook.presto.execution.SqlStageExecution.createSqlStageExecution; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION; import static com.facebook.presto.sql.planner.SystemPartitioningHandle.SOURCE_DISTRIBUTION; import static com.facebook.presto.sql.planner.plan.ExchangeNode.Type.REPARTITION; @@ -93,7 +93,7 @@ public void tearDownExecutor() @Test public void testCreateExecutionSchedule() { - Session session = testSessionBuilder(new SessionPropertyManager(new SystemSessionProperties( + Session session = testSessionBuilder(createTestingSessionPropertyManager(new SystemSessionProperties( new QueryManagerConfig(), new TaskManagerConfig(), new MemoryManagerConfig(), diff --git a/presto-main/src/test/java/com/facebook/presto/resourcemanager/TestResourceManagerClusterStateProvider.java b/presto-main/src/test/java/com/facebook/presto/resourcemanager/TestResourceManagerClusterStateProvider.java index 8c6edd0441de..9e7f6cf7aff4 100644 --- a/presto-main/src/test/java/com/facebook/presto/resourcemanager/TestResourceManagerClusterStateProvider.java +++ b/presto-main/src/test/java/com/facebook/presto/resourcemanager/TestResourceManagerClusterStateProvider.java @@ -20,7 +20,6 @@ import com.facebook.presto.memory.MemoryInfo; import com.facebook.presto.metadata.InMemoryNodeManager; import com.facebook.presto.metadata.InternalNode; -import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.server.BasicQueryInfo; import com.facebook.presto.server.BasicQueryStats; import com.facebook.presto.server.NodeStatus; @@ -59,6 +58,7 @@ import static com.facebook.presto.execution.QueryState.WAITING_FOR_RESOURCES; import static com.facebook.presto.memory.LocalMemoryManager.GENERAL_POOL; import static com.facebook.presto.memory.LocalMemoryManager.RESERVED_POOL; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.operator.BlockedReason.WAITING_FOR_MEMORY; import static io.airlift.units.DataSize.Unit.MEGABYTE; import static java.lang.String.format; @@ -78,7 +78,7 @@ public void testQueryInfo() nodeManager.addNode(new ConnectorId("x"), new InternalNode("node1", URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true)); nodeManager.addNode(new ConnectorId("x"), new InternalNode("node2", URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true)); - ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, new SessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("5s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); + ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, createTestingSessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("5s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); assertEquals(provider.getClusterQueries(), ImmutableList.of()); @@ -129,7 +129,7 @@ public void testOutOfOrderUpdatesIgnored() nodeManager.addNode(new ConnectorId("x"), new InternalNode("node1", URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true)); nodeManager.addNode(new ConnectorId("x"), new InternalNode("node2", URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true)); - ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, new SessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("5s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); + ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, createTestingSessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("5s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); assertEquals(provider.getClusterQueries(), ImmutableList.of()); @@ -157,7 +157,7 @@ public void testResourceGroups() nodeManager.addNode(new ConnectorId("x"), new InternalNode("node2", URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true)); nodeManager.addNode(new ConnectorId("x"), new InternalNode("node3", URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true)); - ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, new SessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("50s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); + ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, createTestingSessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("50s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); provider.registerNodeHeartbeat(createCoordinatorNodeStatus("local")); provider.registerNodeHeartbeat(createCoordinatorNodeStatus("node1")); provider.registerNodeHeartbeat(createCoordinatorNodeStatus("node2")); @@ -204,7 +204,7 @@ public void testResourceGroupsMerged() nodeManager.addNode(new ConnectorId("x"), new InternalNode("node5", URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true)); nodeManager.addNode(new ConnectorId("x"), new InternalNode("node6", URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true)); - ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, new SessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("50s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); + ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, createTestingSessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("50s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); provider.registerNodeHeartbeat(createCoordinatorNodeStatus("local")); provider.registerNodeHeartbeat(createCoordinatorNodeStatus("node1")); provider.registerNodeHeartbeat(createCoordinatorNodeStatus("node2")); @@ -280,7 +280,7 @@ public void testNonLeafResourceGroupsMerged() nodeManager.addNode(new ConnectorId("x"), new InternalNode("node4", URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true)); nodeManager.addNode(new ConnectorId("x"), new InternalNode("node5", URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true)); nodeManager.addNode(new ConnectorId("x"), new InternalNode("node6", URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true)); - ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, new SessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("50s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); + ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, createTestingSessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("50s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); provider.registerNodeHeartbeat(createCoordinatorNodeStatus("local")); provider.registerNodeHeartbeat(createCoordinatorNodeStatus("node1")); provider.registerNodeHeartbeat(createCoordinatorNodeStatus("node2")); @@ -394,7 +394,7 @@ public void testClusterMemoryPoolInfo() long query2Sequence = 0; long query3Sequence = 0; - ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, new SessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("4s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); + ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, createTestingSessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("4s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); // Memory pool starts off empty assertMemoryPoolMap(provider, 2, GENERAL_POOL, 0, 0, 0, 0, 0, Optional.empty()); @@ -481,7 +481,7 @@ public void testAdjustedQueueSize(Map> no InMemoryNodeManager nodeManager = new InMemoryNodeManager(); nodeHeartBeats.keySet().stream().forEach(nodeIdentifier -> nodeManager.addNode(new ConnectorId("x"), new InternalNode(nodeIdentifier, URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true))); - ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, new SessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("4s"), Duration.valueOf("0s"), Duration.valueOf("20s"), true, newSingleThreadScheduledExecutor()); + ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, createTestingSessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("4s"), Duration.valueOf("0s"), Duration.valueOf("20s"), true, newSingleThreadScheduledExecutor()); nodeHeartBeats.entrySet().stream().forEach(entry -> provider.registerResourceGroupRuntimeHeartbeat(entry.getKey(), entry.getValue())); Thread.sleep(SECONDS.toMillis(5)); @@ -492,7 +492,7 @@ public void testAdjustedQueueSize(Map> no public void testWorkerMemoryInfo() throws Exception { - ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(new InMemoryNodeManager(), new SessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("5s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); + ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(new InMemoryNodeManager(), createTestingSessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("5s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); assertWorkerMemoryInfo(provider, 0); @@ -515,7 +515,7 @@ public void testShuttingDownCoordinatorHeartbeat() InMemoryNodeManager nodeManager = new InMemoryNodeManager(); nodeManager.addShuttingDownNode(new InternalNode("node1", URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true)); - ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, new SessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("5s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); + ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, createTestingSessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("5s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); assertEquals(provider.getClusterQueries(), ImmutableList.of()); @@ -547,7 +547,7 @@ public void testRunningTaskCount() InMemoryNodeManager nodeManager = new InMemoryNodeManager(); nodeManager.addShuttingDownNode(new InternalNode("node1", URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true)); - ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, new SessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("5s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); + ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, createTestingSessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("5s"), Duration.valueOf("0s"), Duration.valueOf("4s"), true, newSingleThreadScheduledExecutor()); assertEquals(provider.getRunningTaskCount(), 0); @@ -587,7 +587,7 @@ public void testResourceGroupStatsExpiry() InMemoryNodeManager nodeManager = new InMemoryNodeManager(); - ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, new SessionPropertyManager(), 10, Duration.valueOf("4s"), + ResourceManagerClusterStateProvider provider = new ResourceManagerClusterStateProvider(nodeManager, createTestingSessionPropertyManager(), 10, Duration.valueOf("4s"), Duration.valueOf("8s"), Duration.valueOf("5s"), Duration.valueOf("10s"), Duration.valueOf("10s"), true, newSingleThreadScheduledExecutor()); resourceGroupStates.entrySet().stream().forEach(entry -> provider.registerResourceGroupRuntimeHeartbeat(entry.getKey(), entry.getValue())); diff --git a/presto-main/src/test/java/com/facebook/presto/server/TestQuerySessionSupplier.java b/presto-main/src/test/java/com/facebook/presto/server/TestQuerySessionSupplier.java index e8c4d37ca8c7..d4eb03bb64a8 100644 --- a/presto-main/src/test/java/com/facebook/presto/server/TestQuerySessionSupplier.java +++ b/presto-main/src/test/java/com/facebook/presto/server/TestQuerySessionSupplier.java @@ -17,7 +17,6 @@ import com.facebook.presto.common.WarningHandlingLevel; import com.facebook.presto.common.type.TimeZoneNotSupportedException; import com.facebook.presto.execution.warnings.WarningCollectorFactory; -import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.server.security.SecurityConfig; import com.facebook.presto.spi.QueryId; import com.facebook.presto.spi.WarningCollector; @@ -52,6 +51,7 @@ import static com.facebook.presto.client.PrestoHeaders.PRESTO_TIME_ZONE; import static com.facebook.presto.client.PrestoHeaders.PRESTO_USER; import static com.facebook.presto.common.type.TimeZoneKey.getTimeZoneKey; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.server.TestHttpRequestSessionContext.createFunctionAdd; import static com.facebook.presto.server.TestHttpRequestSessionContext.createSqlFunctionIdAdd; import static com.facebook.presto.server.TestHttpRequestSessionContext.urlEncode; @@ -93,7 +93,7 @@ public void testCreateSession() QuerySessionSupplier sessionSupplier = new QuerySessionSupplier( createTestTransactionManager(), new AllowAllAccessControl(), - new SessionPropertyManager(), + createTestingSessionPropertyManager(), new SqlEnvironmentConfig(), new SecurityConfig()); WarningCollectorFactory warningCollectorFactory = new WarningCollectorFactory() @@ -167,7 +167,7 @@ public void testInvalidTimeZone() QuerySessionSupplier sessionSupplier = new QuerySessionSupplier( createTestTransactionManager(), new AllowAllAccessControl(), - new SessionPropertyManager(), + createTestingSessionPropertyManager(), new SqlEnvironmentConfig(), new SecurityConfig()); WarningCollectorFactory warningCollectorFactory = new WarningCollectorFactory() diff --git a/presto-main/src/test/java/com/facebook/presto/server/TestSessionPropertyDefaults.java b/presto-main/src/test/java/com/facebook/presto/server/TestSessionPropertyDefaults.java index 034d81caaff7..13693673fc12 100644 --- a/presto-main/src/test/java/com/facebook/presto/server/TestSessionPropertyDefaults.java +++ b/presto-main/src/test/java/com/facebook/presto/server/TestSessionPropertyDefaults.java @@ -16,7 +16,6 @@ import com.facebook.airlift.node.NodeInfo; import com.facebook.presto.Session; import com.facebook.presto.client.NodeVersion; -import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.spi.QueryId; import com.facebook.presto.spi.resourceGroups.ResourceGroupId; import com.facebook.presto.spi.security.Identity; @@ -32,6 +31,7 @@ import static com.facebook.presto.SystemSessionProperties.HASH_PARTITION_COUNT; import static com.facebook.presto.SystemSessionProperties.JOIN_DISTRIBUTION_TYPE; import static com.facebook.presto.SystemSessionProperties.QUERY_MAX_MEMORY; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static org.testng.Assert.assertEquals; public class TestSessionPropertyDefaults @@ -46,11 +46,11 @@ public void testApplyDefaultProperties() SessionPropertyDefaults sessionPropertyDefaults = new SessionPropertyDefaults(TEST_NODE_INFO, TEST_NODE_VERSION); SessionPropertyConfigurationManagerFactory factory = new TestingSessionPropertyConfigurationManagerFactory( new SystemSessionPropertyConfiguration( - ImmutableMap.builder() - .put(QUERY_MAX_MEMORY, "override") - .put("system_default", "system_default") - .build(), - ImmutableMap.of("override", "overridden")), + ImmutableMap.builder() + .put(QUERY_MAX_MEMORY, "override") + .put("system_default", "system_default") + .build(), + ImmutableMap.of("override", "overridden")), ImmutableMap.of( "testCatalog", ImmutableMap.builder() @@ -60,7 +60,7 @@ public void testApplyDefaultProperties() sessionPropertyDefaults.addConfigurationManagerFactory(factory); sessionPropertyDefaults.setConfigurationManager(factory.getName(), ImmutableMap.of()); - SessionBuilder sessionBuilder = Session.builder(new SessionPropertyManager()) + SessionBuilder sessionBuilder = Session.builder(createTestingSessionPropertyManager()) .setQueryId(new QueryId("test_query_id")) .setIdentity(new Identity("testUser", Optional.empty())) .setSystemProperty(QUERY_MAX_MEMORY, "1GB") diff --git a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestAnalyzer.java b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestAnalyzer.java index 1d5fd3a3c422..83616f63c777 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestAnalyzer.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestAnalyzer.java @@ -22,7 +22,6 @@ import com.facebook.presto.execution.warnings.WarningCollectorConfig; import com.facebook.presto.memory.MemoryManagerConfig; import com.facebook.presto.memory.NodeMemoryConfig; -import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.spi.PrestoWarning; import com.facebook.presto.spi.StandardWarningCode; import com.facebook.presto.spi.WarningCollector; @@ -34,6 +33,7 @@ import java.util.List; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.spi.StandardWarningCode.PERFORMANCE_WARNING; import static com.facebook.presto.spi.StandardWarningCode.SEMANTIC_WARNING; import static com.facebook.presto.sql.analyzer.SemanticErrorCode.AMBIGUOUS_ATTRIBUTE; @@ -192,7 +192,7 @@ public void testWindowOrderByAnalysis() "FROM (values (1,10), (2, 10)) AS T(x, y)"), PERFORMANCE_WARNING, "ORDER BY literals/constants with window function:"); // Now test for error when the session param is set to disallow this. - Session session = testSessionBuilder(new SessionPropertyManager(new SystemSessionProperties( + Session session = testSessionBuilder(createTestingSessionPropertyManager(new SystemSessionProperties( new QueryManagerConfig(), new TaskManagerConfig(), new MemoryManagerConfig(), @@ -588,7 +588,7 @@ public void testOrderByWithGroupByAndSubquerySelectExpression() @Test public void testTooManyGroupingElements() { - Session session = testSessionBuilder(new SessionPropertyManager(new SystemSessionProperties( + Session session = testSessionBuilder(createTestingSessionPropertyManager(new SystemSessionProperties( new QueryManagerConfig(), new TaskManagerConfig(), new MemoryManagerConfig(), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java index ebe971197a36..332ebeac1423 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java @@ -102,17 +102,6 @@ public void testDefaults() .setAggregationPartitioningMergingStrategy(LEGACY) .setSpillEnabled(false) .setJoinSpillingEnabled(true) - .setAggregationSpillEnabled(true) - .setDistinctAggregationSpillEnabled(true) - .setDedupBasedDistinctAggregationSpillEnabled(false) - .setDistinctAggregationLargeBlockSpillEnabled(false) - .setDistinctAggregationLargeBlockSizeThreshold(DataSize.valueOf("50MB")) - .setOrderByAggregationSpillEnabled(true) - .setWindowSpillEnabled(true) - .setOrderBySpillEnabled(true) - .setTopNSpillEnabled(true) - .setAggregationOperatorUnspillMemoryLimit(DataSize.valueOf("4MB")) - .setTopNOperatorUnspillMemoryLimit(DataSize.valueOf("4MB")) .setSpillerSpillPaths("") .setSpillerThreads(4) .setSpillMaxUsedSpaceThreshold(0.9) @@ -323,17 +312,6 @@ public void testExplicitPropertyMappings() .put("optimizer.aggregation-partition-merging", "top_down") .put("experimental.spill-enabled", "true") .put("experimental.join-spill-enabled", "false") - .put("experimental.aggregation-spill-enabled", "false") - .put("experimental.distinct-aggregation-spill-enabled", "false") - .put("experimental.dedup-based-distinct-aggregation-spill-enabled", "true") - .put("experimental.distinct-aggregation-large-block-spill-enabled", "true") - .put("experimental.distinct-aggregation-large-block-size-threshold", "10MB") - .put("experimental.order-by-aggregation-spill-enabled", "false") - .put("experimental.window-spill-enabled", "false") - .put("experimental.order-by-spill-enabled", "false") - .put("experimental.topn-spill-enabled", "false") - .put("experimental.aggregation-operator-unspill-memory-limit", "100MB") - .put("experimental.topn-operator-unspill-memory-limit", "100MB") .put("experimental.spiller-spill-path", "/tmp/custom/spill/path1,/tmp/custom/spill/path2") .put("experimental.spiller-threads", "42") .put("experimental.spiller-max-used-space-threshold", "0.8") @@ -525,17 +503,6 @@ public void testExplicitPropertyMappings() .setPushAggregationThroughJoin(false) .setSpillEnabled(true) .setJoinSpillingEnabled(false) - .setAggregationSpillEnabled(false) - .setDistinctAggregationSpillEnabled(false) - .setDedupBasedDistinctAggregationSpillEnabled(true) - .setDistinctAggregationLargeBlockSpillEnabled(true) - .setDistinctAggregationLargeBlockSizeThreshold(DataSize.valueOf("10MB")) - .setOrderByAggregationSpillEnabled(false) - .setWindowSpillEnabled(false) - .setOrderBySpillEnabled(false) - .setTopNSpillEnabled(false) - .setAggregationOperatorUnspillMemoryLimit(DataSize.valueOf("100MB")) - .setTopNOperatorUnspillMemoryLimit(DataSize.valueOf("100MB")) .setSpillerSpillPaths("/tmp/custom/spill/path1,/tmp/custom/spill/path2") .setSpillerThreads(42) .setSpillMaxUsedSpaceThreshold(0.8) diff --git a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestJavaFeaturesConfig.java b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestJavaFeaturesConfig.java new file mode 100644 index 000000000000..f83f6b3eb03b --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestJavaFeaturesConfig.java @@ -0,0 +1,76 @@ +/* + * 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.airlift.configuration.testing.ConfigAssertions; +import com.google.common.collect.ImmutableMap; +import io.airlift.units.DataSize; +import org.testng.annotations.Test; + +import java.util.Map; + +import static com.facebook.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static com.facebook.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; + +public class TestJavaFeaturesConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(ConfigAssertions.recordDefaults(JavaFeaturesConfig.class) + .setAggregationSpillEnabled(true) + .setDistinctAggregationSpillEnabled(true) + .setDedupBasedDistinctAggregationSpillEnabled(false) + .setDistinctAggregationLargeBlockSpillEnabled(false) + .setDistinctAggregationLargeBlockSizeThreshold(DataSize.valueOf("50MB")) + .setOrderByAggregationSpillEnabled(true) + .setWindowSpillEnabled(true) + .setOrderBySpillEnabled(true) + .setTopNSpillEnabled(true) + .setAggregationOperatorUnspillMemoryLimit(DataSize.valueOf("4MB")) + .setTopNOperatorUnspillMemoryLimit(DataSize.valueOf("4MB"))); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("experimental.aggregation-spill-enabled", "false") + .put("experimental.distinct-aggregation-spill-enabled", "false") + .put("experimental.dedup-based-distinct-aggregation-spill-enabled", "true") + .put("experimental.distinct-aggregation-large-block-spill-enabled", "true") + .put("experimental.distinct-aggregation-large-block-size-threshold", "10MB") + .put("experimental.order-by-aggregation-spill-enabled", "false") + .put("experimental.window-spill-enabled", "false") + .put("experimental.order-by-spill-enabled", "false") + .put("experimental.topn-spill-enabled", "false") + .put("experimental.aggregation-operator-unspill-memory-limit", "100MB") + .put("experimental.topn-operator-unspill-memory-limit", "100MB") + .build(); + + JavaFeaturesConfig expected = new JavaFeaturesConfig() + .setAggregationSpillEnabled(false) + .setDistinctAggregationSpillEnabled(false) + .setDedupBasedDistinctAggregationSpillEnabled(true) + .setDistinctAggregationLargeBlockSpillEnabled(true) + .setDistinctAggregationLargeBlockSizeThreshold(DataSize.valueOf("10MB")) + .setOrderByAggregationSpillEnabled(false) + .setWindowSpillEnabled(false) + .setOrderBySpillEnabled(false) + .setTopNSpillEnabled(false) + .setAggregationOperatorUnspillMemoryLimit(DataSize.valueOf("100MB")) + .setTopNOperatorUnspillMemoryLimit(DataSize.valueOf("100MB")); + assertFullMapping(properties, expected); + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleTester.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleTester.java index bad0f6b342d1..3c9540ef42b2 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleTester.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleTester.java @@ -43,6 +43,7 @@ import java.util.Optional; import java.util.Set; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static java.util.Collections.emptyList; @@ -83,7 +84,7 @@ public RuleTester(List plugins, Map sessionProperties, O public RuleTester(List plugins, Map sessionProperties, Optional nodeCountForStats, ConnectorFactory connectorFactory) { - this(plugins, sessionProperties, new SessionPropertyManager(), nodeCountForStats, connectorFactory); + this(plugins, sessionProperties, createTestingSessionPropertyManager(), nodeCountForStats, connectorFactory); } public RuleTester(List plugins, Map sessionProperties, SessionPropertyManager sessionPropertyManager, Optional nodeCountForStats, ConnectorFactory connectorFactory) diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestValidateStreamingJoins.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestValidateStreamingJoins.java index 4edbbdee229b..caa37100245c 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestValidateStreamingJoins.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestValidateStreamingJoins.java @@ -24,7 +24,6 @@ import com.facebook.presto.memory.MemoryManagerConfig; import com.facebook.presto.memory.NodeMemoryConfig; import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorId; import com.facebook.presto.spi.TableHandle; @@ -54,6 +53,7 @@ import java.util.function.Function; import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.spi.plan.JoinType.INNER; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; @@ -77,7 +77,7 @@ public void setup() .setSchema("tiny"); defaultSession = sessionBuilder.build(); spillSession = testSessionBuilder( - new SessionPropertyManager(new SystemSessionProperties( + createTestingSessionPropertyManager(new SystemSessionProperties( new QueryManagerConfig(), new TaskManagerConfig(), new MemoryManagerConfig(), 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 090b7375b320..646795365133 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 @@ -19,10 +19,13 @@ import com.facebook.presto.spi.ConnectorTableMetadata; import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.sql.analyzer.FeaturesConfig; +import com.facebook.presto.testing.MaterializedResult; +import com.facebook.presto.testing.MaterializedRow; import com.facebook.presto.testing.QueryRunner; import com.facebook.presto.tests.AbstractTestQueryFramework; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.intellij.lang.annotations.Language; import org.testng.annotations.Test; import java.util.ArrayList; @@ -30,6 +33,7 @@ import java.util.Map; import java.util.Optional; import java.util.UUID; +import java.util.regex.Pattern; import static com.facebook.presto.SystemSessionProperties.INLINE_SQL_FUNCTIONS; import static com.facebook.presto.SystemSessionProperties.JOIN_DISTRIBUTION_TYPE; @@ -73,6 +77,7 @@ import static com.facebook.presto.sql.planner.plan.ExchangeNode.Type.REPARTITION; import static com.facebook.presto.transaction.TransactionBuilder.transaction; import static java.lang.String.format; +import static java.util.stream.Collectors.toList; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; @@ -320,7 +325,8 @@ public void testAnalyzeStatsOnDecimals() } @Test - public void testIPAddressIPPrefix() throws InterruptedException + public void testIPAddressIPPrefix() + throws InterruptedException { String tmpTableName = generateRandomTableName(); try { @@ -346,6 +352,7 @@ public void testIPAddressIPPrefix() throws InterruptedException dropTableIfExists(tmpTableName); } } + @Test public void testTableSample() { @@ -1167,6 +1174,33 @@ public void testShowAndDescribe() assertQuery("DESCRIBE lineitem"); } + @Test + public void testShowSessionWithoutJavaSessionProperties() + { + // SHOW SESSION will exclude java-worker session properties + @Language("SQL") String sql = "SHOW SESSION"; + MaterializedResult actualResult = computeActual(sql); + List actualRows = actualResult.getMaterializedRows(); + + String javaSessionProperty = "distinct_aggregation_spill_enabled"; + List filteredRows = getJavaWorkerSessionProperties(actualRows, javaSessionProperty); + assertTrue(filteredRows.isEmpty()); + } + + @Test + public void testSetSessionJavaWorkerSessionProperty() + { + // SET SESSION on a java-worker session property + @Language("SQL") String setSession = "SET SESSION distinct_aggregation_spill_enabled=false"; + MaterializedResult setSessionResult = computeActual(setSession); + assertEquals( + setSessionResult.toString(), + "MaterializedResult{rows=[[true]], " + + "types=[boolean], " + + "setSessionProperties={distinct_aggregation_spill_enabled=false}, " + + "resetSessionProperties=[], updateType=SET SESSION}"); + } + @Test public void testBucketedExecution() { @@ -1826,4 +1860,11 @@ private void assertQueryResultCount(Session session, String sql, int expectedRes { assertEquals(getQueryRunner().execute(session, sql).getRowCount(), expectedResultCount); } + + private List getJavaWorkerSessionProperties(List inputRows, String sessionPropertyName) + { + return inputRows.stream() + .filter(row -> Pattern.matches(sessionPropertyName, row.getFields().get(4).toString())) + .collect(toList()); + } } diff --git a/presto-native-execution/src/test/java/com/facebook/presto/spark/TestPrestoSparkNativeGeneralQueries.java b/presto-native-execution/src/test/java/com/facebook/presto/spark/TestPrestoSparkNativeGeneralQueries.java index e3706ee19e8e..fba01435cd8c 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/spark/TestPrestoSparkNativeGeneralQueries.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/spark/TestPrestoSparkNativeGeneralQueries.java @@ -59,4 +59,12 @@ public void testAnalyzeStats() {} @Override @Ignore public void testUnionAllInsert() {} + + @Override + @Ignore + public void testShowSessionWithoutJavaSessionProperties() {} + + @Override + @Ignore + public void testSetSessionJavaWorkerSessionProperty() {} } diff --git a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotQueryBase.java b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotQueryBase.java index e36fc48338d6..ea1bc800a091 100644 --- a/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotQueryBase.java +++ b/presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotQueryBase.java @@ -21,7 +21,6 @@ import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.MetadataManager; -import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.pinot.query.PinotQueryGeneratorContext; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorId; @@ -72,6 +71,7 @@ import static com.facebook.presto.common.type.TimestampType.TIMESTAMP; import static com.facebook.presto.common.type.VarcharType.VARCHAR; import static com.facebook.presto.metadata.FunctionAndTypeManager.createTestFunctionAndTypeManager; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.pinot.PinotColumnHandle.PinotColumnType.REGULAR; import static com.facebook.presto.pinot.query.PinotQueryGeneratorContext.Origin.DERIVED; import static com.facebook.presto.pinot.query.PinotQueryGeneratorContext.Origin.TABLE_COLUMN; @@ -139,7 +139,7 @@ protected static class SessionHolder public SessionHolder(PinotConfig pinotConfig) { connectorSession = new TestingConnectorSession(new PinotSessionProperties(pinotConfig).getSessionProperties()); - session = TestingSession.testSessionBuilder(new SessionPropertyManager(new SystemSessionProperties().getSessionProperties())).build(); + session = TestingSession.testSessionBuilder(createTestingSessionPropertyManager(new SystemSessionProperties().getSessionProperties())).build(); } public SessionHolder(boolean useDateTrunc) 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 9e258ccf9c01..c26cba7e264f 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 @@ -168,6 +168,7 @@ import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.sql.analyzer.ForMetadataExtractor; import com.facebook.presto.sql.analyzer.FunctionsConfig; +import com.facebook.presto.sql.analyzer.JavaFeaturesConfig; import com.facebook.presto.sql.analyzer.MetadataExtractor; import com.facebook.presto.sql.analyzer.MetadataExtractorMBean; import com.facebook.presto.sql.analyzer.QueryExplainer; @@ -258,6 +259,7 @@ protected void setup(Binder binder) configBinder(binder).bindConfigGlobalDefaults(QueryManagerConfig.class, PrestoSparkSettingsRequirements::setDefaults); configBinder(binder).bindConfig(FeaturesConfig.class); configBinder(binder).bindConfig(FunctionsConfig.class); + configBinder(binder).bindConfig(JavaFeaturesConfig.class); configBinder(binder).bindConfigGlobalDefaults(FeaturesConfig.class, PrestoSparkSettingsRequirements::setDefaults); configBinder(binder).bindConfig(MemoryManagerConfig.class); configBinder(binder).bindConfig(TaskManagerConfig.class); diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkSessionPropertyManagerProvider.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkSessionPropertyManagerProvider.java index 00c088795276..6b82ddbb85ce 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkSessionPropertyManagerProvider.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkSessionPropertyManagerProvider.java @@ -20,6 +20,7 @@ import javax.inject.Inject; import javax.inject.Provider; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.google.common.collect.ImmutableList.toImmutableList; import static java.util.Objects.requireNonNull; @@ -39,9 +40,9 @@ public PrestoSparkSessionPropertyManagerProvider(SystemSessionProperties systemS @Override public SessionPropertyManager get() { - return new SessionPropertyManager(Streams.concat( - systemSessionProperties.getSessionProperties().stream(), - prestoSparkSessionProperties.getSessionProperties().stream()) + return createTestingSessionPropertyManager(Streams.concat( + systemSessionProperties.getSessionProperties().stream(), + prestoSparkSessionProperties.getSessionProperties().stream()) .collect(toImmutableList())); } } diff --git a/presto-spark-base/src/test/java/com/facebook/presto/spark/planner/TestPrestoSparkPhysicalResourceAllocationStrategy.java b/presto-spark-base/src/test/java/com/facebook/presto/spark/planner/TestPrestoSparkPhysicalResourceAllocationStrategy.java index 6b204e7c9d1f..79280f9a03fd 100644 --- a/presto-spark-base/src/test/java/com/facebook/presto/spark/planner/TestPrestoSparkPhysicalResourceAllocationStrategy.java +++ b/presto-spark-base/src/test/java/com/facebook/presto/spark/planner/TestPrestoSparkPhysicalResourceAllocationStrategy.java @@ -16,7 +16,6 @@ import com.facebook.presto.Session; import com.facebook.presto.metadata.AbstractMockMetadata; import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.spark.PrestoSparkPhysicalResourceCalculator; import com.facebook.presto.spark.PrestoSparkSourceStatsCollector; import com.facebook.presto.spi.ColumnHandle; @@ -41,6 +40,7 @@ import java.util.List; import static com.facebook.presto.SystemSessionProperties.HASH_PARTITION_COUNT; +import static com.facebook.presto.metadata.SessionPropertyManager.createTestingSessionPropertyManager; import static com.facebook.presto.spark.PrestoSparkSessionProperties.SPARK_AVERAGE_INPUT_DATA_SIZE_PER_EXECUTOR; import static com.facebook.presto.spark.PrestoSparkSessionProperties.SPARK_AVERAGE_INPUT_DATA_SIZE_PER_PARTITION; import static com.facebook.presto.spark.PrestoSparkSessionProperties.SPARK_EXECUTOR_ALLOCATION_STRATEGY_ENABLED; @@ -85,12 +85,12 @@ public TableStatistics getTableStatistics(Session session, TableHandle tableHand PropertyMetadata.integerProperty(HASH_PARTITION_COUNT, "HASH_PARTITION_COUNT", 150, false) }; // system property with allocation based tuning enabled - private static final Session testSessionWithAllocation = testSessionBuilder(new SessionPropertyManager( + private static final Session testSessionWithAllocation = testSessionBuilder(createTestingSessionPropertyManager( new ImmutableList.Builder>().add(defaultPropertyMetadata).add( PropertyMetadata.booleanProperty(SPARK_RESOURCE_ALLOCATION_STRATEGY_ENABLED, "SPARK_RESOURCE_ALLOCATION_STRATEGY_ENABLED", true, false) ).build())).build(); // system property with allocation based tuning disabled - private static final Session testSessionWithoutAllocation = testSessionBuilder(new SessionPropertyManager( + private static final Session testSessionWithoutAllocation = testSessionBuilder(createTestingSessionPropertyManager( new ImmutableList.Builder>().add(defaultPropertyMetadata).add( PropertyMetadata.booleanProperty(SPARK_RESOURCE_ALLOCATION_STRATEGY_ENABLED, "SPARK_RESOURCE_ALLOCATION_STRATEGY_ENABLED", false, false), PropertyMetadata.booleanProperty(SPARK_HASH_PARTITION_COUNT_ALLOCATION_STRATEGY_ENABLED, "SPARK_HASH_PARTITION_COUNT_ALLOCATION_STRATEGY_ENABLED", false, false), diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/session/SessionPropertyContext.java b/presto-spi/src/main/java/com/facebook/presto/spi/session/SessionPropertyContext.java new file mode 100644 index 000000000000..7b778281ebd3 --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/session/SessionPropertyContext.java @@ -0,0 +1,43 @@ +/* + * 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.session; + +import com.facebook.presto.common.type.TypeManager; +import com.facebook.presto.spi.NodeManager; + +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public class SessionPropertyContext +{ + private final Optional typeManager; + private final Optional nodeManager; + + public SessionPropertyContext(Optional typeManager, Optional nodeManager) + { + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); + } + + public NodeManager getNodeManager() + { + return nodeManager.orElseThrow(() -> new IllegalArgumentException("nodeManager is not present")); + } + + public TypeManager getTypeManager() + { + return typeManager.orElseThrow(() -> new IllegalArgumentException("typeManager is not present")); + } +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/session/WorkerSessionPropertyProvider.java b/presto-spi/src/main/java/com/facebook/presto/spi/session/WorkerSessionPropertyProvider.java new file mode 100644 index 000000000000..8f3bca2a538f --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/session/WorkerSessionPropertyProvider.java @@ -0,0 +1,22 @@ +/* + * 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.session; + +import java.util.List; + +public interface WorkerSessionPropertyProvider +{ + List> getSessionProperties(); +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/session/WorkerSessionPropertyProviderFactory.java b/presto-spi/src/main/java/com/facebook/presto/spi/session/WorkerSessionPropertyProviderFactory.java new file mode 100644 index 000000000000..3e95a13b1619 --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/session/WorkerSessionPropertyProviderFactory.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.session; + +public interface WorkerSessionPropertyProviderFactory +{ + String getName(); + + WorkerSessionPropertyProvider create(SessionPropertyContext context); +} diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestQueries.java b/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestQueries.java index 588ee48790ad..787d43f37279 100644 --- a/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestQueries.java +++ b/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestQueries.java @@ -48,6 +48,7 @@ import java.util.List; import java.util.Optional; import java.util.Set; +import java.util.regex.Pattern; import java.util.stream.IntStream; import static com.facebook.presto.SystemSessionProperties.ADD_PARTIAL_NODE_FOR_ROW_NUMBER_WITH_LIMIT; @@ -3126,6 +3127,32 @@ public void testShowSession() } } + @Test + public void testShowSessionWithoutNativeSessionProperties() + { + // SHOW SESSION will exclude native-worker session properties + @Language("SQL") String sql = "SHOW SESSION"; + MaterializedResult actualResult = computeActual(sql); + List actualRows = actualResult.getMaterializedRows(); + String nativeSessionProperty = "native_expression_max_array_size_in_reduce"; + List filteredRows = getNativeWorkerSessionProperties(actualRows, nativeSessionProperty); + assertTrue(filteredRows.isEmpty()); + } + + @Test + public void testSetSessionNativeWorkerSessionProperty() + { + // SET SESSION on a native-worker session property + @Language("SQL") String setSession = "SET SESSION native_expression_max_array_size_in_reduce=50000"; + MaterializedResult setSessionResult = computeActual(setSession); + assertEquals( + setSessionResult.toString(), + "MaterializedResult{rows=[[true]], " + + "types=[boolean], " + + "setSessionProperties={native_expression_max_array_size_in_reduce=50000}, " + + "resetSessionProperties=[], updateType=SET SESSION}"); + } + @Test public void testTry() { @@ -7928,4 +7955,11 @@ public void testEvaluateProjectOnValues() "SELECT a * 2, a - 1 FROM (SELECT x * 2 as a FROM (VALUES 15) t(x))", "SELECT * FROM (VALUES (60, 29))"); } + + private List getNativeWorkerSessionProperties(List inputRows, String sessionPropertyName) + { + return inputRows.stream() + .filter(row -> Pattern.matches(sessionPropertyName, row.getFields().get(4).toString())) + .collect(toList()); + } } diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestDistributedSpilledQueries.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestDistributedSpilledQueries.java index 5cd3335009a1..6ba69a8c68d2 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestDistributedSpilledQueries.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestDistributedSpilledQueries.java @@ -22,6 +22,8 @@ import java.nio.file.Paths; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; @@ -42,9 +44,9 @@ public static QueryRunner localCreateQueryRunner() .setCatalog("tpch") .setSchema(TINY_SCHEMA_NAME) .setSystemProperty(SystemSessionProperties.TASK_CONCURRENCY, "2") - .setSystemProperty(SystemSessionProperties.AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT, "128kB") + .setSystemProperty(AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT, "128kB") .setSystemProperty(SystemSessionProperties.USE_MARK_DISTINCT, "false") - .setSystemProperty(SystemSessionProperties.DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED, "false") + .setSystemProperty(DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED, "false") .build(); ImmutableMap extraProperties = ImmutableMap.builder() diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestDistributedSpilledQueriesWithTempStorage.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestDistributedSpilledQueriesWithTempStorage.java index 8109cb95a4f8..c4ca56a3f78c 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestDistributedSpilledQueriesWithTempStorage.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestDistributedSpilledQueriesWithTempStorage.java @@ -19,6 +19,7 @@ import com.facebook.presto.tpch.TpchPlugin; import com.google.common.collect.ImmutableMap; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; @@ -40,7 +41,7 @@ public static DistributedQueryRunner localCreateQueryRunner() .setSchema(TINY_SCHEMA_NAME) .setSystemProperty(SystemSessionProperties.TASK_CONCURRENCY, "2") .setSystemProperty(SystemSessionProperties.SPILL_ENABLED, "true") - .setSystemProperty(SystemSessionProperties.AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT, "128kB") + .setSystemProperty(AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT, "128kB") .setSystemProperty(SystemSessionProperties.USE_MARK_DISTINCT, "false") .build(); diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalQueries.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalQueries.java index 730d87e30ce4..63d4fc2e5838 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalQueries.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalQueries.java @@ -144,4 +144,9 @@ public void testIOExplain() jsonCodec(IOPlan.class).fromJson((String) getOnlyElement(result.getOnlyColumnAsSet())), new IOPlan(ImmutableSet.of(input), Optional.empty())); } + + @Override + public void testSetSessionNativeWorkerSessionProperty() + { + } } diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestQueryPlanDeterminism.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestQueryPlanDeterminism.java index ed8a304ffca6..9f51fc213040 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestQueryPlanDeterminism.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestQueryPlanDeterminism.java @@ -186,6 +186,11 @@ public void testExplainValidate() { } + @Override + public void testSetSessionNativeWorkerSessionProperty() + { + } + @Override protected void assertAccessAllowed(@Language("SQL") String sql, TestingAccessControlManager.TestingPrivilege... deniedPrivileges) { diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregationWithHighMemoryRevokingThreshold.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregationWithHighMemoryRevokingThreshold.java index 83f78b4ddd1e..4b4d1fca82b0 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregationWithHighMemoryRevokingThreshold.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregationWithHighMemoryRevokingThreshold.java @@ -21,6 +21,8 @@ import java.nio.file.Paths; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; @@ -35,9 +37,9 @@ protected QueryRunner createQueryRunner() .setCatalog("tpch") .setSchema(TINY_SCHEMA_NAME) .setSystemProperty(SystemSessionProperties.TASK_CONCURRENCY, "2") - .setSystemProperty(SystemSessionProperties.AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT, "128kB") + .setSystemProperty(AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT, "128kB") .setSystemProperty(SystemSessionProperties.USE_MARK_DISTINCT, "false") - .setSystemProperty(SystemSessionProperties.DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED, "true") + .setSystemProperty(DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED, "true") .build(); ImmutableMap extraProperties = ImmutableMap.builder() diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregationWithLargeBlockSpillingEnabled.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregationWithLargeBlockSpillingEnabled.java index ec7581029a68..18ff403b31cf 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregationWithLargeBlockSpillingEnabled.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregationWithLargeBlockSpillingEnabled.java @@ -21,8 +21,10 @@ import java.nio.file.Paths; -import static com.facebook.presto.SystemSessionProperties.DISTINCT_AGGREGATION_LARGE_BLOCK_SIZE_THRESHOLD; -import static com.facebook.presto.SystemSessionProperties.DISTINCT_AGGREGATION_LARGE_BLOCK_SPILL_ENABLED; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.DISTINCT_AGGREGATION_LARGE_BLOCK_SIZE_THRESHOLD; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.DISTINCT_AGGREGATION_LARGE_BLOCK_SPILL_ENABLED; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; @@ -37,9 +39,9 @@ protected QueryRunner createQueryRunner() .setCatalog("tpch") .setSchema(TINY_SCHEMA_NAME) .setSystemProperty(SystemSessionProperties.TASK_CONCURRENCY, "2") - .setSystemProperty(SystemSessionProperties.AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT, "128kB") + .setSystemProperty(AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT, "128kB") .setSystemProperty(SystemSessionProperties.USE_MARK_DISTINCT, "false") - .setSystemProperty(SystemSessionProperties.DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED, "true") + .setSystemProperty(DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED, "true") // Enable Large block spilling .setSystemProperty(DISTINCT_AGGREGATION_LARGE_BLOCK_SPILL_ENABLED, "true") .setSystemProperty(DISTINCT_AGGREGATION_LARGE_BLOCK_SIZE_THRESHOLD, "150B") diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregationWithPreprocessingEnabled.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregationWithPreprocessingEnabled.java index 7574dbeb631a..1f58b463e2fd 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregationWithPreprocessingEnabled.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregationWithPreprocessingEnabled.java @@ -21,6 +21,8 @@ import java.nio.file.Paths; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; @@ -35,10 +37,10 @@ protected QueryRunner createQueryRunner() .setCatalog("tpch") .setSchema(TINY_SCHEMA_NAME) .setSystemProperty(SystemSessionProperties.TASK_CONCURRENCY, "2") - .setSystemProperty(SystemSessionProperties.AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT, "128kB") + .setSystemProperty(AGGREGATION_OPERATOR_UNSPILL_MEMORY_LIMIT, "128kB") .setSystemProperty(SystemSessionProperties.USE_MARK_DISTINCT, "false") // Enable preprocessing - .setSystemProperty(SystemSessionProperties.DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED, "true") + .setSystemProperty(DEDUP_BASED_DISTINCT_AGGREGATION_SPILL_ENABLED, "true") .build(); ImmutableMap extraProperties = ImmutableMap.builder() diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregations.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregations.java index 2a12ade58fca..379cbe012c90 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregations.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledAggregations.java @@ -17,10 +17,10 @@ import com.facebook.presto.testing.QueryRunner; import org.testng.annotations.Test; -import static com.facebook.presto.SystemSessionProperties.AGGREGATION_SPILL_ENABLED; -import static com.facebook.presto.SystemSessionProperties.DISTINCT_AGGREGATION_SPILL_ENABLED; -import static com.facebook.presto.SystemSessionProperties.ORDER_BY_AGGREGATION_SPILL_ENABLED; import static com.facebook.presto.SystemSessionProperties.QUERY_MAX_REVOCABLE_MEMORY_PER_NODE; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.AGGREGATION_SPILL_ENABLED; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.DISTINCT_AGGREGATION_SPILL_ENABLED; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.ORDER_BY_AGGREGATION_SPILL_ENABLED; public class TestSpilledAggregations extends AbstractTestAggregations diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledOrderByQueries.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledOrderByQueries.java index 5c05a027fe63..19080a1f8689 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledOrderByQueries.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledOrderByQueries.java @@ -18,8 +18,8 @@ import com.facebook.presto.testing.QueryRunner; import org.testng.annotations.Test; -import static com.facebook.presto.SystemSessionProperties.ORDER_BY_SPILL_ENABLED; import static com.facebook.presto.SystemSessionProperties.QUERY_MAX_REVOCABLE_MEMORY_PER_NODE; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.ORDER_BY_SPILL_ENABLED; public class TestSpilledOrderByQueries extends AbstractTestOrderByQueries diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledTopNQueries.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledTopNQueries.java index 79b04b50d0e6..739c36e9b7db 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledTopNQueries.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledTopNQueries.java @@ -23,7 +23,8 @@ import java.nio.file.Paths; import static com.facebook.presto.SystemSessionProperties.QUERY_MAX_TOTAL_MEMORY_PER_NODE; -import static com.facebook.presto.SystemSessionProperties.TOPN_SPILL_ENABLED; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.TOPN_OPERATOR_UNSPILL_MEMORY_LIMIT; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.TOPN_SPILL_ENABLED; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; @@ -39,7 +40,7 @@ protected QueryRunner createQueryRunner() .setCatalog("tpch") .setSchema(TINY_SCHEMA_NAME) .setSystemProperty(SystemSessionProperties.TASK_CONCURRENCY, "2") - .setSystemProperty(SystemSessionProperties.TOPN_OPERATOR_UNSPILL_MEMORY_LIMIT, "120kB") + .setSystemProperty(TOPN_OPERATOR_UNSPILL_MEMORY_LIMIT, "120kB") .setSystemProperty(SystemSessionProperties.QUERY_MAX_MEMORY_PER_NODE, "1500kB") .build(); diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledWindowQueries.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledWindowQueries.java index 9ec23774445f..fa273c388a4c 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledWindowQueries.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestSpilledWindowQueries.java @@ -18,7 +18,7 @@ import org.testng.annotations.Test; import static com.facebook.presto.SystemSessionProperties.QUERY_MAX_REVOCABLE_MEMORY_PER_NODE; -import static com.facebook.presto.SystemSessionProperties.WINDOW_SPILL_ENABLED; +import static com.facebook.presto.sessionpropertyproviders.JavaWorkerSessionPropertyProvider.WINDOW_SPILL_ENABLED; public class TestSpilledWindowQueries extends AbstractTestWindowQueries diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestVerboseOptimizerInfo.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestVerboseOptimizerInfo.java index 19704398865e..2cbcaf93babc 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestVerboseOptimizerInfo.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestVerboseOptimizerInfo.java @@ -102,6 +102,7 @@ public void testApplicableOptimizers() checkOptimizerInfo(explainPayloadJoinQuery, "Triggered", ImmutableList.of("PayloadJoinOptimizer")); } + @Test public void testCostBasedOptimizers() { @@ -151,6 +152,11 @@ public void testVerboseOptimizerResults() checkOptimizerResults(explain, ImmutableList.of("PayloadJoinOptimizer", "RemoveRedundantIdentityProjections"), ImmutableList.of("PruneUnreferencedOutputs")); } + @Override + public void testSetSessionNativeWorkerSessionProperty() + { + } + private void checkOptimizerInfo(String explain, String optimizerType, List optimizers) { checkOptimizerInfo(explain, optimizerType, optimizers, new ArrayList<>()); From 852cd29f836dedc0037d5e0776c5f794763c93c6 Mon Sep 17 00:00:00 2001 From: Pratik Joseph Dabre Date: Tue, 22 Oct 2024 13:30:29 -0700 Subject: [PATCH 3/8] Introduce session properties SPI Co-authored-by: Joe Abraham --- .../presto/spi/session/PropertyMetadata.java | 78 ++++++----- .../spi/session/SessionPropertyMetadata.java | 123 ++++++++++++++++++ .../session/TestSessionPropertyMetadata.java | 89 +++++++++++++ 3 files changed, 260 insertions(+), 30 deletions(-) create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/session/SessionPropertyMetadata.java create mode 100644 presto-spi/src/test/java/com/facebook/presto/spi/session/TestSessionPropertyMetadata.java diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/session/PropertyMetadata.java b/presto-spi/src/main/java/com/facebook/presto/spi/session/PropertyMetadata.java index 296d250c95fd..8f59cb774174 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/session/PropertyMetadata.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/session/PropertyMetadata.java @@ -17,6 +17,7 @@ import io.airlift.units.DataSize; import io.airlift.units.Duration; +import java.util.Objects; import java.util.function.Function; import static com.facebook.presto.common.type.BigintType.BIGINT; @@ -24,18 +25,14 @@ import static com.facebook.presto.common.type.DoubleType.DOUBLE; import static com.facebook.presto.common.type.IntegerType.INTEGER; import static com.facebook.presto.common.type.VarcharType.VARCHAR; -import static java.lang.String.format; -import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; public final class PropertyMetadata { - private final String name; - private final String description; - private final Type sqlType; + private final SessionPropertyMetadata sessionPropertyMetadata; private final Class javaType; + private final Type sqlType; private final T defaultValue; - private final boolean hidden; private final Function decoder; private final Function encoder; @@ -49,28 +46,17 @@ public PropertyMetadata( Function decoder, Function encoder) { - requireNonNull(name, "name is null"); - requireNonNull(description, "description is null"); - requireNonNull(sqlType, "type is null"); - requireNonNull(javaType, "javaType is null"); - requireNonNull(decoder, "decoder is null"); - requireNonNull(encoder, "encoder is null"); - - if (name.isEmpty() || !name.trim().toLowerCase(ENGLISH).equals(name)) { - throw new IllegalArgumentException(format("Invalid property name '%s'", name)); - } - if (description.isEmpty() || !description.trim().equals(description)) { - throw new IllegalArgumentException(format("Invalid property description '%s'", description)); - } - - this.name = name; - this.description = description; - this.javaType = javaType; - this.sqlType = sqlType; + this.sqlType = requireNonNull(sqlType, "sqlType is null"); + this.sessionPropertyMetadata = new SessionPropertyMetadata( + name, + description, + sqlType.getTypeSignature(), + defaultValue == null ? "" : defaultValue.toString(), + hidden); + this.javaType = requireNonNull(javaType, "javaType is null"); this.defaultValue = defaultValue; - this.hidden = hidden; - this.decoder = decoder; - this.encoder = encoder; + this.decoder = requireNonNull(decoder, "decoder is null"); + this.encoder = requireNonNull(encoder, "encoder is null"); } /** @@ -78,7 +64,7 @@ public PropertyMetadata( */ public String getName() { - return name; + return sessionPropertyMetadata.getName(); } /** @@ -86,7 +72,7 @@ public String getName() */ public String getDescription() { - return description; + return sessionPropertyMetadata.getDescription(); } /** @@ -118,7 +104,7 @@ public T getDefaultValue() */ public boolean isHidden() { - return hidden; + return sessionPropertyMetadata.isHidden(); } /** @@ -227,4 +213,36 @@ public static PropertyMetadata durationProperty(String name, String de value -> Duration.valueOf((String) value), Duration::toString); } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (!(o instanceof PropertyMetadata)) { + return false; + } + + PropertyMetadata that = (PropertyMetadata) o; + + boolean isSessionPropertyMetadataEqual = this.sessionPropertyMetadata.equals(that.sessionPropertyMetadata); + + boolean isJavaTypeEqual = this.javaType.equals(that.javaType); + boolean isDefaultValueEqual = (this.defaultValue == null && that.defaultValue == null) + || (this.defaultValue != null && this.defaultValue.equals(that.defaultValue)); + + return isSessionPropertyMetadataEqual && isJavaTypeEqual && isDefaultValueEqual; + } + + @Override + public int hashCode() + { + int result = sessionPropertyMetadata.hashCode(); + + result = 31 * result + Objects.hashCode(javaType); + result = 31 * result + Objects.hashCode(defaultValue); + + return result; + } } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/session/SessionPropertyMetadata.java b/presto-spi/src/main/java/com/facebook/presto/spi/session/SessionPropertyMetadata.java new file mode 100644 index 000000000000..7402a68d12e2 --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/session/SessionPropertyMetadata.java @@ -0,0 +1,123 @@ +/* + * 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.session; + +import com.facebook.presto.common.type.TypeSignature; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +import static java.lang.String.format; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public class SessionPropertyMetadata +{ + private final String name; + + private final String description; + + private final TypeSignature typeSignature; + + private final String defaultValue; + + private final boolean hidden; + + @JsonCreator + public SessionPropertyMetadata( + @JsonProperty("name") String name, + @JsonProperty("description") String description, + @JsonProperty("typeSignature") TypeSignature typeSignature, + @JsonProperty("defaultValue") String defaultValue, + @JsonProperty("hidden") boolean hidden) + { + this.name = requireNonNull(name, "name is null"); + this.description = requireNonNull(description, "description is null"); + this.typeSignature = requireNonNull(typeSignature, "typeSignature is null"); + this.defaultValue = defaultValue; + this.hidden = hidden; + + if (name.isEmpty() || !name.trim().toLowerCase(ENGLISH).equals(name)) { + throw new IllegalArgumentException(format("Invalid property name '%s'", name)); + } + if (description.isEmpty() || !description.trim().equals(description)) { + throw new IllegalArgumentException(format("Invalid property description '%s'", description)); + } + } + + /** + * Name of the property. This must be a valid identifier. + */ + @JsonProperty + public String getName() + { + return name; + } + + /** + * Description for the end user. + */ + @JsonProperty + public String getDescription() + { + return description; + } + + /** + * TypeSignature of the property. + */ + @JsonProperty + public TypeSignature getTypeSignature() + { + return typeSignature; + } + + /** + * Gets the default value for this property. + */ + @JsonProperty + public String getDefaultValue() + { + return defaultValue; + } + + /** + * Is this property hidden from users? + */ + @JsonProperty + public boolean isHidden() + { + return hidden; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (!(o instanceof SessionPropertyMetadata)) { + return false; + } + SessionPropertyMetadata that = (SessionPropertyMetadata) o; + return hidden == that.hidden && Objects.equals(name, that.name) && Objects.equals(description, that.description) && Objects.equals(typeSignature, that.typeSignature) && Objects.equals(defaultValue, that.defaultValue); + } + + @Override + public int hashCode() + { + return Objects.hash(name, description, typeSignature, defaultValue, hidden); + } +} diff --git a/presto-spi/src/test/java/com/facebook/presto/spi/session/TestSessionPropertyMetadata.java b/presto-spi/src/test/java/com/facebook/presto/spi/session/TestSessionPropertyMetadata.java new file mode 100644 index 000000000000..94b71cdc6ba5 --- /dev/null +++ b/presto-spi/src/test/java/com/facebook/presto/spi/session/TestSessionPropertyMetadata.java @@ -0,0 +1,89 @@ +/* + * 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.session; + +import com.facebook.airlift.json.JsonCodec; +import org.testng.annotations.Test; + +import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.common.type.BooleanType.BOOLEAN; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; +import static com.facebook.presto.common.type.IntegerType.INTEGER; +import static com.facebook.presto.common.type.VarcharType.VARCHAR; +import static org.testng.Assert.assertEquals; + +public class TestSessionPropertyMetadata +{ + public static final JsonCodec CODEC = JsonCodec.jsonCodec(SessionPropertyMetadata.class); + + @Test + public void testBool() + { + String jsonString = "{\n \"name\" : \"test\",\n \"description\" : \"test description\",\n \"typeSignature\" : \"boolean\",\n \"defaultValue\" : \"false\",\n \"hidden\" : false\n}"; + SessionPropertyMetadata actual = new SessionPropertyMetadata("test", "test description", BOOLEAN.getTypeSignature(), "false", false); + SessionPropertyMetadata expected = CODEC.fromJson(CODEC.toJson(actual)); + assertEquals(expected, actual); + assertEquals(CODEC.toJson(actual), jsonString); + } + + @Test + public void testInteger() + { + String jsonString = "{\n \"name\" : \"test\",\n \"description\" : \"test description\",\n \"typeSignature\" : \"integer\",\n \"defaultValue\" : \"0\",\n \"hidden\" : false\n}"; + SessionPropertyMetadata actual = new SessionPropertyMetadata("test", "test description", INTEGER.getTypeSignature(), "0", false); + SessionPropertyMetadata expected = CODEC.fromJson(CODEC.toJson(actual)); + assertEquals(expected, actual); + assertEquals(CODEC.toJson(actual), jsonString); + } + + @Test + public void testLong() + { + String jsonString = "{\n \"name\" : \"test\",\n \"description\" : \"test description\",\n \"typeSignature\" : \"bigint\",\n \"defaultValue\" : \"987654321012345678\",\n \"hidden\" : false\n}"; + SessionPropertyMetadata actual = new SessionPropertyMetadata("test", "test description", BIGINT.getTypeSignature(), "987654321012345678", false); + SessionPropertyMetadata expected = CODEC.fromJson(CODEC.toJson(actual)); + assertEquals(expected, actual); + assertEquals(CODEC.toJson(actual), jsonString); + } + + @Test + public void testDouble() + { + String jsonString = "{\n \"name\" : \"test\",\n \"description\" : \"test description\",\n \"typeSignature\" : \"double\",\n \"defaultValue\" : \"0.0\",\n \"hidden\" : false\n}"; + SessionPropertyMetadata actual = new SessionPropertyMetadata("test", "test description", DOUBLE.getTypeSignature(), "0.0", false); + SessionPropertyMetadata expected = CODEC.fromJson(CODEC.toJson(actual)); + assertEquals(expected, actual); + assertEquals(CODEC.toJson(actual), jsonString); + } + + @Test + public void testString() + { + String jsonString = "{\n \"name\" : \"test\",\n \"description\" : \"test description\",\n \"typeSignature\" : \"varchar\",\n \"defaultValue\" : \"defaultValue\",\n \"hidden\" : false\n}"; + SessionPropertyMetadata actual = new SessionPropertyMetadata("test", "test description", VARCHAR.getTypeSignature(), "defaultValue", false); + SessionPropertyMetadata expected = CODEC.fromJson(CODEC.toJson(actual)); + assertEquals(expected, actual); + assertEquals(CODEC.toJson(actual), jsonString); + } + + @Test + public void testDataSize() + { + String jsonString = "{\n \"name\" : \"test\",\n \"description\" : \"test description\",\n \"typeSignature\" : \"varchar\",\n \"defaultValue\" : \"1MB\",\n \"hidden\" : false\n}"; + SessionPropertyMetadata actual = new SessionPropertyMetadata("test", "test description", VARCHAR.getTypeSignature(), "1MB", false); + SessionPropertyMetadata expected = CODEC.fromJson(CODEC.toJson(actual)); + assertEquals(expected, actual); + assertEquals(CODEC.toJson(actual), jsonString); + } +} From 64e57ca2571cf029d8bcbafd43b846e4288dbcd2 Mon Sep 17 00:00:00 2001 From: Pratik Joseph Dabre Date: Tue, 22 Oct 2024 13:43:29 -0700 Subject: [PATCH 4/8] Integrate session property SPI with Presto engine Co-authored-by: Abe Varghese Kodiyan Co-authored-by: Joe Abraham Co-authored-by: Deepthy Davis --- pom.xml | 7 ++ .../metadata/SessionPropertyManager.java | 26 +++- .../facebook/presto/server/PluginManager.java | 6 + .../server/testing/TestingPrestoServer.java | 10 ++ .../facebook/presto/testing/QueryRunner.java | 5 + presto-native-sidecar-plugin/pom.xml | 107 ++++++++++++++++ .../presto/sidecar/NativeSidecarPlugin.java | 29 +++++ .../facebook/presto/sidecar/NoOpPlugin.java | 24 ++++ .../NativeSystemSessionPropertyProvider.java | 110 ++++++++++++++++ ...eSystemSessionPropertyProviderFactory.java | 45 +++++++ ...veSystemSessionPropertyProviderModule.java | 53 ++++++++ .../com.facebook.presto.spi.CoordinatorPlugin | 1 + .../sidecar/TestNativeSidecarPlugin.java | 39 ++++++ .../sidecar/UnimplementedNodeManager.java | 117 ++++++++++++++++++ .../sidecar/UnimplementedTypeManager.java | 43 +++++++ .../presto/spi/CoordinatorPlugin.java | 6 + .../presto/spi/session/PropertyMetadata.java | 14 +++ .../presto/tests/DistributedQueryRunner.java | 11 ++ 18 files changed, 650 insertions(+), 3 deletions(-) create mode 100644 presto-native-sidecar-plugin/pom.xml create mode 100644 presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/NativeSidecarPlugin.java create mode 100644 presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/NoOpPlugin.java create mode 100644 presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProvider.java create mode 100644 presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderFactory.java create mode 100644 presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderModule.java create mode 100644 presto-native-sidecar-plugin/src/main/resources/META-INF/services/com.facebook.presto.spi.CoordinatorPlugin create mode 100644 presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSidecarPlugin.java create mode 100644 presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedNodeManager.java create mode 100644 presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedTypeManager.java diff --git a/pom.xml b/pom.xml index fd8bb1baddba..e00a654e529f 100644 --- a/pom.xml +++ b/pom.xml @@ -200,6 +200,7 @@ presto-singlestore presto-hana presto-openapi + presto-native-sidecar-plugin @@ -450,6 +451,12 @@ ${project.version} + + com.facebook.presto + presto-session-property-providers + ${project.version} + + com.facebook.presto presto-bigquery diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/SessionPropertyManager.java b/presto-main/src/main/java/com/facebook/presto/metadata/SessionPropertyManager.java index 4372102c211d..3e9e23e1a6e9 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/SessionPropertyManager.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/SessionPropertyManager.java @@ -24,6 +24,7 @@ import com.facebook.presto.common.type.DoubleType; import com.facebook.presto.common.type.IntegerType; import com.facebook.presto.common.type.MapType; +import com.facebook.presto.common.type.TinyintType; import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.VarcharType; @@ -137,11 +138,11 @@ public static SessionPropertyManager createTestingSessionPropertyManager( Optional.empty()); } - public void loadSessionPropertyProvider(String sessionPropertyProviderName) + public void loadSessionPropertyProvider(String sessionPropertyProviderName, Optional typeManager, Optional nodeManager) { WorkerSessionPropertyProviderFactory factory = workerSessionPropertyProviderFactories.get(sessionPropertyProviderName); checkState(factory != null, "No factory for session property provider : " + sessionPropertyProviderName); - WorkerSessionPropertyProvider sessionPropertyProvider = factory.create(new SessionPropertyContext(functionAndTypeManager, nodeManager)); + WorkerSessionPropertyProvider sessionPropertyProvider = factory.create(new SessionPropertyContext(typeManager, nodeManager)); if (workerSessionPropertyProviders.putIfAbsent(sessionPropertyProviderName, sessionPropertyProvider) != null) { throw new IllegalArgumentException("System session property provider is already registered for property provider : " + sessionPropertyProviderName); } @@ -150,7 +151,14 @@ public void loadSessionPropertyProvider(String sessionPropertyProviderName) public void loadSessionPropertyProviders() { for (String sessionPropertyProviderName : workerSessionPropertyProviderFactories.keySet()) { - loadSessionPropertyProvider(sessionPropertyProviderName); + loadSessionPropertyProvider(sessionPropertyProviderName, functionAndTypeManager, nodeManager); + } + } + + public void addSessionPropertyProviderFactory(WorkerSessionPropertyProviderFactory factory) + { + if (workerSessionPropertyProviderFactories.putIfAbsent(factory.getName(), factory) != null) { + throw new IllegalArgumentException(format("System Session property provider factory" + factory.getName() + "is already registered")); } } @@ -367,6 +375,9 @@ public static String serializeSessionProperty(Type type, Object value) if (VarcharType.VARCHAR.equals(type)) { return value.toString(); } + if (TinyintType.TINYINT.equals(type)) { + return value.toString(); + } if (type instanceof ArrayType || type instanceof MapType) { return getJsonCodecForType(type).toJson(value); } @@ -393,6 +404,9 @@ private static Object deserializeSessionProperty(Type type, String value) if (DoubleType.DOUBLE.equals(type)) { return Double.valueOf(value); } + if (TinyintType.TINYINT.equals(type)) { + return Byte.valueOf(value); + } if (type instanceof ArrayType || type instanceof MapType) { return getJsonCodecForType(type).fromJson(value); } @@ -416,6 +430,9 @@ private static JsonCodec getJsonCodecForType(Type type) if (DoubleType.DOUBLE.equals(type)) { return (JsonCodec) JSON_CODEC_FACTORY.jsonCodec(Double.class); } + if (TinyintType.TINYINT.equals(type)) { + return (JsonCodec) JSON_CODEC_FACTORY.jsonCodec(Byte.class); + } if (type instanceof ArrayType) { Type elementType = ((ArrayType) type).getElementType(); return (JsonCodec) JSON_CODEC_FACTORY.listJsonCodec(getJsonCodecForType(elementType)); @@ -445,6 +462,9 @@ private static Class getMapKeyType(Type type) if (DoubleType.DOUBLE.equals(type)) { return Double.class; } + if (TinyintType.TINYINT.equals(type)) { + return Byte.class; + } throw new PrestoException(INVALID_SESSION_PROPERTY, format("Session property map key type %s is not supported", type)); } 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 7983e017d024..e71d1ee56199 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 @@ -42,6 +42,7 @@ import com.facebook.presto.spi.security.PasswordAuthenticatorFactory; import com.facebook.presto.spi.security.SystemAccessControlFactory; import com.facebook.presto.spi.session.SessionPropertyConfigurationManagerFactory; +import com.facebook.presto.spi.session.WorkerSessionPropertyProviderFactory; import com.facebook.presto.spi.statistics.HistoryBasedPlanStatisticsProvider; import com.facebook.presto.spi.storage.TempStorageFactory; import com.facebook.presto.spi.tracing.TracerProvider; @@ -366,6 +367,11 @@ public void installCoordinatorPlugin(CoordinatorPlugin plugin) log.info("Registering function namespace manager %s", functionNamespaceManagerFactory.getName()); metadata.getFunctionAndTypeManager().addFunctionNamespaceFactory(functionNamespaceManagerFactory); } + + for (WorkerSessionPropertyProviderFactory providerFactory : plugin.getWorkerSessionPropertyProviderFactories()) { + log.info("Registering system session property provider factory %s", providerFactory.getName()); + metadata.getSessionPropertyManager().addSessionPropertyProviderFactory(providerFactory); + } } private URLClassLoader buildClassLoader(String plugin) diff --git a/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java b/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java index 86f72c2df636..034763c47212 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java +++ b/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java @@ -51,6 +51,7 @@ import com.facebook.presto.metadata.InternalNode; import com.facebook.presto.metadata.InternalNodeManager; import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.nodeManager.PluginNodeManager; import com.facebook.presto.resourcemanager.ResourceManagerClusterStateProvider; import com.facebook.presto.security.AccessControlManager; import com.facebook.presto.server.GracefulShutdownHandler; @@ -61,6 +62,7 @@ import com.facebook.presto.server.security.ServerSecurityModule; import com.facebook.presto.spi.ConnectorId; import com.facebook.presto.spi.CoordinatorPlugin; +import com.facebook.presto.spi.NodeManager; import com.facebook.presto.spi.Plugin; import com.facebook.presto.spi.QueryId; import com.facebook.presto.spi.eventlistener.EventListener; @@ -175,6 +177,7 @@ public class TestingPrestoServer private final ServerInfoResource serverInfoResource; private final ResourceManagerClusterStateProvider clusterStateProvider; private final PlanCheckerProviderManager planCheckerProviderManager; + private final NodeManager pluginNodeManager; public static class TestShutdownAction implements ShutdownAction @@ -440,6 +443,7 @@ else if (catalogServer) { announcer = injector.getInstance(Announcer.class); requestBlocker = injector.getInstance(RequestBlocker.class); serverInfoResource = injector.getInstance(ServerInfoResource.class); + pluginNodeManager = injector.getInstance(PluginNodeManager.class); // Announce Thrift server address DriftServer driftServer = injector.getInstance(DriftServer.class); @@ -510,6 +514,7 @@ public PluginManager getPluginManager() { return pluginManager; } + public void installPlugin(Plugin plugin) { pluginManager.installPlugin(plugin); @@ -647,6 +652,11 @@ public InternalNodeManager getNodeManager() return nodeManager; } + public NodeManager getPluginNodeManager() + { + return pluginNodeManager; + } + public NodePartitioningManager getNodePartitioningManager() { return nodePartitioningManager; diff --git a/presto-main/src/main/java/com/facebook/presto/testing/QueryRunner.java b/presto-main/src/main/java/com/facebook/presto/testing/QueryRunner.java index b19fa40da024..d2121ce335f8 100644 --- a/presto-main/src/main/java/com/facebook/presto/testing/QueryRunner.java +++ b/presto-main/src/main/java/com/facebook/presto/testing/QueryRunner.java @@ -97,6 +97,11 @@ default void installCoordinatorPlugin(CoordinatorPlugin plugin) void loadFunctionNamespaceManager(String functionNamespaceManagerName, String catalogName, Map properties); + default void loadSessionPropertyProvider(String sessionPropertyProviderName) + { + throw new UnsupportedOperationException(); + } + Lock getExclusiveLock(); class MaterializedResultWithPlan diff --git a/presto-native-sidecar-plugin/pom.xml b/presto-native-sidecar-plugin/pom.xml new file mode 100644 index 000000000000..a788b6c384d7 --- /dev/null +++ b/presto-native-sidecar-plugin/pom.xml @@ -0,0 +1,107 @@ + + + 4.0.0 + + + com.facebook.presto + presto-root + 0.290-SNAPSHOT + + + presto-native-sidecar-plugin + Presto - Native Sidecar Plugin + presto-plugin + + + ${project.parent.basedir} + + + + + com.facebook.airlift + bootstrap + + + + com.facebook.airlift + log + + + + com.facebook.airlift + json + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + + com.facebook.presto + presto-spi + provided + + + + com.facebook.presto + presto-common + provided + + + + io.airlift + units + provided + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + com.facebook.drift + drift-api + provided + + + + io.airlift + slice + provided + + + + org.openjdk.jol + jol-core + provided + + + + + com.facebook.presto + presto-testng-services + test + + + + org.testng + testng + test + + + + com.facebook.airlift + testing + test + + + diff --git a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/NativeSidecarPlugin.java b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/NativeSidecarPlugin.java new file mode 100644 index 000000000000..4c11cfa9b34d --- /dev/null +++ b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/NativeSidecarPlugin.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.sidecar; + +import com.facebook.presto.sidecar.sessionpropertyproviders.NativeSystemSessionPropertyProviderFactory; +import com.facebook.presto.spi.CoordinatorPlugin; +import com.facebook.presto.spi.session.WorkerSessionPropertyProviderFactory; +import com.google.common.collect.ImmutableList; + +public class NativeSidecarPlugin + implements CoordinatorPlugin +{ + @Override + public Iterable getWorkerSessionPropertyProviderFactories() + { + return ImmutableList.of(new NativeSystemSessionPropertyProviderFactory()); + } +} diff --git a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/NoOpPlugin.java b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/NoOpPlugin.java new file mode 100644 index 000000000000..d42cfd99e66e --- /dev/null +++ b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/NoOpPlugin.java @@ -0,0 +1,24 @@ +/* + * 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.sidecar; + +import com.facebook.presto.spi.Plugin; + +/** + * Todo: Remove this class when support for CoordinatorPlugin is added in presto-maven-plugin. + */ +public class NoOpPlugin + implements Plugin +{ +} diff --git a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProvider.java b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProvider.java new file mode 100644 index 000000000000..e1e889210b34 --- /dev/null +++ b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProvider.java @@ -0,0 +1,110 @@ +/* + * 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.sidecar.sessionpropertyproviders; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.airlift.log.Logger; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.common.type.TypeManager; +import com.facebook.presto.spi.NodeManager; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.session.PropertyMetadata; +import com.facebook.presto.spi.session.SessionPropertyMetadata; +import com.facebook.presto.spi.session.WorkerSessionPropertyProvider; +import com.google.inject.Inject; + +import java.util.List; + +import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.common.type.BooleanType.BOOLEAN; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; +import static com.facebook.presto.common.type.IntegerType.INTEGER; +import static com.facebook.presto.common.type.TinyintType.TINYINT; +import static com.facebook.presto.common.type.VarcharType.VARCHAR; +import static com.facebook.presto.spi.StandardErrorCode.INVALID_ARGUMENTS; +import static com.facebook.presto.spi.StandardErrorCode.INVALID_SESSION_PROPERTY; +import static com.facebook.presto.spi.session.PropertyMetadata.booleanProperty; +import static com.facebook.presto.spi.session.PropertyMetadata.doubleProperty; +import static com.facebook.presto.spi.session.PropertyMetadata.integerProperty; +import static com.facebook.presto.spi.session.PropertyMetadata.longProperty; +import static com.facebook.presto.spi.session.PropertyMetadata.stringProperty; +import static com.facebook.presto.spi.session.PropertyMetadata.tinyIntProperty; +import static java.util.Objects.requireNonNull; + +public class NativeSystemSessionPropertyProvider + implements WorkerSessionPropertyProvider +{ + private static final Logger log = Logger.get(NativeSystemSessionPropertyProvider.class); + private final NodeManager nodeManager; + private final TypeManager typeManager; + private final JsonCodec> nativeSessionPropertiesJsonCodec; + + @Inject + public NativeSystemSessionPropertyProvider( + JsonCodec> nativeSessionPropertiesJsonCodec, + NodeManager nodeManager, + TypeManager typeManager) + { + this.nativeSessionPropertiesJsonCodec = requireNonNull(nativeSessionPropertiesJsonCodec, "nativeSessionPropertiesJsonCodec is null"); + this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + @Override + public List> getSessionProperties() + { + return fetchSessionProperties(); + } + + private List> fetchSessionProperties() + { + try { + throw new UnsupportedOperationException(); + } + catch (Exception e) { + throw new PrestoException(INVALID_ARGUMENTS, "Failed to get session properties from sidecar."); + } + } + + public PropertyMetadata toPropertyMetadata(SessionPropertyMetadata data) + { + requireNonNull(data, "data is null"); + PropertyMetadata propertyMetadata; + Type type = typeManager.getType(data.getTypeSignature()); + if (type == BOOLEAN) { + propertyMetadata = booleanProperty(data.getName(), data.getDescription(), Boolean.valueOf(data.getDefaultValue()), data.isHidden()); + } + else if (type == INTEGER) { + propertyMetadata = integerProperty(data.getName(), data.getDescription(), Integer.valueOf(data.getDefaultValue()), data.isHidden()); + } + else if (type == BIGINT) { + propertyMetadata = longProperty(data.getName(), data.getDescription(), Long.valueOf(data.getDefaultValue()), data.isHidden()); + } + else if (type == VARCHAR) { + propertyMetadata = stringProperty(data.getName(), data.getDescription(), String.valueOf(data.getDefaultValue()), data.isHidden()); + } + else if (type == DOUBLE) { + propertyMetadata = doubleProperty(data.getName(), data.getDescription(), Double.valueOf(data.getDefaultValue()), data.isHidden()); + } + else if (type == TINYINT) { + propertyMetadata = tinyIntProperty(data.getName(), data.getDescription(), Byte.valueOf(data.getDefaultValue()), data.isHidden()); + } + else { + // TODO: Custom types need to be converted + throw new PrestoException(INVALID_SESSION_PROPERTY, "Unknown type"); + } + return propertyMetadata; + } +} diff --git a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderFactory.java b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderFactory.java new file mode 100644 index 000000000000..d537729710af --- /dev/null +++ b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderFactory.java @@ -0,0 +1,45 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.sidecar.sessionpropertyproviders; + +import com.facebook.airlift.bootstrap.Bootstrap; +import com.facebook.presto.spi.session.SessionPropertyContext; +import com.facebook.presto.spi.session.WorkerSessionPropertyProvider; +import com.facebook.presto.spi.session.WorkerSessionPropertyProviderFactory; +import com.google.inject.Injector; + +public class NativeSystemSessionPropertyProviderFactory + implements WorkerSessionPropertyProviderFactory +{ + public static final String NAME = "native"; + + @Override + public String getName() + { + return NAME; + } + + @Override + public WorkerSessionPropertyProvider create(SessionPropertyContext context) + { + Bootstrap app = new Bootstrap( + new NativeSystemSessionPropertyProviderModule( + context.getNodeManager(), context.getTypeManager())); + + Injector injector = app + .doNotInitializeLogging() + .initialize(); + return injector.getInstance(NativeSystemSessionPropertyProvider.class); + } +} diff --git a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderModule.java b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderModule.java new file mode 100644 index 000000000000..201033f0ce0d --- /dev/null +++ b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderModule.java @@ -0,0 +1,53 @@ +/* + * 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.sidecar.sessionpropertyproviders; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.airlift.json.JsonCodecFactory; +import com.facebook.presto.common.type.TypeManager; +import com.facebook.presto.spi.NodeManager; +import com.facebook.presto.spi.session.SessionPropertyMetadata; +import com.facebook.presto.spi.session.WorkerSessionPropertyProvider; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; +import com.google.inject.TypeLiteral; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class NativeSystemSessionPropertyProviderModule + implements Module +{ + private final NodeManager nodeManager; + private final TypeManager typeManager; + + public NativeSystemSessionPropertyProviderModule(NodeManager nodeManager, TypeManager typeManager) + { + this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + @Override + public void configure(Binder binder) + { + binder.bind(new TypeLiteral>>() {}) + .toInstance(new JsonCodecFactory().listJsonCodec(SessionPropertyMetadata.class)); + binder.bind(NodeManager.class).toInstance(nodeManager); + binder.bind(TypeManager.class).toInstance(typeManager); + binder.bind(NativeSystemSessionPropertyProvider.class).in(Scopes.SINGLETON); + binder.bind(WorkerSessionPropertyProvider.class).to(NativeSystemSessionPropertyProvider.class).in(Scopes.SINGLETON); + } +} diff --git a/presto-native-sidecar-plugin/src/main/resources/META-INF/services/com.facebook.presto.spi.CoordinatorPlugin b/presto-native-sidecar-plugin/src/main/resources/META-INF/services/com.facebook.presto.spi.CoordinatorPlugin new file mode 100644 index 000000000000..4f23fd095c6f --- /dev/null +++ b/presto-native-sidecar-plugin/src/main/resources/META-INF/services/com.facebook.presto.spi.CoordinatorPlugin @@ -0,0 +1 @@ +com.facebook.presto.sidecar.NativeSidecarPlugin diff --git a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSidecarPlugin.java b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSidecarPlugin.java new file mode 100644 index 000000000000..131d01d6146d --- /dev/null +++ b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSidecarPlugin.java @@ -0,0 +1,39 @@ +/* + * 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.sidecar; + +import com.facebook.presto.spi.CoordinatorPlugin; +import com.facebook.presto.spi.session.SessionPropertyContext; +import com.facebook.presto.spi.session.WorkerSessionPropertyProviderFactory; +import org.testng.annotations.Test; + +import java.util.Optional; + +import static com.google.common.collect.Iterables.getOnlyElement; + +public class TestNativeSidecarPlugin +{ + private final CoordinatorPlugin plugin = new NativeSidecarPlugin(); + + @Test + public void testLoadNativeSessionPropertyManager() + { + Iterable workerPropertyProviderFactories = plugin.getWorkerSessionPropertyProviderFactories(); + WorkerSessionPropertyProviderFactory factory = getOnlyElement(workerPropertyProviderFactories); + factory.create( + new SessionPropertyContext( + Optional.of(new UnimplementedTypeManager()), + Optional.of(new UnimplementedNodeManager()))); + } +} diff --git a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedNodeManager.java b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedNodeManager.java new file mode 100644 index 000000000000..38f16b9d8c95 --- /dev/null +++ b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedNodeManager.java @@ -0,0 +1,117 @@ +/* + * 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.sidecar; + +import com.facebook.presto.spi.HostAddress; +import com.facebook.presto.spi.Node; +import com.facebook.presto.spi.NodeManager; +import com.facebook.presto.spi.NodePoolType; + +import java.net.URI; +import java.util.Set; + +class UnimplementedNodeManager + implements NodeManager +{ + @Override + public Set getAllNodes() + { + throw new UnsupportedOperationException(); + } + + @Override + public Set getWorkerNodes() + { + throw new UnsupportedOperationException(); + } + + @Override + public Node getCurrentNode() + { + return new Node() + { + @Override + public String getHost() + { + throw new UnsupportedOperationException(); + } + + @Override + public HostAddress getHostAndPort() + { + throw new UnsupportedOperationException(); + } + + @Override + public URI getHttpUri() + { + throw new UnsupportedOperationException(); + } + + @Override + public String getNodeIdentifier() + { + throw new UnsupportedOperationException(); + } + + @Override + public String getVersion() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isCoordinator() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isResourceManager() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isCatalogServer() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isCoordinatorSidecar() + { + throw new UnsupportedOperationException(); + } + + @Override + public NodePoolType getPoolType() + { + throw new UnsupportedOperationException(); + } + }; + } + + @Override + public Node getSidecarNode() + { + throw new UnsupportedOperationException(); + } + + @Override + public String getEnvironment() + { + throw new UnsupportedOperationException(); + } +} diff --git a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedTypeManager.java b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedTypeManager.java new file mode 100644 index 000000000000..61d6e9a6453c --- /dev/null +++ b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedTypeManager.java @@ -0,0 +1,43 @@ +/* + * 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.sidecar; + +import com.facebook.presto.common.type.Type; +import com.facebook.presto.common.type.TypeManager; +import com.facebook.presto.common.type.TypeSignature; +import com.facebook.presto.common.type.TypeSignatureParameter; + +import java.util.List; + +public class UnimplementedTypeManager + implements TypeManager +{ + @Override + public Type getType(TypeSignature signature) + { + throw new UnsupportedOperationException(); + } + + @Override + public Type getParameterizedType(String baseTypeName, List typeParameters) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean canCoerce(Type actualType, Type expectedType) + { + throw new UnsupportedOperationException(); + } +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/CoordinatorPlugin.java b/presto-spi/src/main/java/com/facebook/presto/spi/CoordinatorPlugin.java index f855e6a63e2d..81e57f7e1235 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/CoordinatorPlugin.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/CoordinatorPlugin.java @@ -14,6 +14,7 @@ package com.facebook.presto.spi; import com.facebook.presto.spi.function.FunctionNamespaceManagerFactory; +import com.facebook.presto.spi.session.WorkerSessionPropertyProviderFactory; import static java.util.Collections.emptyList; @@ -30,4 +31,9 @@ default Iterable getFunctionNamespaceManagerFac { return emptyList(); } + + default Iterable getWorkerSessionPropertyProviderFactories() + { + return emptyList(); + } } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/session/PropertyMetadata.java b/presto-spi/src/main/java/com/facebook/presto/spi/session/PropertyMetadata.java index 8f59cb774174..f61a039efa59 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/session/PropertyMetadata.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/session/PropertyMetadata.java @@ -24,6 +24,7 @@ import static com.facebook.presto.common.type.BooleanType.BOOLEAN; import static com.facebook.presto.common.type.DoubleType.DOUBLE; import static com.facebook.presto.common.type.IntegerType.INTEGER; +import static com.facebook.presto.common.type.TinyintType.TINYINT; import static com.facebook.presto.common.type.VarcharType.VARCHAR; import static java.util.Objects.requireNonNull; @@ -162,6 +163,19 @@ public static PropertyMetadata longProperty(String name, String descriptio object -> object); } + public static PropertyMetadata tinyIntProperty(String name, String description, Byte defaultValue, boolean hidden) + { + return new PropertyMetadata<>( + name, + description, + TINYINT, + Byte.class, + defaultValue, + hidden, + value -> ((Number) value).byteValue(), + object -> object); + } + public static PropertyMetadata doubleProperty(String name, String description, Double defaultValue, boolean hidden) { return new PropertyMetadata<>( diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/DistributedQueryRunner.java b/presto-tests/src/main/java/com/facebook/presto/tests/DistributedQueryRunner.java index 5329d380ce9e..5ad1d9885335 100644 --- a/presto-tests/src/main/java/com/facebook/presto/tests/DistributedQueryRunner.java +++ b/presto-tests/src/main/java/com/facebook/presto/tests/DistributedQueryRunner.java @@ -932,6 +932,17 @@ private void installCoordinatorPlugin(CoordinatorPlugin plugin, boolean coordina log.info("Installed plugin %s in %s", plugin.getClass().getSimpleName(), nanosSince(start).convertToMostSuccinctTimeUnit()); } + @Override + public void loadSessionPropertyProvider(String sessionPropertyProviderName) + { + for (TestingPrestoServer server : servers) { + server.getMetadata().getSessionPropertyManager().loadSessionPropertyProvider( + sessionPropertyProviderName, + Optional.ofNullable(server.getMetadata().getFunctionAndTypeManager()), + Optional.ofNullable(server.getPluginNodeManager())); + } + } + private static void closeUnchecked(AutoCloseable closeable) { try { From c81a00fdc1898077876ab7996ec7f363a6eb649e Mon Sep 17 00:00:00 2001 From: Pratik Joseph Dabre Date: Tue, 22 Oct 2024 15:00:08 -0700 Subject: [PATCH 5/8] Integrate session property plugin with sidecar --- presto-native-sidecar-plugin/pom.xml | 10 ++++ .../presto/sidecar/ForSidecarInfo.java | 26 ++++++++++ .../NativeSidecarCommunicationModule.java | 29 ++++++++++++ .../NativeSystemSessionPropertyProvider.java | 45 ++++++++++++++++-- ...veSystemSessionPropertyProviderConfig.java | 38 +++++++++++++++ ...eSystemSessionPropertyProviderFactory.java | 5 +- ...veSystemSessionPropertyProviderModule.java | 1 + ...veSystemSessionPropertyProviderConfig.java | 47 +++++++++++++++++++ 8 files changed, 197 insertions(+), 4 deletions(-) create mode 100644 presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/ForSidecarInfo.java create mode 100644 presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/NativeSidecarCommunicationModule.java create mode 100644 presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderConfig.java create mode 100644 presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSystemSessionPropertyProviderConfig.java diff --git a/presto-native-sidecar-plugin/pom.xml b/presto-native-sidecar-plugin/pom.xml index a788b6c384d7..3b2b02f87f19 100644 --- a/presto-native-sidecar-plugin/pom.xml +++ b/presto-native-sidecar-plugin/pom.xml @@ -42,6 +42,11 @@ guice + + com.facebook.airlift + configuration + + com.facebook.presto @@ -103,5 +108,10 @@ testing test + + + com.facebook.airlift + http-client + diff --git a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/ForSidecarInfo.java b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/ForSidecarInfo.java new file mode 100644 index 000000000000..57a62fc95a5c --- /dev/null +++ b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/ForSidecarInfo.java @@ -0,0 +1,26 @@ +/* + * 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.sidecar; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; + +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@BindingAnnotation +public @interface ForSidecarInfo +{ +} diff --git a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/NativeSidecarCommunicationModule.java b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/NativeSidecarCommunicationModule.java new file mode 100644 index 000000000000..5f4025898e3d --- /dev/null +++ b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/NativeSidecarCommunicationModule.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.sidecar; + +import com.google.inject.Binder; +import com.google.inject.Module; + +import static com.facebook.airlift.http.client.HttpClientBinder.httpClientBinder; + +public class NativeSidecarCommunicationModule + implements Module +{ + @Override + public void configure(Binder binder) + { + httpClientBinder(binder).bindHttpClient("sidecar", ForSidecarInfo.class); + } +} diff --git a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProvider.java b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProvider.java index e1e889210b34..5d51ef08c524 100644 --- a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProvider.java +++ b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProvider.java @@ -14,19 +14,30 @@ package com.facebook.presto.sidecar.sessionpropertyproviders; +import com.facebook.airlift.http.client.HttpClient; +import com.facebook.airlift.http.client.HttpUriBuilder; +import com.facebook.airlift.http.client.Request; import com.facebook.airlift.json.JsonCodec; import com.facebook.airlift.log.Logger; import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; +import com.facebook.presto.sidecar.ForSidecarInfo; +import com.facebook.presto.spi.Node; import com.facebook.presto.spi.NodeManager; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.session.PropertyMetadata; import com.facebook.presto.spi.session.SessionPropertyMetadata; import com.facebook.presto.spi.session.WorkerSessionPropertyProvider; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.inject.Inject; +import java.net.URI; +import java.util.ArrayList; import java.util.List; +import static com.facebook.airlift.http.client.JsonResponseHandler.createJsonResponseHandler; +import static com.facebook.airlift.http.client.Request.Builder.prepareGet; import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.BooleanType.BOOLEAN; import static com.facebook.presto.common.type.DoubleType.DOUBLE; @@ -42,6 +53,7 @@ import static com.facebook.presto.spi.session.PropertyMetadata.stringProperty; import static com.facebook.presto.spi.session.PropertyMetadata.tinyIntProperty; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; public class NativeSystemSessionPropertyProvider implements WorkerSessionPropertyProvider @@ -50,28 +62,44 @@ public class NativeSystemSessionPropertyProvider private final NodeManager nodeManager; private final TypeManager typeManager; private final JsonCodec> nativeSessionPropertiesJsonCodec; + private final Supplier>> memoizedSessionPropertiesSupplier; + private final HttpClient httpClient; + private static final String SESSION_PROPERTIES_ENDPOINT = "/v1/properties/session"; @Inject public NativeSystemSessionPropertyProvider( + @ForSidecarInfo HttpClient httpClient, JsonCodec> nativeSessionPropertiesJsonCodec, NodeManager nodeManager, - TypeManager typeManager) + TypeManager typeManager, + NativeSystemSessionPropertyProviderConfig config) { this.nativeSessionPropertiesJsonCodec = requireNonNull(nativeSessionPropertiesJsonCodec, "nativeSessionPropertiesJsonCodec is null"); this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); this.typeManager = requireNonNull(typeManager, "typeManager is null"); + this.httpClient = requireNonNull(httpClient, "typeManager is null"); + requireNonNull(config, "config is null"); + this.memoizedSessionPropertiesSupplier = + Suppliers.memoizeWithExpiration(this::fetchSessionProperties, config.getSessionPropertiesCacheExpiration().toMillis(), MILLISECONDS); } @Override public List> getSessionProperties() { - return fetchSessionProperties(); + return memoizedSessionPropertiesSupplier.get(); } private List> fetchSessionProperties() { try { - throw new UnsupportedOperationException(); + List> propertyMetadataList = new ArrayList<>(); + Request request = prepareGet().setUri(getSidecarLocation()).build(); + List nativeSessionProperties = httpClient.execute(request, createJsonResponseHandler(nativeSessionPropertiesJsonCodec)); + for (SessionPropertyMetadata sessionProperty : nativeSessionProperties) { + PropertyMetadata propertyMetadata = toPropertyMetadata(sessionProperty); + propertyMetadataList.add(propertyMetadata); + } + return propertyMetadataList; } catch (Exception e) { throw new PrestoException(INVALID_ARGUMENTS, "Failed to get session properties from sidecar."); @@ -107,4 +135,15 @@ else if (type == TINYINT) { } return propertyMetadata; } + + private URI getSidecarLocation() + { + Node sidecarNode = nodeManager.getSidecarNode(); + return HttpUriBuilder.uriBuilder() + .scheme("http") + .host(sidecarNode.getHost()) + .port(sidecarNode.getHostAndPort().getPort()) + .appendPath(SESSION_PROPERTIES_ENDPOINT) + .build(); + } } diff --git a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderConfig.java b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderConfig.java new file mode 100644 index 000000000000..b8a8dd7c9565 --- /dev/null +++ b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderConfig.java @@ -0,0 +1,38 @@ +/* + * 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.sidecar.sessionpropertyproviders; + +import com.facebook.airlift.configuration.Config; +import io.airlift.units.Duration; +import io.airlift.units.MinDuration; + +import static java.util.concurrent.TimeUnit.MINUTES; + +public class NativeSystemSessionPropertyProviderConfig +{ + private Duration sessionPropertiesCacheExpiration = new Duration(5, MINUTES); + + @MinDuration("0ns") + public Duration getSessionPropertiesCacheExpiration() + { + return sessionPropertiesCacheExpiration; + } + + @Config("session-properties-cache-expiration") + public NativeSystemSessionPropertyProviderConfig setSessionPropertiesCacheExpiration(Duration functionCacheExpiration) + { + this.sessionPropertiesCacheExpiration = functionCacheExpiration; + return this; + } +} diff --git a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderFactory.java b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderFactory.java index d537729710af..e8693ad77eaa 100644 --- a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderFactory.java +++ b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderFactory.java @@ -14,6 +14,7 @@ package com.facebook.presto.sidecar.sessionpropertyproviders; import com.facebook.airlift.bootstrap.Bootstrap; +import com.facebook.presto.sidecar.NativeSidecarCommunicationModule; import com.facebook.presto.spi.session.SessionPropertyContext; import com.facebook.presto.spi.session.WorkerSessionPropertyProvider; import com.facebook.presto.spi.session.WorkerSessionPropertyProviderFactory; @@ -35,10 +36,12 @@ public WorkerSessionPropertyProvider create(SessionPropertyContext context) { Bootstrap app = new Bootstrap( new NativeSystemSessionPropertyProviderModule( - context.getNodeManager(), context.getTypeManager())); + context.getNodeManager(), context.getTypeManager()), + new NativeSidecarCommunicationModule()); Injector injector = app .doNotInitializeLogging() + .noStrictConfig() .initialize(); return injector.getInstance(NativeSystemSessionPropertyProvider.class); } diff --git a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderModule.java b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderModule.java index 201033f0ce0d..c86bd98a0d47 100644 --- a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderModule.java +++ b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/sessionpropertyproviders/NativeSystemSessionPropertyProviderModule.java @@ -47,6 +47,7 @@ public void configure(Binder binder) .toInstance(new JsonCodecFactory().listJsonCodec(SessionPropertyMetadata.class)); binder.bind(NodeManager.class).toInstance(nodeManager); binder.bind(TypeManager.class).toInstance(typeManager); + binder.bind(NativeSystemSessionPropertyProviderConfig.class).in(Scopes.SINGLETON); binder.bind(NativeSystemSessionPropertyProvider.class).in(Scopes.SINGLETON); binder.bind(WorkerSessionPropertyProvider.class).to(NativeSystemSessionPropertyProvider.class).in(Scopes.SINGLETON); } diff --git a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSystemSessionPropertyProviderConfig.java b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSystemSessionPropertyProviderConfig.java new file mode 100644 index 000000000000..678492804447 --- /dev/null +++ b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSystemSessionPropertyProviderConfig.java @@ -0,0 +1,47 @@ +/* + * 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.sidecar; + +import com.facebook.presto.sidecar.sessionpropertyproviders.NativeSystemSessionPropertyProviderConfig; +import com.google.common.collect.ImmutableMap; +import io.airlift.units.Duration; +import org.testng.annotations.Test; + +import java.util.Map; + +import static com.facebook.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static com.facebook.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static com.facebook.airlift.configuration.testing.ConfigAssertions.recordDefaults; +import static java.util.concurrent.TimeUnit.MINUTES; + +public class TestNativeSystemSessionPropertyProviderConfig +{ + @Test + public void testDefault() + { + assertRecordedDefaults(recordDefaults(NativeSystemSessionPropertyProviderConfig.class) + .setSessionPropertiesCacheExpiration(new Duration(5, MINUTES))); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("session-properties-cache-expiration", "10m") + .build(); + NativeSystemSessionPropertyProviderConfig expected = new NativeSystemSessionPropertyProviderConfig() + .setSessionPropertiesCacheExpiration(new Duration(10, MINUTES)); + assertFullMapping(properties, expected); + } +} From 5622db7e16955f6ed888bf99b0e73a9d2d5b7fce Mon Sep 17 00:00:00 2001 From: Pratik Joseph Dabre Date: Tue, 22 Oct 2024 15:01:31 -0700 Subject: [PATCH 6/8] [native] Add e2e native session property provider tests with sidecar Co-authored-by: Tim Meehan --- .circleci/continue_config.yml | 49 ++++++++ .github/workflows/test-other-modules.yml | 3 +- .../HiveExternalWorkerQueryRunner.java | 2 +- .../nativeworker/NativeQueryRunnerUtils.java | 7 ++ .../PrestoNativeQueryRunnerUtils.java | 39 +++--- presto-native-sidecar-plugin/pom.xml | 98 +++++++++++++++ .../NativeSidecarPluginQueryRunner.java | 48 +++++++ .../NativeSidecarPluginQueryRunnerUtils.java | 27 ++++ .../sidecar/TestNativeSidecarPlugin.java | 39 ------ ...veSidecarPluginSystemPropertyProvider.java | 97 +++++++++++++++ .../sidecar/UnimplementedNodeManager.java | 117 ------------------ .../sidecar/UnimplementedTypeManager.java | 43 ------- 12 files changed, 354 insertions(+), 215 deletions(-) create mode 100644 presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/NativeSidecarPluginQueryRunner.java create mode 100644 presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/NativeSidecarPluginQueryRunnerUtils.java delete mode 100644 presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSidecarPlugin.java create mode 100644 presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSidecarPluginSystemPropertyProvider.java delete mode 100644 presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedNodeManager.java delete mode 100644 presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedTypeManager.java diff --git a/.circleci/continue_config.yml b/.circleci/continue_config.yml index bddbafb0ef43..193be83bb7f6 100644 --- a/.circleci/continue_config.yml +++ b/.circleci/continue_config.yml @@ -33,6 +33,10 @@ workflows: run_linux_tests: << pipeline.parameters.run_linux_tests >> requires: - linux-build-and-unit-test + - linux-presto-native-sidecar-tests: + run_linux_tests: << pipeline.parameters.run_linux_tests >> + requires: + - linux-build-and-unit-test conditionals: when: << pipeline.parameters.run_native_specific_jobs >> @@ -193,6 +197,51 @@ jobs: - store_artifacts: path: '/tmp/PrestoNativeQueryRunnerUtils' + linux-presto-native-sidecar-tests: + executor: build + parameters: + run_linux_tests: + type: boolean + default: false + parallelism: 5 + steps: + - run: echo "Run Linux tests is << parameters.run_linux_tests >>" + - when: + condition: << parameters.run_linux_tests >> + steps: + - checkout + - attach_workspace: + at: presto-native-execution + - maven_install: + maven_install_opts: ${MAVEN_INSTALL_OPTS} + maven_fast_install: ${MAVEN_FAST_INSTALL} + - run: + name: 'Run Presto native sidecar tests' + command: | + export LD_LIBRARY_PATH="${LD_LIBRARY_PATH}:/usr/local/lib:/usr/local/lib64" + export PRESTO_SERVER_PATH="${HOME}/project/presto-native-execution/_build/debug/presto_cpp/main/presto_server" + export TEMP_PATH="/tmp" + TESTFILES=$(circleci tests glob "presto-native-sidecar-plugin/src/test/**/Test*.java" | circleci tests split --split-by=timings) + # Convert file paths to comma separated class names + export TESTCLASSES= + for test_file in $TESTFILES + do + tmp=${test_file##*/} + test_class=${tmp%%\.*} + export TESTCLASSES="${TESTCLASSES},$test_class" + done + export TESTCLASSES=${TESTCLASSES#,} + if [ ! -z $TESTCLASSES ]; then + mvn test \ + ${MAVEN_TEST} \ + -pl 'presto-native-sidecar-plugin' \ + -Dtest="${TESTCLASSES}" \ + -DPRESTO_SERVER=${PRESTO_SERVER_PATH} \ + -DDATA_DIR=${TEMP_PATH} \ + -Duser.timezone=America/Bahia_Banderas \ + -T1C + fi + linux-spark-e2e-tests: executor: build parameters: diff --git a/.github/workflows/test-other-modules.yml b/.github/workflows/test-other-modules.yml index ccfccf6a79c0..3c3f84817a0d 100644 --- a/.github/workflows/test-other-modules.yml +++ b/.github/workflows/test-other-modules.yml @@ -83,4 +83,5 @@ jobs: !presto-native-execution, !presto-test-coverage, !presto-iceberg, - !presto-singlestore' + !presto-singlestore, + !presto-native-sidecar-plugin' diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/HiveExternalWorkerQueryRunner.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/HiveExternalWorkerQueryRunner.java index cd3bfa11353a..6d9392a26d22 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/HiveExternalWorkerQueryRunner.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/HiveExternalWorkerQueryRunner.java @@ -34,7 +34,7 @@ public static void main(String[] args) javaQueryRunner.close(); // Launch distributed runner. - DistributedQueryRunner queryRunner = (DistributedQueryRunner) PrestoNativeQueryRunnerUtils.createQueryRunner(false); + DistributedQueryRunner queryRunner = (DistributedQueryRunner) PrestoNativeQueryRunnerUtils.createQueryRunner(false, false); Thread.sleep(10); Logger log = Logger.get(DistributedQueryRunner.class); log.info("======== SERVER STARTED ========"); diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/NativeQueryRunnerUtils.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/NativeQueryRunnerUtils.java index 64d9d9851b83..1d2711170183 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/NativeQueryRunnerUtils.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/NativeQueryRunnerUtils.java @@ -56,6 +56,13 @@ public static Map getNativeWorkerSystemProperties() .build(); } + public static Map getNativeSidecarProperties() + { + return ImmutableMap.builder() + .put("coordinator-sidecar-enabled", "true") + .build(); + } + /** * Creates all tables for local testing, except for bench tables. * 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 e21d87f0182a..4835fc7226fa 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 @@ -37,6 +37,7 @@ import static com.facebook.presto.hive.HiveTestUtils.getProperty; import static com.facebook.presto.iceberg.IcebergQueryRunner.createIcebergQueryRunner; +import static com.facebook.presto.nativeworker.NativeQueryRunnerUtils.getNativeSidecarProperties; import static com.facebook.presto.nativeworker.NativeQueryRunnerUtils.getNativeWorkerHiveProperties; import static com.facebook.presto.nativeworker.NativeQueryRunnerUtils.getNativeWorkerIcebergProperties; import static com.facebook.presto.nativeworker.NativeQueryRunnerUtils.getNativeWorkerSystemProperties; @@ -57,7 +58,7 @@ public class PrestoNativeQueryRunnerUtils private static final String DEFAULT_STORAGE_FORMAT = "DWRF"; private PrestoNativeQueryRunnerUtils() {} - public static QueryRunner createQueryRunner(boolean addStorageFormatToPath) + public static QueryRunner createQueryRunner(boolean addStorageFormatToPath, boolean isCoordinatorSidecarEnabled) throws Exception { int cacheMaxSize = 4096; // 4GB size cache @@ -68,7 +69,8 @@ public static QueryRunner createQueryRunner(boolean addStorageFormatToPath) nativeQueryRunnerParameters.workerCount, cacheMaxSize, DEFAULT_STORAGE_FORMAT, - addStorageFormatToPath); + addStorageFormatToPath, + isCoordinatorSidecarEnabled); } public static QueryRunner createQueryRunner( @@ -77,7 +79,8 @@ public static QueryRunner createQueryRunner( Optional workerCount, int cacheMaxSize, String storageFormat, - boolean addStorageFormatToPath) + boolean addStorageFormatToPath, + boolean isCoordinatorSidecarEnabled) throws Exception { QueryRunner defaultQueryRunner = createJavaQueryRunner(dataDirectory, storageFormat, addStorageFormatToPath); @@ -88,7 +91,7 @@ public static QueryRunner createQueryRunner( defaultQueryRunner.close(); - return createNativeQueryRunner(dataDirectory.get().toString(), prestoServerPath.get(), workerCount, cacheMaxSize, true, Optional.empty(), storageFormat, addStorageFormatToPath, false); + return createNativeQueryRunner(dataDirectory.get().toString(), prestoServerPath.get(), workerCount, cacheMaxSize, true, Optional.empty(), storageFormat, addStorageFormatToPath, false, isCoordinatorSidecarEnabled); } public static QueryRunner createJavaQueryRunner() @@ -251,7 +254,7 @@ public static QueryRunner createNativeIcebergQueryRunner( false, false, OptionalInt.of(workerCount.orElse(4)), - getExternalWorkerLauncher("iceberg", prestoServerPath, cacheMaxSize, remoteFunctionServerUds, false), + getExternalWorkerLauncher("iceberg", prestoServerPath, cacheMaxSize, remoteFunctionServerUds, false, false), dataDirectory, addStorageFormatToPath); } @@ -265,7 +268,8 @@ public static QueryRunner createNativeQueryRunner( Optional remoteFunctionServerUds, String storageFormat, boolean addStorageFormatToPath, - Boolean failOnNestedLoopJoin) + Boolean failOnNestedLoopJoin, + boolean isCoordinatorSidecarEnabled) throws Exception { // The property "hive.allow-drop-table" needs to be set to true because security is always "legacy" in NativeQueryRunner. @@ -282,13 +286,14 @@ public static QueryRunner createNativeQueryRunner( .put("http-server.http.port", "8081") .put("experimental.internal-communication.thrift-transport-enabled", String.valueOf(useThrift)) .putAll(getNativeWorkerSystemProperties()) + .putAll(isCoordinatorSidecarEnabled ? getNativeSidecarProperties() : ImmutableMap.of()) .build(), ImmutableMap.of(), "legacy", hiveProperties, workerCount, Optional.of(Paths.get(addStorageFormatToPath ? dataDirectory + "/" + storageFormat : dataDirectory)), - getExternalWorkerLauncher("hive", prestoServerPath, cacheMaxSize, remoteFunctionServerUds, failOnNestedLoopJoin)); + getExternalWorkerLauncher("hive", prestoServerPath, cacheMaxSize, remoteFunctionServerUds, failOnNestedLoopJoin, isCoordinatorSidecarEnabled)); } public static QueryRunner createNativeCteQueryRunner(boolean useThrift, String storageFormat) @@ -331,13 +336,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(), false)); + getExternalWorkerLauncher("hive", prestoServerPath, cacheMaxSize, Optional.empty(), false, false)); } public static QueryRunner createNativeQueryRunner(String remoteFunctionServerUds) throws Exception { - return createNativeQueryRunner(false, DEFAULT_STORAGE_FORMAT, Optional.ofNullable(remoteFunctionServerUds), false); + return createNativeQueryRunner(false, DEFAULT_STORAGE_FORMAT, Optional.ofNullable(remoteFunctionServerUds), false, false); } public static QueryRunner createNativeQueryRunner(boolean useThrift) @@ -349,16 +354,16 @@ public static QueryRunner createNativeQueryRunner(boolean useThrift) public static QueryRunner createNativeQueryRunner(boolean useThrift, boolean failOnNestedLoopJoin) throws Exception { - return createNativeQueryRunner(useThrift, DEFAULT_STORAGE_FORMAT, Optional.empty(), failOnNestedLoopJoin); + return createNativeQueryRunner(useThrift, DEFAULT_STORAGE_FORMAT, Optional.empty(), failOnNestedLoopJoin, false); } public static QueryRunner createNativeQueryRunner(boolean useThrift, String storageFormat) throws Exception { - return createNativeQueryRunner(useThrift, storageFormat, Optional.empty(), false); + return createNativeQueryRunner(useThrift, storageFormat, Optional.empty(), false, false); } - public static QueryRunner createNativeQueryRunner(boolean useThrift, String storageFormat, Optional remoteFunctionServerUds, Boolean failOnNestedLoopJoin) + public static QueryRunner createNativeQueryRunner(boolean useThrift, String storageFormat, Optional remoteFunctionServerUds, Boolean failOnNestedLoopJoin, boolean isCoordinatorSidecarEnabled) throws Exception { int cacheMaxSize = 0; @@ -372,7 +377,8 @@ public static QueryRunner createNativeQueryRunner(boolean useThrift, String stor remoteFunctionServerUds, storageFormat, true, - failOnNestedLoopJoin); + failOnNestedLoopJoin, + isCoordinatorSidecarEnabled); } // Start the remote function server. Return the UDS path used to communicate with it. @@ -419,7 +425,7 @@ public static NativeQueryRunnerParameters getNativeQueryRunnerParameters() return new NativeQueryRunnerParameters(prestoServerPath, dataDirectory, workerCount); } - public static Optional> getExternalWorkerLauncher(String catalogName, String prestoServerPath, int cacheMaxSize, Optional remoteFunctionServerUds, Boolean failOnNestedLoopJoin) + public static Optional> getExternalWorkerLauncher(String catalogName, String prestoServerPath, int cacheMaxSize, Optional remoteFunctionServerUds, Boolean failOnNestedLoopJoin, boolean isCoordinatorSidecarEnabled) { return Optional.of((workerIndex, discoveryUri) -> { @@ -436,6 +442,11 @@ public static Optional> getExternalWorkerLaunc "system-memory-gb=4%n" + "http-server.http.port=%d", discoveryUri, port); + if (isCoordinatorSidecarEnabled) { + configProperties = format("%s%n" + + "native-sidecar=true%n", configProperties); + } + if (remoteFunctionServerUds.isPresent()) { String jsonSignaturesPath = Resources.getResource(REMOTE_FUNCTION_JSON_SIGNATURES).getFile(); configProperties = format("%s%n" + diff --git a/presto-native-sidecar-plugin/pom.xml b/presto-native-sidecar-plugin/pom.xml index 3b2b02f87f19..ec80a8d91987 100644 --- a/presto-native-sidecar-plugin/pom.xml +++ b/presto-native-sidecar-plugin/pom.xml @@ -47,6 +47,11 @@ configuration + + com.facebook.airlift + log-manager + + com.facebook.presto @@ -113,5 +118,98 @@ com.facebook.airlift http-client + + + com.facebook.presto + presto-tests + test + test-jar + 0.290-SNAPSHOT + + + + com.facebook.presto + presto-tests + test + + + + com.facebook.presto + presto-native-execution + test + test-jar + 0.290-SNAPSHOT + + + + com.facebook.presto + presto-main + test + test-jar + + + + com.facebook.presto + presto-main + test + + + + com.facebook.presto + presto-tpcds + test + + + + com.facebook.airlift + jaxrs + test + + + + com.facebook.presto + presto-client + test + + + + com.facebook.airlift + jaxrs-testing + test + + + org.jetbrains + annotations + test + + + + + + + org.basepom.maven + duplicate-finder-maven-plugin + + + parquet.thrift + about.html + mozilla/public-suffix-list.txt + iceberg-build.properties + org.apache.avro.data/Json.avsc + + + com.esotericsoftware.kryo.* + com.esotericsoftware.minlog.Log + com.esotericsoftware.reflectasm.* + module-info + META-INF.versions.9.module-info + org.apache.avro.* + com.github.benmanes.caffeine.* + org.roaringbitmap.* + + + + + diff --git a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/NativeSidecarPluginQueryRunner.java b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/NativeSidecarPluginQueryRunner.java new file mode 100644 index 000000000000..45142a0a217b --- /dev/null +++ b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/NativeSidecarPluginQueryRunner.java @@ -0,0 +1,48 @@ +/* + * 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.sidecar; + +import com.facebook.airlift.log.Logger; +import com.facebook.airlift.log.Logging; +import com.facebook.presto.nativeworker.NativeQueryRunnerUtils; +import com.facebook.presto.nativeworker.PrestoNativeQueryRunnerUtils; +import com.facebook.presto.testing.QueryRunner; +import com.facebook.presto.tests.DistributedQueryRunner; + +import static com.facebook.presto.sidecar.NativeSidecarPluginQueryRunnerUtils.setupNativeSidecarPlugin; + +public class NativeSidecarPluginQueryRunner +{ + private NativeSidecarPluginQueryRunner() {} + + public static void main(String[] args) + throws Exception + { + // You need to add "--user user" to your CLI for your queries to work. + Logging.initialize(); + + // Create tables before launching distributed runner. + QueryRunner javaQueryRunner = PrestoNativeQueryRunnerUtils.createJavaQueryRunner(false); + NativeQueryRunnerUtils.createAllTables(javaQueryRunner); + javaQueryRunner.close(); + + // Launch distributed runner. + DistributedQueryRunner queryRunner = (DistributedQueryRunner) PrestoNativeQueryRunnerUtils.createQueryRunner(false, true); + setupNativeSidecarPlugin(queryRunner); + Thread.sleep(10); + Logger log = Logger.get(DistributedQueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } +} diff --git a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/NativeSidecarPluginQueryRunnerUtils.java b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/NativeSidecarPluginQueryRunnerUtils.java new file mode 100644 index 000000000000..4ac96f6f837f --- /dev/null +++ b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/NativeSidecarPluginQueryRunnerUtils.java @@ -0,0 +1,27 @@ +/* + * 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.sidecar; +import com.facebook.presto.sidecar.sessionpropertyproviders.NativeSystemSessionPropertyProviderFactory; +import com.facebook.presto.testing.QueryRunner; + +public class NativeSidecarPluginQueryRunnerUtils +{ + private NativeSidecarPluginQueryRunnerUtils() {}; + + public static void setupNativeSidecarPlugin(QueryRunner queryRunner) + { + queryRunner.installCoordinatorPlugin(new NativeSidecarPlugin()); + queryRunner.loadSessionPropertyProvider(NativeSystemSessionPropertyProviderFactory.NAME); + } +} diff --git a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSidecarPlugin.java b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSidecarPlugin.java deleted file mode 100644 index 131d01d6146d..000000000000 --- a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSidecarPlugin.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.sidecar; - -import com.facebook.presto.spi.CoordinatorPlugin; -import com.facebook.presto.spi.session.SessionPropertyContext; -import com.facebook.presto.spi.session.WorkerSessionPropertyProviderFactory; -import org.testng.annotations.Test; - -import java.util.Optional; - -import static com.google.common.collect.Iterables.getOnlyElement; - -public class TestNativeSidecarPlugin -{ - private final CoordinatorPlugin plugin = new NativeSidecarPlugin(); - - @Test - public void testLoadNativeSessionPropertyManager() - { - Iterable workerPropertyProviderFactories = plugin.getWorkerSessionPropertyProviderFactories(); - WorkerSessionPropertyProviderFactory factory = getOnlyElement(workerPropertyProviderFactories); - factory.create( - new SessionPropertyContext( - Optional.of(new UnimplementedTypeManager()), - Optional.of(new UnimplementedNodeManager()))); - } -} diff --git a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSidecarPluginSystemPropertyProvider.java b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSidecarPluginSystemPropertyProvider.java new file mode 100644 index 000000000000..2e3f6bbb7e7a --- /dev/null +++ b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/TestNativeSidecarPluginSystemPropertyProvider.java @@ -0,0 +1,97 @@ +/* + * 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.sidecar; + +import com.facebook.presto.nativeworker.PrestoNativeQueryRunnerUtils; +import com.facebook.presto.testing.MaterializedResult; +import com.facebook.presto.testing.MaterializedRow; +import com.facebook.presto.testing.QueryRunner; +import com.facebook.presto.tests.AbstractTestQueryFramework; +import com.facebook.presto.tests.DistributedQueryRunner; +import org.intellij.lang.annotations.Language; +import org.testng.annotations.Test; + +import java.util.List; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static com.facebook.presto.sidecar.NativeSidecarPluginQueryRunnerUtils.setupNativeSidecarPlugin; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; + +@Test(singleThreaded = true) +public class TestNativeSidecarPluginSystemPropertyProvider + extends AbstractTestQueryFramework +{ + private static final String REGEX_SESSION_NAMESPACE = "Native Execution only.*"; + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + DistributedQueryRunner queryRunner = (DistributedQueryRunner) PrestoNativeQueryRunnerUtils.createQueryRunner(false, true); + setupNativeSidecarPlugin(queryRunner); + return queryRunner; + } + + @Override + protected QueryRunner createExpectedQueryRunner() + throws Exception + { + return PrestoNativeQueryRunnerUtils.createJavaQueryRunner(); + } + + private List excludeSystemSessionProperties(List inputRows) + { + return inputRows.stream() + .filter(row -> Pattern.matches(REGEX_SESSION_NAMESPACE, row.getFields().get(4).toString())) + .collect(Collectors.toList()); + } + + @Test + public void testShowSession() + { + @Language("SQL") String sql = "SHOW SESSION"; + MaterializedResult actualResult = computeActual(sql); + List actualRows = actualResult.getMaterializedRows(); + List filteredRows = excludeSystemSessionProperties(actualRows); + assertFalse(filteredRows.isEmpty()); + } + + @Test + public void testJavaWorkerProperties() + { + @Language("SQL") String setSession = "SET SESSION aggregation_spill_enabled=false"; + MaterializedResult setSessionResult = computeActual(setSession); + assertEquals( + setSessionResult.toString(), + "MaterializedResult{rows=[[true]], " + + "types=[boolean], " + + "setSessionProperties={aggregation_spill_enabled=false}, " + + "resetSessionProperties=[], updateType=SET SESSION}"); + } + + @Test + public void testSetNativeSessionProperty() + { + @Language("SQL") String setSession = "SET SESSION driver_cpu_time_slice_limit_ms=500"; + MaterializedResult setSessionResult = computeActual(setSession); + assertEquals( + setSessionResult.toString(), + "MaterializedResult{rows=[[true]], " + + "types=[boolean], " + + "setSessionProperties={driver_cpu_time_slice_limit_ms=500}, " + + "resetSessionProperties=[], updateType=SET SESSION}"); + } +} diff --git a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedNodeManager.java b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedNodeManager.java deleted file mode 100644 index 38f16b9d8c95..000000000000 --- a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedNodeManager.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * 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.sidecar; - -import com.facebook.presto.spi.HostAddress; -import com.facebook.presto.spi.Node; -import com.facebook.presto.spi.NodeManager; -import com.facebook.presto.spi.NodePoolType; - -import java.net.URI; -import java.util.Set; - -class UnimplementedNodeManager - implements NodeManager -{ - @Override - public Set getAllNodes() - { - throw new UnsupportedOperationException(); - } - - @Override - public Set getWorkerNodes() - { - throw new UnsupportedOperationException(); - } - - @Override - public Node getCurrentNode() - { - return new Node() - { - @Override - public String getHost() - { - throw new UnsupportedOperationException(); - } - - @Override - public HostAddress getHostAndPort() - { - throw new UnsupportedOperationException(); - } - - @Override - public URI getHttpUri() - { - throw new UnsupportedOperationException(); - } - - @Override - public String getNodeIdentifier() - { - throw new UnsupportedOperationException(); - } - - @Override - public String getVersion() - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isCoordinator() - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isResourceManager() - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isCatalogServer() - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isCoordinatorSidecar() - { - throw new UnsupportedOperationException(); - } - - @Override - public NodePoolType getPoolType() - { - throw new UnsupportedOperationException(); - } - }; - } - - @Override - public Node getSidecarNode() - { - throw new UnsupportedOperationException(); - } - - @Override - public String getEnvironment() - { - throw new UnsupportedOperationException(); - } -} diff --git a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedTypeManager.java b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedTypeManager.java deleted file mode 100644 index 61d6e9a6453c..000000000000 --- a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/UnimplementedTypeManager.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.sidecar; - -import com.facebook.presto.common.type.Type; -import com.facebook.presto.common.type.TypeManager; -import com.facebook.presto.common.type.TypeSignature; -import com.facebook.presto.common.type.TypeSignatureParameter; - -import java.util.List; - -public class UnimplementedTypeManager - implements TypeManager -{ - @Override - public Type getType(TypeSignature signature) - { - throw new UnsupportedOperationException(); - } - - @Override - public Type getParameterizedType(String baseTypeName, List typeParameters) - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean canCoerce(Type actualType, Type expectedType) - { - throw new UnsupportedOperationException(); - } -} From a3ea29952c4026e2c3203edc299a1a3aec9f0bc4 Mon Sep 17 00:00:00 2001 From: Pratik Joseph Dabre Date: Tue, 22 Oct 2024 15:01:49 -0700 Subject: [PATCH 7/8] Add OpenAPI documentation for /v1/properties/session --- .../src/main/resources/schemas.yaml | 20 ++++++++++++++++++- .../main/resources/session_properties.yaml | 20 +++++++++++++++++++ 2 files changed, 39 insertions(+), 1 deletion(-) create mode 100644 presto-openapi/src/main/resources/session_properties.yaml diff --git a/presto-openapi/src/main/resources/schemas.yaml b/presto-openapi/src/main/resources/schemas.yaml index 50e6423494f5..b9f3c312bc4f 100644 --- a/presto-openapi/src/main/resources/schemas.yaml +++ b/presto-openapi/src/main/resources/schemas.yaml @@ -999,7 +999,25 @@ components: QueryInfo: type: object # TODO generate all the classes for this object - + SessionPropertyMetadata: + type: object + required: + - name + - description + - typeSignature + - defaultValue + - hidden + properties: + name: + type: string + description: + type: string + typeSignature: + $ref: '#/components/schemas/TypeSignature' + defaultValue: + type: string + hidden: + type: boolean diff --git a/presto-openapi/src/main/resources/session_properties.yaml b/presto-openapi/src/main/resources/session_properties.yaml new file mode 100644 index 000000000000..fc6621633869 --- /dev/null +++ b/presto-openapi/src/main/resources/session_properties.yaml @@ -0,0 +1,20 @@ +openapi: 3.0.0 +info: + title: Presto session properties API + description: API for retrieving session properties in Presto. + version: "1" +servers: + - url: http://localhost:8080 + description: Presto endpoint when running locally +paths: + /v1/properties/session: + get: + summary: Returns the list of session properties. + description: This endpoint retrieves the supported session properties list from a Prestissimo cluster. + responses: + '200': + description: List of session properties. + content: + application/json: + schema: + $ref: './schemas.yaml/#/components/schemas/SessionPropertyMetadata' From 8d8c805822294dab96122db9f80460f03c322be3 Mon Sep 17 00:00:00 2001 From: Pratik Joseph Dabre Date: Tue, 22 Oct 2024 18:18:18 -0700 Subject: [PATCH 8/8] Fixed presto-spark-base errors --- .../com/facebook/presto/spark/TestPrestoQueries.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/presto-spark-base/src/test/java/com/facebook/presto/spark/TestPrestoQueries.java b/presto-spark-base/src/test/java/com/facebook/presto/spark/TestPrestoQueries.java index fa31deb91cda..4da201f77a9d 100644 --- a/presto-spark-base/src/test/java/com/facebook/presto/spark/TestPrestoQueries.java +++ b/presto-spark-base/src/test/java/com/facebook/presto/spark/TestPrestoQueries.java @@ -181,4 +181,16 @@ public void testExplainDdl() { // DDL statements are not supported by Presto on Spark } + + @Override + public void testShowSessionWithoutNativeSessionProperties() + { + // prepared statement is not supported by Presto on Spark + } + + @Override + public void testSetSessionNativeWorkerSessionProperty() + { + // prepared statement is not supported by Presto on Spark + } }