From 63af9357ae7e1aaf951d7da01833a2427ab262a7 Mon Sep 17 00:00:00 2001 From: Rongrong Zhong Date: Tue, 29 Sep 2020 12:51:44 -0700 Subject: [PATCH] Change FunctionManager to FunctionAndTypeManager --- .../presto/accumulo/model/TestField.java | 4 +- .../AbstractTestAccumuloRowSerializer.java | 4 +- .../optimization/TestJdbcComputePushdown.java | 10 +- .../benchmark/AbstractOperatorBenchmark.java | 2 +- .../benchmark/CountAggregationBenchmark.java | 8 +- .../DoubleSumAggregationBenchmark.java | 8 +- .../presto/benchmark/HandTpchQuery1.java | 20 +-- .../presto/benchmark/HandTpchQuery6.java | 8 +- .../benchmark/HashAggregationBenchmark.java | 8 +- .../benchmark/PredicateFilterBenchmark.java | 2 +- .../presto/druid/TestDruidQueryBase.java | 4 +- .../geospatial/TestBingTileFunctions.java | 8 +- ...patialPartitioningInternalAggregation.java | 8 +- .../AbstractTestGeoAggregationFunctions.java | 8 +- .../presto/hive/AbstractTestHiveClient.java | 6 +- .../facebook/presto/hive/HiveTestUtils.java | 8 +- .../presto/hive/TestDomainTranslator.java | 10 +- .../hive/TestHiveIntegrationSmokeTest.java | 2 +- .../presto/hive/TestHiveLogicalPlanner.java | 8 +- .../presto/hive/TestSubfieldExtractor.java | 10 +- .../parquet/TestParquetPageSourceFactory.java | 2 +- .../presto/connector/ConnectorManager.java | 6 +- .../presto/cost/FilterStatsCalculator.java | 32 ++-- .../presto/cost/ScalarStatsCalculator.java | 10 +- .../SimpleFilterProjectSemiJoinStatsRule.java | 10 +- .../presto/cost/StatsCalculatorModule.java | 2 +- .../com/facebook/presto/cost/StatsUtil.java | 10 +- .../facebook/presto/event/QueryMonitor.java | 8 +- .../presto/execution/AlterFunctionTask.java | 4 +- .../presto/execution/CreateFunctionTask.java | 4 +- .../presto/execution/DropFunctionTask.java | 4 +- .../presto/execution/SqlQueryExecution.java | 4 +- .../scheduler/LegacySqlQueryScheduler.java | 14 +- .../scheduler/SqlQueryScheduler.java | 14 +- .../BuiltInFunctionNamespaceManager.java | 10 +- ...nager.java => FunctionAndTypeManager.java} | 154 ++++++++++++------ .../metadata/FunctionInvokerProvider.java | 8 +- .../facebook/presto/metadata/Metadata.java | 2 +- .../presto/metadata/MetadataManager.java | 18 +- .../metadata/PolymorphicScalarFunction.java | 8 +- .../PolymorphicScalarFunctionBuilder.java | 10 +- .../metadata/SqlAggregationFunction.java | 2 +- .../presto/metadata/SqlScalarFunction.java | 2 +- .../StaticFunctionNamespaceStore.java | 8 +- .../operator/ExplainAnalyzeOperator.java | 20 +-- .../facebook/presto/operator/PagesIndex.java | 20 +-- .../operator/ParametricFunctionHelpers.java | 6 +- .../operator/RemoteProjectOperator.java | 20 +-- .../operator/SimplePagesHashStrategy.java | 8 +- .../AbstractMinMaxAggregationFunction.java | 8 +- .../AbstractMinMaxNAggregationFunction.java | 4 +- .../aggregation/AggregationUtils.java | 10 +- .../ArbitraryAggregationFunction.java | 4 +- .../ChecksumAggregationFunction.java | 4 +- .../operator/aggregation/CountColumn.java | 4 +- .../DecimalAverageAggregation.java | 4 +- .../aggregation/DecimalSumAggregation.java | 4 +- .../aggregation/MapAggregationFunction.java | 4 +- .../aggregation/MapUnionAggregation.java | 4 +- .../MergeStatisticalDigestFunction.java | 4 +- .../aggregation/ParametricAggregation.java | 20 +-- .../aggregation/RealAverageAggregation.java | 4 +- .../ReduceAggregationFunction.java | 4 +- .../StatisticalDigestAggregationFunction.java | 4 +- .../arrayagg/ArrayAggregationFunction.java | 4 +- .../arrayagg/SetAggregationFunction.java | 4 +- .../arrayagg/SetUnionFunction.java | 4 +- .../aggregation/histogram/Histogram.java | 4 +- .../minmaxby/AbstractMinMaxBy.java | 10 +- .../AbstractMinMaxByNAggregationFunction.java | 4 +- .../MultimapAggregationFunction.java | 4 +- .../FunctionImplementationDependency.java | 6 +- .../annotations/ImplementationDependency.java | 4 +- .../LiteralImplementationDependency.java | 4 +- .../OperatorImplementationDependency.java | 6 +- .../ScalarImplementationDependency.java | 12 +- .../TypeImplementationDependency.java | 4 +- .../index/FieldSetFilteringRecordSet.java | 10 +- .../scalar/AbstractGreatestLeast.java | 8 +- .../presto/operator/scalar/ApplyFunction.java | 4 +- .../operator/scalar/ArrayConcatFunction.java | 4 +- .../operator/scalar/ArrayConstructor.java | 4 +- .../operator/scalar/ArrayFlattenFunction.java | 4 +- .../presto/operator/scalar/ArrayJoin.java | 14 +- .../operator/scalar/ArrayReduceFunction.java | 4 +- .../scalar/ArraySubscriptOperator.java | 4 +- .../operator/scalar/ArrayToArrayCast.java | 10 +- .../scalar/ArrayToElementConcatFunction.java | 4 +- .../operator/scalar/ArrayToJsonCast.java | 4 +- .../scalar/ArrayTransformFunction.java | 4 +- .../scalar/CastFromUnknownOperator.java | 4 +- .../operator/scalar/ConcatFunction.java | 4 +- .../scalar/ElementToArrayConcatFunction.java | 4 +- .../presto/operator/scalar/IdentityCast.java | 4 +- .../operator/scalar/InvokeFunction.java | 4 +- .../scalar/JsonStringToArrayCast.java | 6 +- .../operator/scalar/JsonStringToMapCast.java | 6 +- .../operator/scalar/JsonStringToRowCast.java | 6 +- .../operator/scalar/JsonToArrayCast.java | 4 +- .../presto/operator/scalar/JsonToMapCast.java | 4 +- .../presto/operator/scalar/JsonToRowCast.java | 4 +- .../operator/scalar/MapConcatFunction.java | 4 +- .../operator/scalar/MapConstructor.java | 12 +- .../operator/scalar/MapElementAtFunction.java | 8 +- .../operator/scalar/MapFilterFunction.java | 4 +- .../operator/scalar/MapHashCodeOperator.java | 8 +- .../operator/scalar/MapSubscriptOperator.java | 16 +- .../presto/operator/scalar/MapToJsonCast.java | 4 +- .../presto/operator/scalar/MapToMapCast.java | 12 +- .../scalar/MapTransformKeyFunction.java | 4 +- .../scalar/MapTransformValueFunction.java | 4 +- .../operator/scalar/MapZipWithFunction.java | 8 +- .../operator/scalar/ParametricScalar.java | 10 +- .../scalar/Re2JCastToRegexpFunction.java | 4 +- .../scalar/RowComparisonOperator.java | 8 +- .../scalar/RowDistinctFromOperator.java | 8 +- .../operator/scalar/RowEqualOperator.java | 16 +- .../scalar/RowGreaterThanOperator.java | 6 +- .../scalar/RowGreaterThanOrEqualOperator.java | 6 +- .../operator/scalar/RowHashCodeOperator.java | 4 +- .../scalar/RowIndeterminateOperator.java | 14 +- .../operator/scalar/RowLessThanOperator.java | 6 +- .../scalar/RowLessThanOrEqualOperator.java | 6 +- .../operator/scalar/RowNotEqualOperator.java | 6 +- .../presto/operator/scalar/RowToJsonCast.java | 4 +- .../presto/operator/scalar/RowToRowCast.java | 12 +- .../operator/scalar/TryCastFunction.java | 8 +- .../presto/operator/scalar/ZipFunction.java | 4 +- .../operator/scalar/ZipWithFunction.java | 4 +- .../ParametricScalarImplementation.java | 8 +- .../operator/window/SqlWindowFunction.java | 4 +- .../facebook/presto/server/PluginManager.java | 2 +- .../presto/server/ServerMainModule.java | 4 +- .../sql/InterpretedFunctionInvoker.java | 10 +- .../sql/analyzer/AggregationAnalyzer.java | 6 +- .../presto/sql/analyzer/Analyzer.java | 10 +- .../sql/analyzer/ExpressionAnalyzer.java | 50 +++--- .../sql/analyzer/ExpressionTreeUtils.java | 18 +- .../presto/sql/analyzer/QueryExplainer.java | 10 +- .../sql/analyzer/StatementAnalyzer.java | 20 +-- .../sql/analyzer/WindowFunctionValidator.java | 10 +- .../sql/gen/BytecodeGeneratorContext.java | 8 +- .../sql/gen/FunctionCallCodeGenerator.java | 8 +- .../presto/sql/gen/InCodeGenerator.java | 10 +- .../facebook/presto/sql/gen/JoinCompiler.java | 8 +- .../sql/gen/LambdaBytecodeGenerator.java | 2 +- .../presto/sql/gen/NullIfCodeGenerator.java | 8 +- .../presto/sql/gen/PageFunctionCompiler.java | 2 +- .../presto/sql/gen/RowExpressionCompiler.java | 18 +- .../planner/EffectivePredicateExtractor.java | 24 +-- .../presto/sql/planner/EqualityInference.java | 40 ++--- .../planner/ExpressionDomainTranslator.java | 6 +- .../sql/planner/ExpressionInterpreter.java | 24 +-- .../sql/planner/LiteralInterpreter.java | 6 +- .../sql/planner/LocalExecutionPlanner.java | 32 ++-- .../presto/sql/planner/LogicalPlanner.java | 2 +- .../sql/planner/NullabilityAnalyzer.java | 20 +-- .../presto/sql/planner/PlanFragmenter.java | 6 +- .../presto/sql/planner/PlanOptimizers.java | 42 ++--- .../sql/planner/RowExpressionInterpreter.java | 48 +++--- .../sql/planner/SortExpressionExtractor.java | 16 +- .../planner/StatisticsAggregationPlanner.java | 10 +- .../iterative/rule/ExtractSpatialJoins.java | 38 ++--- .../iterative/rule/InlineProjections.java | 6 +- .../iterative/rule/InlineSqlFunctions.java | 8 +- .../iterative/rule/PickTableLayout.java | 6 +- .../iterative/rule/PlanRemotePojections.java | 20 +-- .../rule/PruneCountAggregationOverScalar.java | 8 +- .../rule/PruneOrderByInAggregation.java | 10 +- .../rule/PushAggregationThroughOuterJoin.java | 10 +- ...PushPartialAggregationThroughExchange.java | 14 +- .../rule/RemoveUnsupportedDynamicFilters.java | 12 +- .../planner/iterative/rule/ReorderJoins.java | 6 +- ...teFilterWithExternalFunctionToProject.java | 10 +- ...RewriteSpatialPartitioningAggregation.java | 6 +- .../rule/SimplifyCountOverConstant.java | 8 +- .../rule/SimplifyRowExpressions.java | 12 +- .../TransformCorrelatedInPredicateToJoin.java | 8 +- ...formCorrelatedScalarAggregationToJoin.java | 10 +- .../TransformExistsApplyToLateralNode.java | 8 +- .../iterative/rule/TranslateExpressions.java | 6 +- .../planner/optimizations/AddExchanges.java | 2 +- .../optimizations/AddLocalExchanges.java | 6 +- .../optimizations/AggregationNodeUtils.java | 6 +- .../optimizations/ExpressionEquivalence.java | 16 +- .../ExternalCallExpressionChecker.java | 10 +- .../HashGenerationOptimizer.java | 64 ++++---- .../ImplementIntersectAndExceptAsUnion.java | 16 +- .../optimizations/IndexJoinOptimizer.java | 8 +- .../optimizations/MetadataQueryOptimizer.java | 8 +- .../OptimizeMixedDistinctAggregations.java | 6 +- .../optimizations/PredicatePushDown.java | 70 ++++---- .../optimizations/PushdownSubfields.java | 4 +- .../ScalarAggregationToJoinRewriter.java | 8 +- ...uantifiedComparisonApplyToLateralJoin.java | 8 +- .../UnaliasSymbolReferences.java | 14 +- .../optimizations/WindowFilterPushDown.java | 28 ++-- .../presto/sql/planner/plan/JoinNode.java | 6 +- .../planner/plan/StatisticAggregations.java | 16 +- .../sql/planner/planPrinter/PlanPrinter.java | 84 +++++----- .../planPrinter/RowExpressionFormatter.java | 8 +- .../CheckUnsupportedExternalFunctions.java | 8 +- .../sql/planner/sanity/TypeValidator.java | 6 +- .../sanity/VerifyProjectionLocality.java | 8 +- .../presto/sql/relational/Expressions.java | 10 +- .../sql/relational/FunctionResolution.java | 66 ++++---- .../RowExpressionDeterminismEvaluator.java | 20 +-- .../RowExpressionDomainTranslator.java | 36 ++-- .../sql/relational/SqlFunctionUtils.java | 2 +- .../SqlToRowExpressionTranslator.java | 38 ++--- .../sql/rewrite/ShowQueriesRewrite.java | 4 +- .../presto/testing/LocalQueryRunner.java | 6 +- .../testing/TestingConnectorContext.java | 12 +- .../presto/testing/TestingEnvironment.java | 6 +- .../presto/type/MapParametricType.java | 10 +- .../facebook/presto/type/TypeRegistry.java | 12 +- .../presto/util/FastutilSetHelper.java | 28 ++-- .../facebook/presto/util/GraphvizPrinter.java | 24 +-- .../presto/util/SpatialJoinUtils.java | 34 ++-- .../presto/block/AbstractTestBlock.java | 4 +- .../presto/block/ColumnarTestUtils.java | 4 +- .../block/TestRowBasedSerialization.java | 6 +- .../presto/cost/TestCostCalculator.java | 2 +- .../presto/cost/TestStatsNormalizer.java | 6 +- .../presto/cost/TestValuesNodeStats.java | 4 +- .../presto/metadata/AbstractMockMetadata.java | 2 +- ...r.java => TestFunctionAndTypeManager.java} | 66 ++++---- .../TestPolymorphicScalarFunction.java | 2 +- .../presto/metadata/TestSignatureBinder.java | 2 +- ...kHashAndStreamingAggregationOperators.java | 12 +- .../presto/operator/GenericLongFunction.java | 4 +- .../operator/TestAggregationOperator.java | 6 +- .../TestAnnotationEngineForAggregates.java | 18 +- .../TestFilterAndProjectOperator.java | 10 +- .../operator/TestHashAggregationOperator.java | 10 +- .../presto/operator/TestPositionLinks.java | 2 +- .../operator/TestRealAverageAggregation.java | 8 +- .../TestScanFilterAndProjectOperator.java | 16 +- .../TestStreamingAggregationOperator.java | 12 +- .../operator/TestTableFinishOperator.java | 8 +- .../operator/TestTableWriterOperator.java | 8 +- .../AbstractTestAggregationFunction.java | 16 +- .../AbstractTestApproximateCountDistinct.java | 4 +- .../BenchmarkArrayAggregation.java | 8 +- .../BenchmarkGroupedTypedHistogram.java | 8 +- .../TestApproximateCountDistinctBoolean.java | 4 +- .../TestApproximateCountDistinctDouble.java | 4 +- .../TestApproximateCountDistinctInteger.java | 4 +- ...TestApproximateCountDistinctIpAddress.java | 4 +- .../TestApproximateCountDistinctLong.java | 4 +- ...stApproximateCountDistinctLongDecimal.java | 4 +- .../TestApproximateCountDistinctSmallint.java | 4 +- .../TestApproximateCountDistinctTinyint.java | 4 +- ...TestApproximateCountDistinctVarBinary.java | 4 +- .../TestApproximatePercentileAggregation.java | 6 +- .../aggregation/TestArbitraryAggregation.java | 6 +- .../aggregation/TestArrayAggregation.java | 6 +- .../aggregation/TestChecksumAggregation.java | 6 +- .../aggregation/TestCountNullAggregation.java | 2 +- .../TestDoubleHistogramAggregation.java | 8 +- .../aggregation/TestEntropyAggregation.java | 8 +- .../operator/aggregation/TestHistogram.java | 12 +- .../aggregation/TestMapAggAggregation.java | 6 +- .../aggregation/TestMapUnionAggregation.java | 32 ++-- .../TestMultimapAggAggregation.java | 6 +- .../TestPrecisionRecallAggregation.java | 8 +- ...TestQuantileDigestAggregationFunction.java | 8 +- .../TestRealHistogramAggregation.java | 8 +- .../TestTDigestAggregationFunction.java | 8 +- .../TestIllegalMethodAggregation.java | 14 +- .../minmaxby/TestMinMaxByAggregation.java | 8 +- .../minmaxby/TestMinMaxByNAggregation.java | 12 +- .../index/TestFieldSetFilteringRecordSet.java | 6 +- .../operator/project/TestPageProcessor.java | 2 +- .../scalar/AbstractTestFunctions.java | 6 +- .../scalar/BenchmarkArrayDistinct.java | 6 +- .../operator/scalar/BenchmarkArrayFilter.java | 10 +- .../BenchmarkArrayHashCodeOperator.java | 6 +- .../scalar/BenchmarkArrayIntersect.java | 6 +- .../operator/scalar/BenchmarkArrayJoin.java | 6 +- .../operator/scalar/BenchmarkArraySort.java | 2 +- .../scalar/BenchmarkArraySubscript.java | 2 +- .../scalar/BenchmarkArrayTransform.java | 8 +- .../scalar/BenchmarkEqualsOperator.java | 18 +- .../scalar/BenchmarkJsonToArrayCast.java | 6 +- .../scalar/BenchmarkJsonToMapCast.java | 6 +- .../operator/scalar/BenchmarkMapConcat.java | 2 +- .../scalar/BenchmarkMapSubscript.java | 2 +- .../scalar/BenchmarkMapToMapCast.java | 2 +- .../scalar/BenchmarkRowToRowCast.java | 2 +- .../scalar/BenchmarkTransformKey.java | 8 +- .../scalar/BenchmarkTransformValue.java | 8 +- .../operator/scalar/FunctionAssertions.java | 2 +- .../scalar/TestPageProcessorCompiler.java | 24 +-- ...idedBlockBuilderReturnPlaceConvention.java | 6 +- .../presto/sql/TestExpressionInterpreter.java | 4 +- .../presto/sql/TestRowExpressionSerde.java | 8 +- .../sql/TestingRowExpressionTranslator.java | 6 +- .../presto/sql/analyzer/TestAnalyzer.java | 2 +- .../sql/gen/BenchmarkPageProcessor.java | 12 +- .../sql/gen/CommonSubExpressionBenchmark.java | 2 +- .../sql/gen/PageProcessorBenchmark.java | 2 +- .../gen/TestCommonSubExpressionRewritter.java | 2 +- .../sql/gen/TestCursorProcessorCompiler.java | 4 +- .../presto/sql/gen/TestInCodeGenerator.java | 10 +- .../presto/sql/gen/TestJoinCompiler.java | 4 +- .../sql/gen/TestPageFunctionCompiler.java | 4 +- .../TestRowExpressionPredicateCompiler.java | 2 +- .../TestVarArgsToArrayAdapterGenerator.java | 4 +- .../presto/sql/planner/TestCanonicalize.java | 2 +- .../TestEffectivePredicateExtractor.java | 16 +- .../sql/planner/TestEqualityInference.java | 6 +- .../sql/planner/TestNullabilityAnalyzer.java | 2 +- .../TestRowExpressionDomainTranslator.java | 14 +- .../planner/TestRowExpressionFormatter.java | 28 ++-- .../planner/TestRowExpressionRewriter.java | 22 +-- .../planner/TestSortExpressionExtractor.java | 4 +- .../presto/sql/planner/TestTypeValidator.java | 4 +- .../AggregationFunctionMatcher.java | 8 +- .../sql/planner/assertions/BasePlanTest.java | 2 +- .../assertions/DynamicFilterMatcher.java | 6 +- .../sql/planner/assertions/PlanAssert.java | 2 +- .../assertions/RowExpressionVerifier.java | 8 +- .../assertions/WindowFunctionMatcher.java | 2 +- .../rule/TestInlineSqlFunctions.java | 14 +- .../iterative/rule/TestJoinEnumerator.java | 2 +- .../iterative/rule/TestJoinNodeFlattener.java | 2 +- .../rule/TestMergeAdjacentWindows.java | 6 +- .../rule/TestPlanRemoteProjections.java | 14 +- .../rule/TestPruneOrderByInAggregation.java | 4 +- .../rule/TestPruneWindowColumns.java | 2 +- .../rule/TestPushProjectionThroughUnion.java | 2 +- .../iterative/rule/TestReorderJoins.java | 6 +- ...teFilterWithExternalFunctionToProject.java | 10 +- ...stSwapAdjacentWindowsBySpecifications.java | 2 +- ...formCorrelatedScalarAggregationToJoin.java | 12 +- ...TestTransformExistsApplyToLateralJoin.java | 8 +- .../rule/TestTranslateExpressions.java | 4 +- .../iterative/rule/test/BaseRuleTest.java | 6 +- .../iterative/rule/test/PlanBuilder.java | 12 +- .../iterative/rule/test/RuleAssert.java | 4 +- .../optimizations/TestEliminateSorts.java | 2 +- .../optimizations/TestMergeWindows.java | 2 +- ...TestOptimizeMixedDistinctAggregations.java | 2 +- .../TestPruneUnreferencedOutputs.java | 2 +- .../TestRemoveUnsupportedDynamicFilters.java | 22 +-- .../optimizations/TestReorderWindows.java | 2 +- .../TestSetFlatteningOptimizer.java | 2 +- .../sql/planner/plan/TestWindowNode.java | 2 +- .../sanity/TestDynamicFiltersChecker.java | 16 +- .../TestVerifyNoOriginalExpression.java | 2 +- .../relational/TestDeterminismEvaluator.java | 10 +- .../relational/TestFunctionResolution.java | 6 +- .../relational/TestLogicalRowExpressions.java | 18 +- .../TestRowExpressionOptimizer.java | 26 +-- .../TestRowExpressionTranslator.java | 30 ++-- .../sql/relational/TestSubExpressions.java | 4 +- .../type/BenchmarkDecimalOperators.java | 2 +- .../presto/type/TestTypeRegistry.java | 22 +-- .../TestKHyperLogLogAggregationFunction.java | 6 +- .../presto/util/StructuralTestUtil.java | 4 +- .../ml/TestEvaluateClassifierPredictions.java | 8 +- .../presto/ml/TestLearnAggregations.java | 4 +- .../com/facebook/presto/orc/OrcTester.java | 4 +- .../orc/TestMapFlatBatchStreamReader.java | 4 +- .../presto/orc/TestOrcMapNullKey.java | 4 +- .../presto/orc/TestOrcReaderMemoryUsage.java | 4 +- .../orc/TestTupleDomainFilterUtils.java | 4 +- .../presto/pinot/TestPinotQueryBase.java | 4 +- .../presto/raptor/storage/OrcTestingUtil.java | 8 +- .../raptor/storage/TestOrcFileRewriter.java | 6 +- .../raptor/storage/TestShardWriter.java | 4 +- .../facebook/presto/rcfile/RcFileTester.java | 4 +- .../presto/spark/PrestoSparkModule.java | 4 +- .../PrestoSparkQueryExecutionFactory.java | 2 +- .../PrestoSparkTaskExecutorFactory.java | 14 +- .../presto/spark/PrestoSparkQueryRunner.java | 2 +- .../presto/tests/DistributedQueryRunner.java | 2 +- .../presto/tests/PlanDeterminismChecker.java | 2 +- .../presto/tests/StandaloneQueryRunner.java | 2 +- .../presto/tests/StatefulSleepingSum.java | 4 +- .../presto/tests/StructuralTestUtil.java | 4 +- .../verifier/framework/VerifierModule.java | 4 +- .../presto/verifier/VerifierTestUtil.java | 4 +- .../checksum/TestChecksumValidator.java | 4 +- 385 files changed, 1857 insertions(+), 1803 deletions(-) rename presto-main/src/main/java/com/facebook/presto/metadata/{FunctionManager.java => FunctionAndTypeManager.java} (93%) rename presto-main/src/test/java/com/facebook/presto/metadata/{TestFunctionManager.java => TestFunctionAndTypeManager.java} (85%) diff --git a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/model/TestField.java b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/model/TestField.java index 5e309b6b1e89..7453bdb2fabe 100644 --- a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/model/TestField.java +++ b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/model/TestField.java @@ -21,7 +21,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.type.TypeRegistry; import com.google.common.collect.ImmutableList; @@ -171,7 +171,7 @@ public void testMap() { TypeManager typeManager = new TypeRegistry(); // associate typeManager with a function manager - new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); Type type = typeManager.getParameterizedType(StandardTypes.MAP, ImmutableList.of( TypeSignatureParameter.of(VARCHAR.getTypeSignature()), diff --git a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/serializers/AbstractTestAccumuloRowSerializer.java b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/serializers/AbstractTestAccumuloRowSerializer.java index a061696022df..bdece1ef2faf 100644 --- a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/serializers/AbstractTestAccumuloRowSerializer.java +++ b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/serializers/AbstractTestAccumuloRowSerializer.java @@ -19,7 +19,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.type.TypeRegistry; import com.google.common.collect.ImmutableList; @@ -190,7 +190,7 @@ public void testMap() { TypeManager typeManager = new TypeRegistry(); // associate typeManager with a function manager - new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); AccumuloRowSerializer serializer = serializerClass.getConstructor().newInstance(); Type type = typeManager.getParameterizedType(StandardTypes.MAP, ImmutableList.of( diff --git a/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/optimization/TestJdbcComputePushdown.java b/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/optimization/TestJdbcComputePushdown.java index 707354c57bee..ae182f4c5ec3 100644 --- a/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/optimization/TestJdbcComputePushdown.java +++ b/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/optimization/TestJdbcComputePushdown.java @@ -19,7 +19,7 @@ import com.facebook.presto.cost.PlanNodeStatsEstimate; import com.facebook.presto.cost.StatsAndCosts; import com.facebook.presto.cost.StatsProvider; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.plugin.jdbc.JdbcColumnHandle; @@ -94,12 +94,12 @@ public class TestJdbcComputePushdown public TestJdbcComputePushdown() { this.sqlToRowExpressionTranslator = new TestingRowExpressionTranslator(METADATA); - FunctionManager functionManager = METADATA.getFunctionManager(); - StandardFunctionResolution functionResolution = new FunctionResolution(functionManager); - DeterminismEvaluator determinismEvaluator = new RowExpressionDeterminismEvaluator(functionManager); + FunctionAndTypeManager functionAndTypeManager = METADATA.getFunctionAndTypeManager(); + StandardFunctionResolution functionResolution = new FunctionResolution(functionAndTypeManager); + DeterminismEvaluator determinismEvaluator = new RowExpressionDeterminismEvaluator(functionAndTypeManager); this.jdbcComputePushdown = new JdbcComputePushdown( - functionManager, + functionAndTypeManager, functionResolution, determinismEvaluator, new RowExpressionOptimizer(METADATA), diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractOperatorBenchmark.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractOperatorBenchmark.java index 3dc4099d10fe..7afd5d9b16a0 100644 --- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractOperatorBenchmark.java +++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractOperatorBenchmark.java @@ -220,7 +220,7 @@ protected final OperatorFactory createHashProjectOperator(int operatorId, PlanNo projections.add(new PageProjectionWithOutputs(new InputPageProjection(channel), new int[] {channel})); } - Optional hashExpression = HashGenerationOptimizer.getHashExpression(localQueryRunner.getMetadata().getFunctionManager(), variables.build()); + Optional hashExpression = HashGenerationOptimizer.getHashExpression(localQueryRunner.getMetadata().getFunctionAndTypeManager(), variables.build()); verify(hashExpression.isPresent()); RowExpression translatedHashExpression = translate(hashExpression.get(), variableToInputMapping.build()); diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/CountAggregationBenchmark.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/CountAggregationBenchmark.java index 5fee68b3bf06..583c6f56284b 100644 --- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/CountAggregationBenchmark.java +++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/CountAggregationBenchmark.java @@ -13,7 +13,7 @@ */ package com.facebook.presto.benchmark; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.AggregationOperator.AggregationOperatorFactory; import com.facebook.presto.operator.OperatorFactory; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; @@ -41,9 +41,9 @@ public CountAggregationBenchmark(LocalQueryRunner localQueryRunner) protected List createOperatorFactories() { OperatorFactory tableScanOperator = createTableScanOperator(0, new PlanNodeId("test"), "orders", "orderkey"); - FunctionManager functionManager = localQueryRunner.getMetadata().getFunctionManager(); - InternalAggregationFunction countFunction = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("count", fromTypes(BIGINT))); + FunctionAndTypeManager functionAndTypeManager = localQueryRunner.getMetadata().getFunctionAndTypeManager(); + InternalAggregationFunction countFunction = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("count", fromTypes(BIGINT))); AggregationOperatorFactory aggregationOperator = new AggregationOperatorFactory(1, new PlanNodeId("test"), Step.SINGLE, ImmutableList.of(countFunction.bind(ImmutableList.of(0), Optional.empty())), false); return ImmutableList.of(tableScanOperator, aggregationOperator); } diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/DoubleSumAggregationBenchmark.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/DoubleSumAggregationBenchmark.java index 113abf985274..88b2860440f1 100644 --- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/DoubleSumAggregationBenchmark.java +++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/DoubleSumAggregationBenchmark.java @@ -13,7 +13,7 @@ */ package com.facebook.presto.benchmark; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.AggregationOperator.AggregationOperatorFactory; import com.facebook.presto.operator.OperatorFactory; @@ -42,9 +42,9 @@ public DoubleSumAggregationBenchmark(LocalQueryRunner localQueryRunner) protected List createOperatorFactories() { OperatorFactory tableScanOperator = createTableScanOperator(0, new PlanNodeId("test"), "orders", "totalprice"); - FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); - InternalAggregationFunction doubleSum = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("sum", fromTypes(DOUBLE))); + FunctionAndTypeManager functionAndTypeManager = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); + InternalAggregationFunction doubleSum = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("sum", fromTypes(DOUBLE))); AggregationOperatorFactory aggregationOperator = new AggregationOperatorFactory(1, new PlanNodeId("test"), Step.SINGLE, ImmutableList.of(doubleSum.bind(ImmutableList.of(0), Optional.empty())), false); return ImmutableList.of(tableScanOperator, aggregationOperator); } diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery1.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery1.java index f934be91d8bf..3cb096ef289e 100644 --- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery1.java +++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery1.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.PageBuilder; import com.facebook.presto.common.block.Block; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.DriverContext; import com.facebook.presto.operator.HashAggregationOperator.HashAggregationOperatorFactory; import com.facebook.presto.operator.Operator; @@ -58,15 +58,15 @@ public HandTpchQuery1(LocalQueryRunner localQueryRunner) { super(localQueryRunner, "hand_tpch_query_1", 1, 5); - FunctionManager functionManager = localQueryRunner.getMetadata().getFunctionManager(); - longAverage = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("avg", fromTypes(BIGINT))); - doubleAverage = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("avg", fromTypes(DOUBLE))); - doubleSum = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("sum", fromTypes(DOUBLE))); - countFunction = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("count", ImmutableList.of())); + FunctionAndTypeManager functionAndTypeManager = localQueryRunner.getMetadata().getFunctionAndTypeManager(); + longAverage = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("avg", fromTypes(BIGINT))); + doubleAverage = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("avg", fromTypes(DOUBLE))); + doubleSum = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("sum", fromTypes(DOUBLE))); + countFunction = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("count", ImmutableList.of())); } @Override diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery6.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery6.java index 21ff0da55249..87398c6271ab 100644 --- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery6.java +++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery6.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.Page; import com.facebook.presto.common.block.Block; import com.facebook.presto.common.function.SqlFunctionProperties; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.AggregationOperator.AggregationOperatorFactory; import com.facebook.presto.operator.FilterAndProjectOperator; import com.facebook.presto.operator.OperatorFactory; @@ -55,9 +55,9 @@ public class HandTpchQuery6 public HandTpchQuery6(LocalQueryRunner localQueryRunner) { super(localQueryRunner, "hand_tpch_query_6", 10, 100); - FunctionManager functionManager = localQueryRunner.getMetadata().getFunctionManager(); - doubleSum = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("sum", fromTypes(DOUBLE))); + FunctionAndTypeManager functionAndTypeManager = localQueryRunner.getMetadata().getFunctionAndTypeManager(); + doubleSum = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("sum", fromTypes(DOUBLE))); } @Override diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HashAggregationBenchmark.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HashAggregationBenchmark.java index ea96901a7a0e..2c8514cc11e8 100644 --- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HashAggregationBenchmark.java +++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HashAggregationBenchmark.java @@ -14,7 +14,7 @@ package com.facebook.presto.benchmark; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.HashAggregationOperator.HashAggregationOperatorFactory; import com.facebook.presto.operator.OperatorFactory; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; @@ -42,9 +42,9 @@ public HashAggregationBenchmark(LocalQueryRunner localQueryRunner) { super(localQueryRunner, "hash_agg", 5, 25); - FunctionManager functionManager = localQueryRunner.getMetadata().getFunctionManager(); - doubleSum = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("sum", fromTypes(DOUBLE))); + FunctionAndTypeManager functionAndTypeManager = localQueryRunner.getMetadata().getFunctionAndTypeManager(); + doubleSum = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("sum", fromTypes(DOUBLE))); } @Override diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/PredicateFilterBenchmark.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/PredicateFilterBenchmark.java index 71d78ec8df11..d24f92f3d4ab 100644 --- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/PredicateFilterBenchmark.java +++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/PredicateFilterBenchmark.java @@ -54,7 +54,7 @@ protected List createOperatorFactories() OperatorFactory tableScanOperator = createTableScanOperator(0, new PlanNodeId("test"), "orders", "totalprice"); RowExpression filter = call( GREATER_THAN_OR_EQUAL.name(), - metadata.getFunctionManager().resolveOperator(GREATER_THAN_OR_EQUAL, fromTypes(DOUBLE, DOUBLE)), + metadata.getFunctionAndTypeManager().resolveOperator(GREATER_THAN_OR_EQUAL, fromTypes(DOUBLE, DOUBLE)), BOOLEAN, field(0, DOUBLE), constant(50000.0, DOUBLE)); 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 5b2d5b8e59d0..9b606d690009 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,7 @@ import com.facebook.presto.common.predicate.TupleDomain; import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.metadata.SessionPropertyManager; @@ -84,7 +84,7 @@ public class TestDruidQueryBase { protected static final TypeManager typeManager = new TypeRegistry(); - protected static final FunctionManager functionMetadataManager = new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + protected static final FunctionAndTypeManager functionMetadataManager = new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); protected static final StandardFunctionResolution standardFunctionResolution = new FunctionResolution(functionMetadataManager); protected static ConnectorId druidConnectorId = new ConnectorId("id"); diff --git a/presto-geospatial/src/test/java/com/facebook/presto/plugin/geospatial/TestBingTileFunctions.java b/presto-geospatial/src/test/java/com/facebook/presto/plugin/geospatial/TestBingTileFunctions.java index 1df7c61eb09c..e89b232d36fa 100644 --- a/presto-geospatial/src/test/java/com/facebook/presto/plugin/geospatial/TestBingTileFunctions.java +++ b/presto-geospatial/src/test/java/com/facebook/presto/plugin/geospatial/TestBingTileFunctions.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.Page; import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; import com.facebook.presto.operator.scalar.AbstractTestFunctions; import com.google.common.collect.ImmutableList; @@ -66,9 +66,9 @@ protected void registerFunctions() } functionAssertions.getMetadata().registerBuiltInFunctions(extractFunctions(plugin.getFunctions())); functionAssertions.getMetadata().registerBuiltInFunctions(ImmutableList.of(APPLY_FUNCTION)); - FunctionManager functionManager = functionAssertions.getMetadata().getFunctionManager(); - approxDistinct = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("approx_distinct", fromTypes(BING_TILE))); + FunctionAndTypeManager functionAndTypeManager = functionAssertions.getMetadata().getFunctionAndTypeManager(); + approxDistinct = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("approx_distinct", fromTypes(BING_TILE))); } @Test diff --git a/presto-geospatial/src/test/java/com/facebook/presto/plugin/geospatial/TestSpatialPartitioningInternalAggregation.java b/presto-geospatial/src/test/java/com/facebook/presto/plugin/geospatial/TestSpatialPartitioningInternalAggregation.java index dcfe2b13b1ba..2a2e220a8967 100644 --- a/presto-geospatial/src/test/java/com/facebook/presto/plugin/geospatial/TestSpatialPartitioningInternalAggregation.java +++ b/presto-geospatial/src/test/java/com/facebook/presto/plugin/geospatial/TestSpatialPartitioningInternalAggregation.java @@ -23,7 +23,7 @@ import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.geospatial.KdbTreeUtils; import com.facebook.presto.geospatial.Rectangle; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.aggregation.Accumulator; import com.facebook.presto.operator.aggregation.AccumulatorFactory; import com.facebook.presto.operator.aggregation.GroupedAccumulator; @@ -119,9 +119,9 @@ public void testEmptyPartitionException() private InternalAggregationFunction getFunction() { - FunctionManager functionManager = functionAssertions.getMetadata().getFunctionManager(); - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("spatial_partitioning", fromTypes(GEOMETRY, INTEGER))); + FunctionAndTypeManager functionAndTypeManager = functionAssertions.getMetadata().getFunctionAndTypeManager(); + return functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("spatial_partitioning", fromTypes(GEOMETRY, INTEGER))); } private List makeGeometries() diff --git a/presto-geospatial/src/test/java/com/facebook/presto/plugin/geospatial/aggregation/AbstractTestGeoAggregationFunctions.java b/presto-geospatial/src/test/java/com/facebook/presto/plugin/geospatial/aggregation/AbstractTestGeoAggregationFunctions.java index 5c63a2b1c0fe..d682011767b1 100644 --- a/presto-geospatial/src/test/java/com/facebook/presto/plugin/geospatial/aggregation/AbstractTestGeoAggregationFunctions.java +++ b/presto-geospatial/src/test/java/com/facebook/presto/plugin/geospatial/aggregation/AbstractTestGeoAggregationFunctions.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.Page; import com.facebook.presto.common.type.Type; import com.facebook.presto.geospatial.serde.EsriGeometrySerde; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; import com.facebook.presto.operator.scalar.AbstractTestFunctions; import com.facebook.presto.plugin.geospatial.GeoPlugin; @@ -49,9 +49,9 @@ public void registerFunctions() functionAssertions.getTypeRegistry().addType(type); } functionAssertions.getMetadata().registerBuiltInFunctions(extractFunctions(plugin.getFunctions())); - FunctionManager functionManager = functionAssertions.getMetadata().getFunctionManager(); - function = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction(getFunctionName(), fromTypes(GEOMETRY))); + FunctionAndTypeManager functionAndTypeManager = functionAssertions.getMetadata().getFunctionAndTypeManager(); + function = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction(getFunctionName(), fromTypes(GEOMETRY))); } protected void assertAggregatedGeometries(String testDescription, String expectedWkt, String... wkts) diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java index 7b7dc4c58e91..386d6bd29460 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java @@ -1412,7 +1412,7 @@ protected void doTestMismatchSchemaTable( ROW_EXPRESSION_SERVICE, FUNCTION_RESOLUTION, hivePartitionManager, - METADATA.getFunctionManager(), + METADATA.getFunctionAndTypeManager(), tableHandle, predicate, Optional.empty()).getLayout().getHandle(); @@ -2166,7 +2166,7 @@ private void doTestBucketedTableEvolution(HiveStorageFormat storageFormat, Schem ROW_EXPRESSION_SERVICE, FUNCTION_RESOLUTION, hivePartitionManager, - METADATA.getFunctionManager(), + METADATA.getFunctionAndTypeManager(), tableHandle, predicate, Optional.empty()).getLayout().getHandle(); @@ -2496,7 +2496,7 @@ protected ConnectorTableLayout getTableLayout(ConnectorSession session, Connecto ROW_EXPRESSION_SERVICE, FUNCTION_RESOLUTION, hivePartitionManager, - METADATA.getFunctionManager(), + METADATA.getFunctionAndTypeManager(), tableHandle, TRUE_CONSTANT, Optional.empty()).getLayout(); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java b/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java index 819f463fe521..c4cff9904184 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java @@ -42,7 +42,7 @@ import com.facebook.presto.hive.rcfile.RcFilePageSourceFactory; import com.facebook.presto.hive.s3.HiveS3Config; import com.facebook.presto.hive.s3.PrestoS3ConfigurationUpdater; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.PagesIndex; import com.facebook.presto.orc.StorageStripeMetadataSource; @@ -95,7 +95,7 @@ private HiveTestUtils() public static final MetadataManager METADATA = MetadataManager.createTestMetadataManager(); - public static final StandardFunctionResolution FUNCTION_RESOLUTION = new FunctionResolution(METADATA.getFunctionManager()); + public static final StandardFunctionResolution FUNCTION_RESOLUTION = new FunctionResolution(METADATA.getFunctionAndTypeManager()); public static final RowExpressionService ROW_EXPRESSION_SERVICE = new RowExpressionService() { @@ -126,7 +126,7 @@ public DeterminismEvaluator getDeterminismEvaluator() @Override public String formatRowExpression(ConnectorSession session, RowExpression expression) { - return new RowExpressionFormatter(METADATA.getFunctionManager()).formatRowExpression(session, expression); + return new RowExpressionFormatter(METADATA.getFunctionAndTypeManager()).formatRowExpression(session, expression); } }; @@ -135,7 +135,7 @@ public String formatRowExpression(ConnectorSession session, RowExpression expres static { // associate TYPE_MANAGER with a function manager - new FunctionManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); } public static final HiveClientConfig HIVE_CLIENT_CONFIG = new HiveClientConfig(); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestDomainTranslator.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestDomainTranslator.java index 4299f8f68a82..9127fc71c658 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestDomainTranslator.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestDomainTranslator.java @@ -107,7 +107,7 @@ public void setup() metadata = createTestMetadataManager(); domainTranslator = new RowExpressionDomainTranslator(metadata); columnExtractor = new SubfieldExtractor( - new FunctionResolution(metadata.getFunctionManager()), + new FunctionResolution(metadata.getFunctionAndTypeManager()), TEST_EXPRESSION_OPTIMIZER, new TestingConnectorSession( new HiveSessionProperties( @@ -170,7 +170,7 @@ private RowExpression isNull(RowExpression expression) private RowExpression not(RowExpression expression) { - return call("not", new FunctionResolution(metadata.getFunctionManager()).notFunction(), BOOLEAN, expression); + return call("not", new FunctionResolution(metadata.getFunctionAndTypeManager()).notFunction(), BOOLEAN, expression); } private RowExpression arraySubscript(RowExpression arrayExpression, int index) @@ -194,7 +194,7 @@ private RowExpression mapSubscript(RowExpression mapExpression, RowExpression ke private FunctionHandle operator(OperatorType operatorType, Type... types) { - return metadata.getFunctionManager().resolveOperator(operatorType, fromTypes(types)); + return metadata.getFunctionAndTypeManager().resolveOperator(operatorType, fromTypes(types)); } private void assertPredicateTranslates(RowExpression predicate, String subfield, Domain domain) @@ -225,7 +225,7 @@ private RowExpression between(RowExpression value, RowExpression min, RowExpress { return call( OperatorType.BETWEEN.name(), - metadata.getFunctionManager().resolveOperator(OperatorType.BETWEEN, fromTypes(value.getType(), min.getType(), max.getType())), + metadata.getFunctionAndTypeManager().resolveOperator(OperatorType.BETWEEN, fromTypes(value.getType(), min.getType(), max.getType())), BOOLEAN, value, min, @@ -247,7 +247,7 @@ private RowExpression binaryOperator(OperatorType operatorType, RowExpression le { return call( operatorType.name(), - metadata.getFunctionManager().resolveOperator(operatorType, fromTypes(left.getType(), right.getType())), + metadata.getFunctionAndTypeManager().resolveOperator(operatorType, fromTypes(left.getType(), right.getType())), BOOLEAN, left, right); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java index c73905336a2a..d02c8b6fa09e 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java @@ -3946,7 +3946,7 @@ public static Consumer assertRemoteExchangesCount(int expectedRemoteExchan .size(); if (actualRemoteExchangesCount != expectedRemoteExchangesCount) { Metadata metadata = queryRunner.getCoordinator().getMetadata(); - String formattedPlan = textLogicalPlan(plan.getRoot(), plan.getTypes(), metadata.getFunctionManager(), StatsAndCosts.empty(), session, 0); + String formattedPlan = textLogicalPlan(plan.getRoot(), plan.getTypes(), metadata.getFunctionAndTypeManager(), StatsAndCosts.empty(), session, 0); throw new AssertionError(format( "Expected [\n%s\n] remote exchanges but found [\n%s\n] remote exchanges. Actual plan is [\n\n%s\n]", expectedRemoteExchangesCount, diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveLogicalPlanner.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveLogicalPlanner.java index 6324921d73ff..badc65a0a42e 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveLogicalPlanner.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveLogicalPlanner.java @@ -21,7 +21,7 @@ import com.facebook.presto.common.predicate.ValueSet; import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.cost.StatsProvider; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorTableLayoutHandle; @@ -231,14 +231,14 @@ public void testPushdownFilter() assertPlan(pushdownFilterEnabled, "SELECT linenumber FROM lineitem WHERE orderkey = 1 AND orderkey = 2 AND linenumber % 2 = 1", output(values("linenumber"))); - FunctionManager functionManager = getQueryRunner().getMetadata().getFunctionManager(); - FunctionResolution functionResolution = new FunctionResolution(functionManager); + FunctionAndTypeManager functionAndTypeManager = getQueryRunner().getMetadata().getFunctionAndTypeManager(); + FunctionResolution functionResolution = new FunctionResolution(functionAndTypeManager); RowExpression remainingPredicate = new CallExpression(EQUAL.name(), functionResolution.comparisonFunction(EQUAL, BIGINT, BIGINT), BOOLEAN, ImmutableList.of( new CallExpression("mod", - functionManager.lookupFunction("mod", fromTypes(BIGINT, BIGINT)), + functionAndTypeManager.lookupFunction("mod", fromTypes(BIGINT, BIGINT)), BIGINT, ImmutableList.of( new VariableReferenceExpression("orderkey", BIGINT), diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestSubfieldExtractor.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestSubfieldExtractor.java index 6c0a1a22f653..aaec7bf5b04b 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestSubfieldExtractor.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestSubfieldExtractor.java @@ -21,7 +21,7 @@ import com.facebook.presto.common.type.RowFieldName; import com.facebook.presto.common.type.RowType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.TestingSession; import com.facebook.presto.spi.function.FunctionHandle; @@ -86,15 +86,15 @@ public Object optimize(RowExpression expression, Level level, ConnectorSession s } }; - private FunctionManager functionManager; + private FunctionAndTypeManager functionAndTypeManager; private SubfieldExtractor subfieldExtractor; @BeforeClass public void setup() { - functionManager = createTestMetadataManager().getFunctionManager(); + functionAndTypeManager = createTestMetadataManager().getFunctionAndTypeManager(); subfieldExtractor = new SubfieldExtractor( - new FunctionResolution(functionManager), + new FunctionResolution(functionAndTypeManager), TEST_EXPRESSION_OPTIMIZER, TestingSession.SESSION); } @@ -174,6 +174,6 @@ private RowExpression mapSubscript(RowExpression mapExpression, RowExpression ke private FunctionHandle operator(OperatorType operatorType, Type... types) { - return functionManager.resolveOperator(operatorType, fromTypes(types)); + return functionAndTypeManager.resolveOperator(operatorType, fromTypes(types)); } } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/TestParquetPageSourceFactory.java b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/TestParquetPageSourceFactory.java index eaa797e2b16b..01eb21dc2049 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/TestParquetPageSourceFactory.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/TestParquetPageSourceFactory.java @@ -44,7 +44,7 @@ public class TestParquetPageSourceFactory private static final String PARQUET_HIVE_SERDE = "parquet.hive.serde.ParquetHiveSerDe"; private ParquetPageSourceFactory parquetPageSourceFactory; - private final StandardFunctionResolution functionResolution = new FunctionResolution(METADATA.getFunctionManager()); + private final StandardFunctionResolution functionResolution = new FunctionResolution(METADATA.getFunctionAndTypeManager()); @BeforeClass public void setUp() diff --git a/presto-main/src/main/java/com/facebook/presto/connector/ConnectorManager.java b/presto-main/src/main/java/com/facebook/presto/connector/ConnectorManager.java index 8d6da0cb5dbd..e50ca0905b96 100644 --- a/presto-main/src/main/java/com/facebook/presto/connector/ConnectorManager.java +++ b/presto-main/src/main/java/com/facebook/presto/connector/ConnectorManager.java @@ -362,8 +362,8 @@ private Connector createConnector(ConnectorId connectorId, ConnectorFactory fact ConnectorContext context = new ConnectorContextInstance( new ConnectorAwareNodeManager(nodeManager, nodeInfo.getEnvironment(), connectorId), typeManager, - metadataManager.getFunctionManager(), - new FunctionResolution(metadataManager.getFunctionManager()), + metadataManager.getFunctionAndTypeManager(), + new FunctionResolution(metadataManager.getFunctionAndTypeManager()), pageSorter, pageIndexerFactory, new ConnectorRowExpressionService( @@ -371,7 +371,7 @@ private Connector createConnector(ConnectorId connectorId, ConnectorFactory fact new RowExpressionOptimizer(metadataManager), predicateCompiler, determinismEvaluator, - new RowExpressionFormatter(metadataManager.getFunctionManager())), + new RowExpressionFormatter(metadataManager.getFunctionAndTypeManager())), new ConnectorFilterStatsCalculatorService(filterStatsCalculator), blockEncodingSerde); diff --git a/presto-main/src/main/java/com/facebook/presto/cost/FilterStatsCalculator.java b/presto-main/src/main/java/com/facebook/presto/cost/FilterStatsCalculator.java index 4c53b410e2ce..674b191ad00f 100644 --- a/presto-main/src/main/java/com/facebook/presto/cost/FilterStatsCalculator.java +++ b/presto-main/src/main/java/com/facebook/presto/cost/FilterStatsCalculator.java @@ -16,7 +16,7 @@ import com.facebook.presto.Session; import com.facebook.presto.common.function.OperatorType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.WarningCollector; @@ -115,7 +115,7 @@ public FilterStatsCalculator(Metadata metadata, ScalarStatsCalculator scalarStat this.scalarStatsCalculator = requireNonNull(scalarStatsCalculator, "scalarStatsCalculator is null"); this.normalizer = requireNonNull(normalizer, "normalizer is null"); this.literalEncoder = new LiteralEncoder(metadata.getBlockEncodingSerde()); - this.functionResolution = new FunctionResolution(metadata.getFunctionManager()); + this.functionResolution = new FunctionResolution(metadata.getFunctionAndTypeManager()); } @Deprecated @@ -136,7 +136,7 @@ public PlanNodeStatsEstimate filterStats( ConnectorSession session) { RowExpression simplifiedExpression = simplifyExpression(session, predicate); - return new FilterRowExpressionStatsCalculatingVisitor(statsEstimate, session, metadata.getFunctionManager()).process(simplifiedExpression); + return new FilterRowExpressionStatsCalculatingVisitor(statsEstimate, session, metadata.getFunctionAndTypeManager()).process(simplifiedExpression); } public PlanNodeStatsEstimate filterStats( @@ -177,7 +177,7 @@ private RowExpression simplifyExpression(ConnectorSession session, RowExpression private Map, Type> getExpressionTypes(Session session, Expression expression, TypeProvider types) { ExpressionAnalyzer expressionAnalyzer = ExpressionAnalyzer.createWithoutSubqueries( - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), session, types, @@ -460,7 +460,7 @@ private Type getType(Expression expression) } ExpressionAnalyzer expressionAnalyzer = ExpressionAnalyzer.createWithoutSubqueries( - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), session, types, @@ -493,13 +493,13 @@ private class FilterRowExpressionStatsCalculatingVisitor { private final PlanNodeStatsEstimate input; private final ConnectorSession session; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - FilterRowExpressionStatsCalculatingVisitor(PlanNodeStatsEstimate input, ConnectorSession session, FunctionManager functionManager) + FilterRowExpressionStatsCalculatingVisitor(PlanNodeStatsEstimate input, ConnectorSession session, FunctionAndTypeManager functionAndTypeManager) { this.input = requireNonNull(input, "input is null"); this.session = requireNonNull(session, "session is null"); - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override @@ -554,7 +554,7 @@ public PlanNodeStatsEstimate visitVariableReference(VariableReferenceExpression public PlanNodeStatsEstimate visitCall(CallExpression node, Void context) { // comparison case - FunctionMetadata functionMetadata = metadata.getFunctionManager().getFunctionMetadata(node.getFunctionHandle()); + FunctionMetadata functionMetadata = metadata.getFunctionAndTypeManager().getFunctionMetadata(node.getFunctionHandle()); if (functionMetadata.getOperatorType().map(OperatorType::isComparisonOperator).orElse(false)) { OperatorType operatorType = functionMetadata.getOperatorType().get(); RowExpression left = node.getArguments().get(0); @@ -565,13 +565,13 @@ public PlanNodeStatsEstimate visitCall(CallExpression node, Void context) if (!(left instanceof VariableReferenceExpression) && right instanceof VariableReferenceExpression) { // normalize so that variable is on the left OperatorType flippedOperator = flip(operatorType); - return process(call(flippedOperator.name(), metadata.getFunctionManager().resolveOperator(flippedOperator, fromTypes(right.getType(), left.getType())), BOOLEAN, right, left)); + return process(call(flippedOperator.name(), metadata.getFunctionAndTypeManager().resolveOperator(flippedOperator, fromTypes(right.getType(), left.getType())), BOOLEAN, right, left)); } if (left instanceof ConstantExpression) { // normalize so that literal is on the right OperatorType flippedOperator = flip(operatorType); - return process(call(flippedOperator.name(), metadata.getFunctionManager().resolveOperator(flippedOperator, fromTypes(right.getType(), left.getType())), BOOLEAN, right, left)); + return process(call(flippedOperator.name(), metadata.getFunctionAndTypeManager().resolveOperator(flippedOperator, fromTypes(right.getType(), left.getType())), BOOLEAN, right, left)); } if (left instanceof VariableReferenceExpression && left.equals(right)) { @@ -585,7 +585,7 @@ public PlanNodeStatsEstimate visitCall(CallExpression node, Void context) if (rightValue == null) { return visitConstant(constantNull(BOOLEAN), null); } - OptionalDouble literal = toStatsRepresentation(metadata.getFunctionManager(), session, right.getType(), rightValue); + OptionalDouble literal = toStatsRepresentation(metadata.getFunctionAndTypeManager(), session, right.getType(), rightValue); return estimateExpressionToLiteralComparison(input, leftStats, leftVariable, literal, getComparisonOperator(operatorType)); } @@ -636,13 +636,13 @@ public PlanNodeStatsEstimate visitCall(CallExpression node, Void context) VariableStatsEstimate valueStats = input.getVariableStatistics((VariableReferenceExpression) value); RowExpression lowerBound = call( OperatorType.GREATER_THAN_OR_EQUAL.name(), - metadata.getFunctionManager().resolveOperator(OperatorType.GREATER_THAN_OR_EQUAL, fromTypes(value.getType(), min.getType())), + metadata.getFunctionAndTypeManager().resolveOperator(OperatorType.GREATER_THAN_OR_EQUAL, fromTypes(value.getType(), min.getType())), BOOLEAN, value, min); RowExpression upperBound = call( OperatorType.LESS_THAN_OR_EQUAL.name(), - metadata.getFunctionManager().resolveOperator(OperatorType.LESS_THAN_OR_EQUAL, fromTypes(value.getType(), max.getType())), + metadata.getFunctionAndTypeManager().resolveOperator(OperatorType.LESS_THAN_OR_EQUAL, fromTypes(value.getType(), max.getType())), BOOLEAN, value, max); @@ -674,7 +674,7 @@ public PlanNodeStatsEstimate visitInputReference(InputReferenceExpression node, private FilterRowExpressionStatsCalculatingVisitor newEstimate(PlanNodeStatsEstimate input) { - return new FilterRowExpressionStatsCalculatingVisitor(input, session, functionManager); + return new FilterRowExpressionStatsCalculatingVisitor(input, session, functionAndTypeManager); } private PlanNodeStatsEstimate process(RowExpression rowExpression) @@ -739,7 +739,7 @@ private PlanNodeStatsEstimate estimateLogicalOr(RowExpression left, RowExpressio private PlanNodeStatsEstimate estimateIn(RowExpression value, List candidates) { ImmutableList equalityEstimates = candidates.stream() - .map(inValue -> process(call(OperatorType.EQUAL.name(), metadata.getFunctionManager().resolveOperator(OperatorType.EQUAL, fromTypes(value.getType(), inValue.getType())), BOOLEAN, value, inValue))) + .map(inValue -> process(call(OperatorType.EQUAL.name(), metadata.getFunctionAndTypeManager().resolveOperator(OperatorType.EQUAL, fromTypes(value.getType(), inValue.getType())), BOOLEAN, value, inValue))) .collect(toImmutableList()); if (equalityEstimates.stream().anyMatch(PlanNodeStatsEstimate::isOutputRowCountUnknown)) { diff --git a/presto-main/src/main/java/com/facebook/presto/cost/ScalarStatsCalculator.java b/presto-main/src/main/java/com/facebook/presto/cost/ScalarStatsCalculator.java index c4bce3879383..19e4af5ebf43 100644 --- a/presto-main/src/main/java/com/facebook/presto/cost/ScalarStatsCalculator.java +++ b/presto-main/src/main/java/com/facebook/presto/cost/ScalarStatsCalculator.java @@ -105,7 +105,7 @@ private class RowExpressionStatsVisitor { private final PlanNodeStatsEstimate input; private final ConnectorSession session; - private final FunctionResolution resolution = new FunctionResolution(metadata.getFunctionManager()); + private final FunctionResolution resolution = new FunctionResolution(metadata.getFunctionAndTypeManager()); public RowExpressionStatsVisitor(PlanNodeStatsEstimate input, ConnectorSession session) { @@ -120,7 +120,7 @@ public VariableStatsEstimate visitCall(CallExpression call, Void context) return computeNegationStatistics(call, context); } - FunctionMetadata functionMetadata = metadata.getFunctionManager().getFunctionMetadata(call.getFunctionHandle()); + FunctionMetadata functionMetadata = metadata.getFunctionAndTypeManager().getFunctionMetadata(call.getFunctionHandle()); if (functionMetadata.getOperatorType().map(OperatorType::isArithmeticOperator).orElse(false)) { return computeArithmeticBinaryStatistics(call, context); } @@ -155,7 +155,7 @@ public VariableStatsEstimate visitConstant(ConstantExpression literal, Void cont return nullStatsEstimate(); } - OptionalDouble doubleValue = toStatsRepresentation(metadata.getFunctionManager(), session, literal.getType(), literal.getValue()); + OptionalDouble doubleValue = toStatsRepresentation(metadata.getFunctionAndTypeManager(), session, literal.getType(), literal.getValue()); VariableStatsEstimate.Builder estimate = VariableStatsEstimate.builder() .setNullsFraction(0) .setDistinctValuesCount(1); @@ -256,7 +256,7 @@ private VariableStatsEstimate computeArithmeticBinaryStatistics(CallExpression c .setNullsFraction(left.getNullsFraction() + right.getNullsFraction() - left.getNullsFraction() * right.getNullsFraction()) .setDistinctValuesCount(min(left.getDistinctValuesCount() * right.getDistinctValuesCount(), input.getOutputRowCount())); - FunctionMetadata functionMetadata = metadata.getFunctionManager().getFunctionMetadata(call.getFunctionHandle()); + FunctionMetadata functionMetadata = metadata.getFunctionAndTypeManager().getFunctionMetadata(call.getFunctionHandle()); checkState(functionMetadata.getOperatorType().isPresent()); OperatorType operatorType = functionMetadata.getOperatorType().get(); double leftLow = left.getLowValue(); @@ -394,7 +394,7 @@ protected VariableStatsEstimate visitFunctionCall(FunctionCall node, Void contex private Map, Type> getExpressionTypes(Session session, Expression expression, TypeProvider types) { ExpressionAnalyzer expressionAnalyzer = ExpressionAnalyzer.createWithoutSubqueries( - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), session, types, diff --git a/presto-main/src/main/java/com/facebook/presto/cost/SimpleFilterProjectSemiJoinStatsRule.java b/presto-main/src/main/java/com/facebook/presto/cost/SimpleFilterProjectSemiJoinStatsRule.java index 0c885f20cff3..c4c7fe915b29 100644 --- a/presto-main/src/main/java/com/facebook/presto/cost/SimpleFilterProjectSemiJoinStatsRule.java +++ b/presto-main/src/main/java/com/facebook/presto/cost/SimpleFilterProjectSemiJoinStatsRule.java @@ -16,7 +16,7 @@ import com.facebook.presto.Session; import com.facebook.presto.expressions.LogicalRowExpressions; import com.facebook.presto.matching.Pattern; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.plan.FilterNode; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.ProjectNode; @@ -63,13 +63,13 @@ public class SimpleFilterProjectSemiJoinStatsRule private final LogicalRowExpressions logicalRowExpressions; private final FunctionResolution functionResolution; - public SimpleFilterProjectSemiJoinStatsRule(StatsNormalizer normalizer, FilterStatsCalculator filterStatsCalculator, FunctionManager functionManager) + public SimpleFilterProjectSemiJoinStatsRule(StatsNormalizer normalizer, FilterStatsCalculator filterStatsCalculator, FunctionAndTypeManager functionAndTypeManager) { super(normalizer); this.filterStatsCalculator = requireNonNull(filterStatsCalculator, "filterStatsCalculator can not be null"); - requireNonNull(functionManager, "functionManager can not be null"); - this.logicalRowExpressions = new LogicalRowExpressions(new RowExpressionDeterminismEvaluator(functionManager), new FunctionResolution(functionManager), functionManager); - this.functionResolution = new FunctionResolution(functionManager); + requireNonNull(functionAndTypeManager, "functionManager can not be null"); + this.logicalRowExpressions = new LogicalRowExpressions(new RowExpressionDeterminismEvaluator(functionAndTypeManager), new FunctionResolution(functionAndTypeManager), functionAndTypeManager); + this.functionResolution = new FunctionResolution(functionAndTypeManager); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/cost/StatsCalculatorModule.java b/presto-main/src/main/java/com/facebook/presto/cost/StatsCalculatorModule.java index 6cf33397cbe4..96474a6424d1 100644 --- a/presto-main/src/main/java/com/facebook/presto/cost/StatsCalculatorModule.java +++ b/presto-main/src/main/java/com/facebook/presto/cost/StatsCalculatorModule.java @@ -44,7 +44,7 @@ public static StatsCalculator createNewStatsCalculator( ImmutableList.Builder> rules = ImmutableList.builder(); rules.add(new OutputStatsRule()); rules.add(new TableScanStatsRule(metadata, normalizer)); - rules.add(new SimpleFilterProjectSemiJoinStatsRule(normalizer, filterStatsCalculator, metadata.getFunctionManager())); // this must be before FilterStatsRule + rules.add(new SimpleFilterProjectSemiJoinStatsRule(normalizer, filterStatsCalculator, metadata.getFunctionAndTypeManager())); // this must be before FilterStatsRule rules.add(new FilterStatsRule(normalizer, filterStatsCalculator)); rules.add(new ValuesStatsRule(metadata)); rules.add(new LimitStatsRule(normalizer)); diff --git a/presto-main/src/main/java/com/facebook/presto/cost/StatsUtil.java b/presto-main/src/main/java/com/facebook/presto/cost/StatsUtil.java index b608db1ce173..449123cfd4d2 100644 --- a/presto-main/src/main/java/com/facebook/presto/cost/StatsUtil.java +++ b/presto-main/src/main/java/com/facebook/presto/cost/StatsUtil.java @@ -24,7 +24,7 @@ import com.facebook.presto.common.type.SmallintType; import com.facebook.presto.common.type.TinyintType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.function.FunctionHandle; @@ -44,16 +44,16 @@ private StatsUtil() {} static OptionalDouble toStatsRepresentation(Metadata metadata, Session session, Type type, Object value) { - return toStatsRepresentation(metadata.getFunctionManager(), session.toConnectorSession(), type, value); + return toStatsRepresentation(metadata.getFunctionAndTypeManager(), session.toConnectorSession(), type, value); } - static OptionalDouble toStatsRepresentation(FunctionManager functionManager, ConnectorSession session, Type type, Object value) + static OptionalDouble toStatsRepresentation(FunctionAndTypeManager functionAndTypeManager, ConnectorSession session, Type type, Object value) { requireNonNull(value, "value is null"); if (convertibleToDoubleWithCast(type)) { - InterpretedFunctionInvoker functionInvoker = new InterpretedFunctionInvoker(functionManager); - FunctionHandle cast = functionManager.lookupCast(CAST, type.getTypeSignature(), DoubleType.DOUBLE.getTypeSignature()); + InterpretedFunctionInvoker functionInvoker = new InterpretedFunctionInvoker(functionAndTypeManager); + FunctionHandle cast = functionAndTypeManager.lookupCast(CAST, type.getTypeSignature(), DoubleType.DOUBLE.getTypeSignature()); return OptionalDouble.of((double) functionInvoker.invoke(cast, session.getSqlFunctionProperties(), singletonList(value))); } diff --git a/presto-main/src/main/java/com/facebook/presto/event/QueryMonitor.java b/presto-main/src/main/java/com/facebook/presto/event/QueryMonitor.java index b9aa32b42d47..5fa29830a099 100644 --- a/presto-main/src/main/java/com/facebook/presto/event/QueryMonitor.java +++ b/presto-main/src/main/java/com/facebook/presto/event/QueryMonitor.java @@ -31,7 +31,7 @@ import com.facebook.presto.execution.TaskId; import com.facebook.presto.execution.TaskInfo; import com.facebook.presto.execution.TaskState; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.operator.OperatorInfo; @@ -91,7 +91,7 @@ public class QueryMonitor private final String serverAddress; private final String environment; private final SessionPropertyManager sessionPropertyManager; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final int maxJsonLimit; @Inject @@ -114,7 +114,7 @@ public QueryMonitor( this.serverAddress = requireNonNull(nodeInfo, "nodeInfo is null").getExternalAddress(); this.environment = requireNonNull(nodeInfo, "nodeInfo is null").getEnvironment(); this.sessionPropertyManager = requireNonNull(sessionPropertyManager, "sessionPropertyManager is null"); - this.functionManager = requireNonNull(metadata, "metadata is null").getFunctionManager(); + this.functionAndTypeManager = requireNonNull(metadata, "metadata is null").getFunctionAndTypeManager(); this.maxJsonLimit = toIntExact(requireNonNull(config, "config is null").getMaxOutputStageJsonSize().toBytes()); } @@ -328,7 +328,7 @@ private Optional createTextQueryPlan(QueryInfo queryInfo) if (queryInfo.getOutputStage().isPresent()) { return Optional.of(textDistributedPlan( queryInfo.getOutputStage().get(), - functionManager, + functionAndTypeManager, queryInfo.getSession().toSession(sessionPropertyManager), false)); } diff --git a/presto-main/src/main/java/com/facebook/presto/execution/AlterFunctionTask.java b/presto-main/src/main/java/com/facebook/presto/execution/AlterFunctionTask.java index 9bf93b7a8f25..f77ec159c475 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/AlterFunctionTask.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/AlterFunctionTask.java @@ -31,7 +31,7 @@ import java.util.List; import java.util.Optional; -import static com.facebook.presto.metadata.FunctionManager.qualifyFunctionName; +import static com.facebook.presto.metadata.FunctionAndTypeManager.qualifyFunctionName; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.util.concurrent.Futures.immediateFuture; import static java.util.Objects.requireNonNull; @@ -70,7 +70,7 @@ public ListenableFuture execute(AlterFunction statement, TransactionManager t statement.getCharacteristics().getNullCallClause() .map(com.facebook.presto.sql.tree.RoutineCharacteristics.NullCallClause::name) .map(NullCallClause::valueOf)); - metadata.getFunctionManager().alterFunction( + metadata.getFunctionAndTypeManager().alterFunction( functionName, statement.getParameterTypes().map(types -> types.stream() .map(TypeSignature::parseTypeSignature) diff --git a/presto-main/src/main/java/com/facebook/presto/execution/CreateFunctionTask.java b/presto-main/src/main/java/com/facebook/presto/execution/CreateFunctionTask.java index 122372588027..a4c5cbc30909 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/CreateFunctionTask.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/CreateFunctionTask.java @@ -42,7 +42,7 @@ import java.util.Optional; import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.metadata.FunctionManager.qualifyFunctionName; +import static com.facebook.presto.metadata.FunctionAndTypeManager.qualifyFunctionName; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static com.facebook.presto.sql.SqlFormatter.formatSql; import static com.google.common.collect.ImmutableList.toImmutableList; @@ -82,7 +82,7 @@ public ListenableFuture execute(CreateFunction statement, TransactionManager throw new PrestoException(NOT_SUPPORTED, "Invoking a dynamically registered function in SQL function body is not supported"); } - metadata.getFunctionManager().createFunction(createSqlInvokedFunction(statement, metadata, analysis), statement.isReplace()); + metadata.getFunctionAndTypeManager().createFunction(createSqlInvokedFunction(statement, metadata, analysis), statement.isReplace()); return immediateFuture(null); } diff --git a/presto-main/src/main/java/com/facebook/presto/execution/DropFunctionTask.java b/presto-main/src/main/java/com/facebook/presto/execution/DropFunctionTask.java index a6e4cf15de90..1203fe742183 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/DropFunctionTask.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/DropFunctionTask.java @@ -28,7 +28,7 @@ import java.util.List; import java.util.Optional; -import static com.facebook.presto.metadata.FunctionManager.qualifyFunctionName; +import static com.facebook.presto.metadata.FunctionAndTypeManager.qualifyFunctionName; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.util.concurrent.Futures.immediateFuture; import static java.util.Objects.requireNonNull; @@ -62,7 +62,7 @@ public ListenableFuture execute(DropFunction statement, TransactionManager tr Analyzer analyzer = new Analyzer(stateMachine.getSession(), metadata, sqlParser, accessControl, Optional.empty(), parameters, stateMachine.getWarningCollector()); analyzer.analyze(statement); - metadata.getFunctionManager().dropFunction( + metadata.getFunctionAndTypeManager().dropFunction( qualifyFunctionName(statement.getFunctionName()), statement.getParameterTypes().map(types -> types.stream().map(TypeSignature::parseTypeSignature).collect(toImmutableList())), statement.isExists()); diff --git a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java index e80742012769..658397009b77 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java @@ -480,7 +480,7 @@ private void planDistribution(PlanRoot plan) remoteTaskFactory, splitSourceFactory, stateMachine.getSession(), - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), stateMachine, outputStagePlan, rootOutputBuffers, @@ -505,7 +505,7 @@ private void planDistribution(PlanRoot plan) stateMachine, outputStagePlan, plan.isSummarizeTaskInfos(), - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), runtimePlanOptimizers, stateMachine.getWarningCollector(), idAllocator, diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/LegacySqlQueryScheduler.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/LegacySqlQueryScheduler.java index 0f9f4318bb36..7ffa4bb06559 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/LegacySqlQueryScheduler.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/LegacySqlQueryScheduler.java @@ -31,7 +31,7 @@ import com.facebook.presto.execution.TaskId; import com.facebook.presto.execution.buffer.OutputBuffers; import com.facebook.presto.execution.buffer.OutputBuffers.OutputBufferId; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.WarningCollector; @@ -121,7 +121,7 @@ public class LegacySqlQueryScheduler // The following fields are required by adaptive optimization in runtime. private final Session session; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final List runtimePlanOptimizers; private final WarningCollector warningCollector; private final PlanNodeIdAllocator idAllocator; @@ -148,7 +148,7 @@ public static LegacySqlQueryScheduler createSqlQueryScheduler( RemoteTaskFactory remoteTaskFactory, SplitSourceFactory splitSourceFactory, Session session, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, QueryStateMachine queryStateMachine, SubPlan plan, OutputBuffers rootOutputBuffers, @@ -170,7 +170,7 @@ public static LegacySqlQueryScheduler createSqlQueryScheduler( remoteTaskFactory, splitSourceFactory, session, - functionManager, + functionAndTypeManager, queryStateMachine, plan, summarizeTaskInfo, @@ -195,7 +195,7 @@ private LegacySqlQueryScheduler( RemoteTaskFactory remoteTaskFactory, SplitSourceFactory splitSourceFactory, Session session, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, QueryStateMachine queryStateMachine, SubPlan plan, boolean summarizeTaskInfo, @@ -215,7 +215,7 @@ private LegacySqlQueryScheduler( this.queryStateMachine = requireNonNull(queryStateMachine, "queryStateMachine is null"); this.plan.compareAndSet(null, requireNonNull(plan, "plan is null")); this.session = requireNonNull(session, "session is null"); - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); this.runtimePlanOptimizers = requireNonNull(runtimePlanOptimizers, "runtimePlanOptimizers is null"); this.warningCollector = requireNonNull(warningCollector, "warningCollector is null"); this.idAllocator = requireNonNull(idAllocator, "idAllocator is null"); @@ -601,7 +601,7 @@ private Optional performRuntimeOptimizations(StreamingSubPlan subP fragment.getStageExecutionDescriptor(), fragment.isOutputTableWriterFragment(), fragment.getStatsAndCosts(), - Optional.of(jsonFragmentPlan(newRoot, fragment.getVariables(), functionManager, session)))); + Optional.of(jsonFragmentPlan(newRoot, fragment.getVariables(), functionAndTypeManager, session)))); } return Optional.empty(); } diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SqlQueryScheduler.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SqlQueryScheduler.java index d513bcda673f..aea3a5438166 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SqlQueryScheduler.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/SqlQueryScheduler.java @@ -32,7 +32,7 @@ import com.facebook.presto.execution.TaskId; import com.facebook.presto.execution.buffer.OutputBuffers; import com.facebook.presto.execution.buffer.OutputBuffers.OutputBufferId; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.InternalNodeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.PrestoException; @@ -130,7 +130,7 @@ public class SqlQueryScheduler private final AtomicReference plan = new AtomicReference<>(); // The following fields are required for adaptive optimization in runtime. - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final List runtimePlanOptimizers; private final WarningCollector warningCollector; private final PlanNodeIdAllocator idAllocator; @@ -163,7 +163,7 @@ public static SqlQueryScheduler createSqlQueryScheduler( QueryStateMachine queryStateMachine, SubPlan plan, boolean summarizeTaskInfo, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, List runtimePlanOptimizers, WarningCollector warningCollector, PlanNodeIdAllocator idAllocator, @@ -185,7 +185,7 @@ public static SqlQueryScheduler createSqlQueryScheduler( queryStateMachine, plan, summarizeTaskInfo, - functionManager, + functionAndTypeManager, runtimePlanOptimizers, warningCollector, idAllocator, @@ -210,7 +210,7 @@ private SqlQueryScheduler( QueryStateMachine queryStateMachine, SubPlan plan, boolean summarizeTaskInfo, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, List runtimePlanOptimizers, WarningCollector warningCollector, PlanNodeIdAllocator idAllocator, @@ -229,7 +229,7 @@ private SqlQueryScheduler( this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); this.session = requireNonNull(session, "session is null"); this.queryStateMachine = requireNonNull(queryStateMachine, "queryStateMachine is null"); - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); this.runtimePlanOptimizers = requireNonNull(runtimePlanOptimizers, "runtimePlanOptimizers is null"); this.warningCollector = requireNonNull(warningCollector, "warningCollector is null"); this.idAllocator = requireNonNull(idAllocator, "idAllocator is null"); @@ -489,7 +489,7 @@ private Optional performRuntimeOptimizations(StreamingSubPlan subP fragment.getStageExecutionDescriptor(), fragment.isOutputTableWriterFragment(), fragment.getStatsAndCosts(), - Optional.of(jsonFragmentPlan(newRoot, fragment.getVariables(), functionManager, session)))); + Optional.of(jsonFragmentPlan(newRoot, fragment.getVariables(), functionAndTypeManager, session)))); } return Optional.empty(); } diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/BuiltInFunctionNamespaceManager.java b/presto-main/src/main/java/com/facebook/presto/metadata/BuiltInFunctionNamespaceManager.java index 27d9127a3ffc..97e5c9092f4d 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/BuiltInFunctionNamespaceManager.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/BuiltInFunctionNamespaceManager.java @@ -403,7 +403,7 @@ public BuiltInFunctionNamespaceManager( TypeManager typeManager, BlockEncodingSerde blockEncodingSerde, FeaturesConfig featuresConfig, - FunctionManager functionManager) + FunctionAndTypeManager functionAndTypeManager) { this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.magicLiteralFunction = new MagicLiteralFunction(blockEncodingSerde); @@ -429,7 +429,7 @@ public BuiltInFunctionNamespaceManager( "Unsupported scalar function class: %s", key.getFunction().getClass()); return key.getFunction() instanceof SqlScalarFunction - ? ((SqlScalarFunction) key.getFunction()).specialize(key.getBoundVariables(), key.getArity(), typeManager, functionManager) + ? ((SqlScalarFunction) key.getFunction()).specialize(key.getBoundVariables(), key.getArity(), typeManager, functionAndTypeManager) : new SqlInvokedScalarFunctionImplementation(((SqlInvokedFunction) key.getFunction()).getBody()); })); @@ -437,7 +437,7 @@ public BuiltInFunctionNamespaceManager( .maximumSize(1000) .expireAfterWrite(1, HOURS) .build(CacheLoader.from(key -> ((SqlAggregationFunction) key.getFunction()) - .specialize(key.getBoundVariables(), key.getArity(), typeManager, functionManager))); + .specialize(key.getBoundVariables(), key.getArity(), typeManager, functionAndTypeManager))); specializedWindowCache = CacheBuilder.newBuilder() .maximumSize(1000) @@ -448,7 +448,7 @@ public BuiltInFunctionNamespaceManager( return supplier(key.getFunction().getSignature(), specializedAggregationCache.getUnchecked(key)); } return ((SqlWindowFunction) key.getFunction()) - .specialize(key.getBoundVariables(), key.getArity(), typeManager, functionManager); + .specialize(key.getBoundVariables(), key.getArity(), typeManager, functionAndTypeManager); })); FunctionListBuilder builder = new FunctionListBuilder() @@ -1063,7 +1063,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type parameterType = boundVariables.getTypeVariable("T"); Type type = boundVariables.getTypeVariable("R"); diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/FunctionManager.java b/presto-main/src/main/java/com/facebook/presto/metadata/FunctionAndTypeManager.java similarity index 93% rename from presto-main/src/main/java/com/facebook/presto/metadata/FunctionManager.java rename to presto-main/src/main/java/com/facebook/presto/metadata/FunctionAndTypeManager.java index 9807f1bd2034..9a6ee05bd690 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/FunctionManager.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/FunctionAndTypeManager.java @@ -21,9 +21,11 @@ import com.facebook.presto.common.block.BlockEncodingSerde; import com.facebook.presto.common.function.OperatorType; import com.facebook.presto.common.function.QualifiedFunctionName; +import com.facebook.presto.common.type.ParametricType; 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 com.facebook.presto.operator.aggregation.InternalAggregationFunction; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation; import com.facebook.presto.operator.window.WindowFunctionSupplier; @@ -103,11 +105,11 @@ import static java.util.Objects.requireNonNull; @ThreadSafe -public class FunctionManager - implements FunctionMetadataManager +public class FunctionAndTypeManager + implements FunctionMetadataManager, TypeManager { - private final TypeManager typeManager; private final TransactionManager transactionManager; + private final TypeRegistry builtInTypeRegistry; private final BuiltInFunctionNamespaceManager builtInFunctionNamespaceManager; private final FunctionInvokerProvider functionInvokerProvider; private final Map functionNamespaceManagerFactories = new ConcurrentHashMap<>(); @@ -117,22 +119,26 @@ public class FunctionManager private final CacheStatsMBean cacheStatsMBean; @Inject - public FunctionManager( + public FunctionAndTypeManager( TypeManager typeManager, TransactionManager transactionManager, BlockEncodingSerde blockEncodingSerde, FeaturesConfig featuresConfig, HandleResolver handleResolver) { - this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); this.builtInFunctionNamespaceManager = new BuiltInFunctionNamespaceManager(typeManager, blockEncodingSerde, featuresConfig, this); this.functionNamespaceManagers.put(DEFAULT_NAMESPACE.getCatalogName(), builtInFunctionNamespaceManager); this.functionInvokerProvider = new FunctionInvokerProvider(this); - this.handleResolver = handleResolver; + this.handleResolver = requireNonNull(handleResolver, "handleResolver is null"); + requireNonNull(typeManager, "typeManager is null"); if (typeManager instanceof TypeRegistry) { - ((TypeRegistry) typeManager).setFunctionManager(this); + this.builtInTypeRegistry = (TypeRegistry) typeManager; } + else { + this.builtInTypeRegistry = new TypeRegistry(); + } + builtInTypeRegistry.setFunctionManager(this); // TODO: Provide a more encapsulated way for TransactionManager to register FunctionNamespaceManager transactionManager.registerFunctionNamespaceManager(DEFAULT_NAMESPACE.getCatalogName(), builtInFunctionNamespaceManager); this.functionCache = CacheBuilder.newBuilder() @@ -143,7 +149,7 @@ public FunctionManager( } @VisibleForTesting - public FunctionManager(TypeManager typeManager, BlockEncodingSerde blockEncodingSerde, FeaturesConfig featuresConfig) + public FunctionAndTypeManager(TypeManager typeManager, BlockEncodingSerde blockEncodingSerde, FeaturesConfig featuresConfig) { // TODO: Convert this constructor to a function in the testing package this(typeManager, createTestTransactionManager(), blockEncodingSerde, featuresConfig, new HandleResolver()); @@ -280,52 +286,52 @@ public FunctionHandle resolveFunction(Optional transactionId, Qua return resolveFunctionInternal(transactionId, functionName, parameterTypes); } - private FunctionHandle resolveFunctionInternal(Optional transactionId, QualifiedFunctionName functionName, List parameterTypes) + @Override + public Type getType(TypeSignature signature) { - FunctionNamespaceManager functionNamespaceManager = getServingFunctionNamespaceManager(functionName.getFunctionNamespace()).orElse(null); - if (functionNamespaceManager == null) { - throw new PrestoException(FUNCTION_NOT_FOUND, constructFunctionNotFoundErrorMessage(functionName, parameterTypes, ImmutableList.of())); - } - - Optional transactionHandle = transactionId - .map(id -> transactionManager.getFunctionNamespaceTransaction(id, functionName.getFunctionNamespace().getCatalogName())); - Collection candidates = functionNamespaceManager.getFunctions(transactionHandle, functionName); - - try { - return lookupFunction(functionNamespaceManager, transactionHandle, functionName, parameterTypes, candidates); - } - catch (PrestoException e) { - if (e.getErrorCode().getCode() != FUNCTION_NOT_FOUND.toErrorCode().getCode()) { - throw e; - } - } + return builtInTypeRegistry.getType(signature); + } - Optional match = matchFunctionWithCoercion(candidates, parameterTypes); - if (match.isPresent()) { - return functionNamespaceManager.getFunctionHandle(transactionHandle, match.get()); - } + @Override + public Type getParameterizedType(String baseTypeName, List typeParameters) + { + return builtInTypeRegistry.getParameterizedType(baseTypeName, typeParameters); + } - if (functionName.getFunctionName().startsWith(MAGIC_LITERAL_FUNCTION_PREFIX)) { - // extract type from function functionName - String typeName = functionName.getFunctionName().substring(MAGIC_LITERAL_FUNCTION_PREFIX.length()); + @Override + public List getTypes() + { + return builtInTypeRegistry.getTypes(); + } - // lookup the type - Type type = typeManager.getType(parseTypeSignature(typeName)); + @Override + public Collection getParametricTypes() + { + return builtInTypeRegistry.getParametricTypes(); + } - // verify we have one parameter of the proper type - checkArgument(parameterTypes.size() == 1, "Expected one argument to literal function, but got %s", parameterTypes); + @Override + public Optional getCommonSuperType(Type firstType, Type secondType) + { + return builtInTypeRegistry.getCommonSuperType(firstType, secondType); + } - return new BuiltInFunctionHandle(getMagicLiteralFunctionSignature(type)); - } + @Override + public boolean canCoerce(Type actualType, Type expectedType) + { + return builtInTypeRegistry.canCoerce(actualType, expectedType); + } - throw new PrestoException(FUNCTION_NOT_FOUND, constructFunctionNotFoundErrorMessage(functionName, parameterTypes, candidates)); + @Override + public boolean isTypeOnlyCoercion(Type actualType, Type expectedType) + { + return builtInTypeRegistry.isTypeOnlyCoercion(actualType, expectedType); } - private FunctionHandle resolveBuiltInFunction(QualifiedFunctionName functionName, List parameterTypes) + @Override + public Optional coerceTypeBase(Type sourceType, String resultTypeBase) { - checkArgument(functionName.getFunctionNamespace().equals(DEFAULT_NAMESPACE), "Expect built-in functions"); - checkArgument(parameterTypes.stream().noneMatch(TypeSignatureProvider::hasDependency), "Expect parameter types not to have dependency"); - return resolveFunctionInternal(Optional.empty(), functionName, parameterTypes); + return builtInTypeRegistry.coerceTypeBase(sourceType, resultTypeBase); } @Override @@ -347,7 +353,7 @@ public CompletableFuture executeFunction(FunctionHandle functionHandle, P { Optional> functionNamespaceManager = getServingFunctionNamespaceManager(functionHandle.getFunctionNamespace()); checkState(functionNamespaceManager.isPresent(), format("FunctionHandle %s should have a serving function namespace", functionHandle)); - return functionNamespaceManager.get().executeFunction(functionHandle, inputPage, channels, typeManager); + return functionNamespaceManager.get().executeFunction(functionHandle, inputPage, channels, this); } public WindowFunctionSupplier getWindowFunctionImplementation(FunctionHandle functionHandle) @@ -428,6 +434,54 @@ public FunctionHandle lookupCast(CastType castType, TypeSignature fromType, Type return builtInFunctionNamespaceManager.getFunctionHandle(Optional.empty(), signature); } + private FunctionHandle resolveFunctionInternal(Optional transactionId, QualifiedFunctionName functionName, List parameterTypes) + { + FunctionNamespaceManager functionNamespaceManager = getServingFunctionNamespaceManager(functionName.getFunctionNamespace()).orElse(null); + if (functionNamespaceManager == null) { + throw new PrestoException(FUNCTION_NOT_FOUND, constructFunctionNotFoundErrorMessage(functionName, parameterTypes, ImmutableList.of())); + } + + Optional transactionHandle = transactionId + .map(id -> transactionManager.getFunctionNamespaceTransaction(id, functionName.getFunctionNamespace().getCatalogName())); + Collection candidates = functionNamespaceManager.getFunctions(transactionHandle, functionName); + + try { + return lookupFunction(functionNamespaceManager, transactionHandle, functionName, parameterTypes, candidates); + } + catch (PrestoException e) { + if (e.getErrorCode().getCode() != FUNCTION_NOT_FOUND.toErrorCode().getCode()) { + throw e; + } + } + + Optional match = matchFunctionWithCoercion(candidates, parameterTypes); + if (match.isPresent()) { + return functionNamespaceManager.getFunctionHandle(transactionHandle, match.get()); + } + + if (functionName.getFunctionName().startsWith(MAGIC_LITERAL_FUNCTION_PREFIX)) { + // extract type from function functionName + String typeName = functionName.getFunctionName().substring(MAGIC_LITERAL_FUNCTION_PREFIX.length()); + + // lookup the type + Type type = getType(parseTypeSignature(typeName)); + + // verify we have one parameter of the proper type + checkArgument(parameterTypes.size() == 1, "Expected one argument to literal function, but got %s", parameterTypes); + + return new BuiltInFunctionHandle(getMagicLiteralFunctionSignature(type)); + } + + throw new PrestoException(FUNCTION_NOT_FOUND, constructFunctionNotFoundErrorMessage(functionName, parameterTypes, candidates)); + } + + private FunctionHandle resolveBuiltInFunction(QualifiedFunctionName functionName, List parameterTypes) + { + checkArgument(functionName.getFunctionNamespace().equals(DEFAULT_NAMESPACE), "Expect built-in functions"); + checkArgument(parameterTypes.stream().noneMatch(TypeSignatureProvider::hasDependency), "Expect parameter types not to have dependency"); + return resolveFunctionInternal(Optional.empty(), functionName, parameterTypes); + } + private FunctionHandle lookupCachedFunction(QualifiedFunctionName functionName, List parameterTypes) { try { @@ -543,7 +597,7 @@ private List identifyApplicableFunctions(Collection applicableFunctions = ImmutableList.builder(); for (SqlFunction function : candidates) { Signature declaredSignature = function.getSignature(); - Optional boundSignature = new SignatureBinder(typeManager, declaredSignature, allowCoercion) + Optional boundSignature = new SignatureBinder(this, declaredSignature, allowCoercion) .bind(actualParameters); if (boundSignature.isPresent()) { applicableFunctions.add(new ApplicableFunction(declaredSignature, boundSignature.get(), function.isCalledOnNullInput())); @@ -561,7 +615,7 @@ private List selectMostSpecificFunctions(List> optionalParameterTypes = toTypes(parameters, typeManager); + Optional> optionalParameterTypes = toTypes(parameters, this); if (!optionalParameterTypes.isPresent()) { // give up and return all remaining matches return mostSpecificFunctions; @@ -635,7 +689,7 @@ private List getUnknownOnlyCastFunctions(List actualParameters) { - List boundTypes = resolveTypes(applicableFunction.getBoundSignature().getArgumentTypes(), typeManager); + List boundTypes = resolveTypes(applicableFunction.getBoundSignature().getArgumentTypes(), this); checkState(actualParameters.size() == boundTypes.size(), "type lists are of different lengths"); for (int i = 0; i < actualParameters.size(); i++) { if (!boundTypes.get(i).equals(actualParameters.get(i)) && actualParameters.get(i) != UNKNOWN) { @@ -648,7 +702,7 @@ private boolean onlyCastsUnknown(ApplicableFunction applicableFunction, List applicableFunctions) { Set returnTypes = applicableFunctions.stream() - .map(function -> typeManager.getType(function.getBoundSignature().getReturnType())) + .map(function -> getType(function.getBoundSignature().getReturnType())) .collect(Collectors.toSet()); return returnTypes.size() == 1; } @@ -699,7 +753,7 @@ private static Optional> toTypes(List typeSign private boolean isMoreSpecificThan(ApplicableFunction left, ApplicableFunction right) { List resolvedTypes = fromTypeSignatures(left.getBoundSignature().getArgumentTypes()); - Optional boundVariables = new SignatureBinder(typeManager, right.getDeclaredSignature(), true) + Optional boundVariables = new SignatureBinder(this, right.getDeclaredSignature(), true) .bindVariables(resolvedTypes); return boundVariables.isPresent(); } diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/FunctionInvokerProvider.java b/presto-main/src/main/java/com/facebook/presto/metadata/FunctionInvokerProvider.java index e177b39d0844..911bfbcfb29f 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/FunctionInvokerProvider.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/FunctionInvokerProvider.java @@ -38,16 +38,16 @@ public class FunctionInvokerProvider { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public FunctionInvokerProvider(FunctionManager functionManager) + public FunctionInvokerProvider(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = functionManager; + this.functionAndTypeManager = functionAndTypeManager; } public FunctionInvoker createFunctionInvoker(FunctionHandle functionHandle, Optional invocationConvention) { - BuiltInScalarFunctionImplementation builtInScalarFunctionImplementation = functionManager.getBuiltInScalarFunctionImplementation(functionHandle); + BuiltInScalarFunctionImplementation builtInScalarFunctionImplementation = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionHandle); for (ScalarImplementationChoice choice : builtInScalarFunctionImplementation.getAllChoices()) { if (checkChoice(choice.getArgumentProperties(), choice.isNullable(), choice.hasProperties(), invocationConvention)) { return new FunctionInvoker(choice.getMethodHandle()); diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/Metadata.java b/presto-main/src/main/java/com/facebook/presto/metadata/Metadata.java index ffa85d7637e2..6b87a2adc62f 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/Metadata.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/Metadata.java @@ -438,7 +438,7 @@ public interface Metadata MetadataUpdates getMetadataUpdateResults(Session session, QueryManager queryManager, MetadataUpdates metadataUpdates, QueryId queryId); - FunctionManager getFunctionManager(); + FunctionAndTypeManager getFunctionAndTypeManager(); ProcedureRegistry getProcedureRegistry(); 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 c0d4031e2e63..09b57bcf1876 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 @@ -127,7 +127,7 @@ public class MetadataManager { private static final Logger log = Logger.get(MetadataManager.class); - private final FunctionManager functions; + private final FunctionAndTypeManager functionAndTypeManager; private final ProcedureRegistry procedures; private final TypeManager typeManager; private final JsonCodec viewCodec; @@ -163,7 +163,7 @@ public MetadataManager( columnPropertyManager, analyzePropertyManager, transactionManager, - new FunctionManager(typeManager, transactionManager, blockEncodingSerde, featuresConfig, new HandleResolver())); + new FunctionAndTypeManager(typeManager, transactionManager, blockEncodingSerde, featuresConfig, new HandleResolver())); } @Inject @@ -177,7 +177,7 @@ public MetadataManager( ColumnPropertyManager columnPropertyManager, AnalyzePropertyManager analyzePropertyManager, TransactionManager transactionManager, - FunctionManager functionManager) + FunctionAndTypeManager functionAndTypeManager) { procedures = new ProcedureRegistry(typeManager); this.typeManager = requireNonNull(typeManager, "types is null"); @@ -189,7 +189,7 @@ public MetadataManager( this.columnPropertyManager = requireNonNull(columnPropertyManager, "columnPropertyManager is null"); this.analyzePropertyManager = requireNonNull(analyzePropertyManager, "analyzePropertyManager is null"); this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); - this.functions = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); verifyComparableOrderableContract(); } @@ -275,13 +275,13 @@ public Type getType(TypeSignature signature) public List listFunctions(Session session) { // TODO: transactional when FunctionManager is made transactional - return functions.listFunctions(session); + return functionAndTypeManager.listFunctions(session); } @Override public void registerBuiltInFunctions(List functionInfos) { - functions.registerBuiltInFunctions(functionInfos); + functionAndTypeManager.registerBuiltInFunctions(functionInfos); } @Override @@ -1236,10 +1236,10 @@ public MetadataUpdates getMetadataUpdateResults(Session session, QueryManager qu } @Override - public FunctionManager getFunctionManager() + public FunctionAndTypeManager getFunctionAndTypeManager() { // TODO: transactional when FunctionManager is made transactional - return functions; + return functionAndTypeManager; } @Override @@ -1362,7 +1362,7 @@ private static JsonCodec createTestingViewCodec() private boolean canResolveOperator(OperatorType operatorType, List argumentTypes) { try { - getFunctionManager().resolveOperator(operatorType, argumentTypes); + getFunctionAndTypeManager().resolveOperator(operatorType, argumentTypes); return true; } catch (OperatorNotFoundException e) { diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunction.java b/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunction.java index 6bfa6f9c6e25..f937cf1643f9 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunction.java @@ -93,12 +93,12 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { ImmutableList.Builder implementationChoices = ImmutableList.builder(); for (PolymorphicScalarFunctionChoice choice : choices) { - implementationChoices.add(getScalarFunctionImplementationChoice(boundVariables, typeManager, functionManager, choice)); + implementationChoices.add(getScalarFunctionImplementationChoice(boundVariables, typeManager, functionAndTypeManager, choice)); } return new BuiltInScalarFunctionImplementation(implementationChoices.build()); @@ -107,12 +107,12 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab private ScalarImplementationChoice getScalarFunctionImplementationChoice( BoundVariables boundVariables, TypeManager typeManager, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, PolymorphicScalarFunctionChoice choice) { List resolvedParameterTypes = applyBoundVariables(typeManager, getSignature().getArgumentTypes(), boundVariables); Type resolvedReturnType = applyBoundVariables(typeManager, getSignature().getReturnType(), boundVariables); - SpecializeContext context = new SpecializeContext(boundVariables, resolvedParameterTypes, resolvedReturnType, typeManager, functionManager); + SpecializeContext context = new SpecializeContext(boundVariables, resolvedParameterTypes, resolvedReturnType, typeManager, functionAndTypeManager); Optional matchingMethod = Optional.empty(); Optional matchingMethodsGroup = Optional.empty(); diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunctionBuilder.java b/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunctionBuilder.java index 143f3a1eb500..d029714465fe 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunctionBuilder.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunctionBuilder.java @@ -152,15 +152,15 @@ public static final class SpecializeContext private final List parameterTypes; private final Type returnType; private final TypeManager typeManager; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - SpecializeContext(BoundVariables boundVariables, List parameterTypes, Type returnType, TypeManager typeManager, FunctionManager functionManager) + SpecializeContext(BoundVariables boundVariables, List parameterTypes, Type returnType, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { this.boundVariables = requireNonNull(boundVariables, "boundVariables is null"); this.parameterTypes = requireNonNull(parameterTypes, "parameterTypes is null"); this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.returnType = requireNonNull(returnType, "returnType is null"); - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } public Type getType(String name) @@ -188,9 +188,9 @@ public TypeManager getTypeManager() return typeManager; } - public FunctionManager getFunctionManager() + public FunctionAndTypeManager getFunctionManager() { - return functionManager; + return functionAndTypeManager; } } diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/SqlAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/metadata/SqlAggregationFunction.java index 761304ae640d..15a733fd371b 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/SqlAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/SqlAggregationFunction.java @@ -122,5 +122,5 @@ public boolean isDeterministic() return true; } - public abstract InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager); + public abstract InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager); } diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/SqlScalarFunction.java b/presto-main/src/main/java/com/facebook/presto/metadata/SqlScalarFunction.java index b9e4356446f4..c9690043859a 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/SqlScalarFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/SqlScalarFunction.java @@ -39,7 +39,7 @@ public final Signature getSignature() return signature; } - public abstract BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager); + public abstract BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager); public static PolymorphicScalarFunctionBuilder builder(Class clazz, OperatorType operatorType) { diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/StaticFunctionNamespaceStore.java b/presto-main/src/main/java/com/facebook/presto/metadata/StaticFunctionNamespaceStore.java index 0ec39dc14159..17bae9ff84a7 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/StaticFunctionNamespaceStore.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/StaticFunctionNamespaceStore.java @@ -34,14 +34,14 @@ public class StaticFunctionNamespaceStore private static final Logger log = Logger.get(StaticFunctionNamespaceStore.class); private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final File configDir; private final AtomicBoolean functionNamespaceLoading = new AtomicBoolean(); @Inject - public StaticFunctionNamespaceStore(FunctionManager functionManager, StaticFunctionNamespaceStoreConfig config) + public StaticFunctionNamespaceStore(FunctionAndTypeManager functionAndTypeManager, StaticFunctionNamespaceStoreConfig config) { - this.functionManager = functionManager; + this.functionAndTypeManager = functionAndTypeManager; this.configDir = config.getFunctionNamespaceConfigurationDir(); } @@ -69,7 +69,7 @@ private void loadFunctionNamespaceManager(File file) String functionNamespaceManagerName = properties.remove("function-namespace-manager.name"); checkState(functionNamespaceManagerName != null, "Function namespace configuration %s does not contain function-namespace-manager.name", file.getAbsoluteFile()); - functionManager.loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties); + functionAndTypeManager.loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties); log.info("-- Added function namespace manager [%s] --", catalogName); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/ExplainAnalyzeOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/ExplainAnalyzeOperator.java index e8e1b20b17e0..6203e823d194 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/ExplainAnalyzeOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/ExplainAnalyzeOperator.java @@ -19,7 +19,7 @@ import com.facebook.presto.execution.QueryPerformanceFetcher; import com.facebook.presto.execution.StageId; import com.facebook.presto.execution.StageInfo; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.plan.PlanNodeId; import com.google.common.collect.ImmutableList; @@ -40,7 +40,7 @@ public static class ExplainAnalyzeOperatorFactory private final int operatorId; private final PlanNodeId planNodeId; private final QueryPerformanceFetcher queryPerformanceFetcher; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final boolean verbose; private boolean closed; @@ -48,13 +48,13 @@ public ExplainAnalyzeOperatorFactory( int operatorId, PlanNodeId planNodeId, QueryPerformanceFetcher queryPerformanceFetcher, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, boolean verbose) { this.operatorId = operatorId; this.planNodeId = requireNonNull(planNodeId, "planNodeId is null"); this.queryPerformanceFetcher = requireNonNull(queryPerformanceFetcher, "queryPerformanceFetcher is null"); - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); this.verbose = verbose; } @@ -63,7 +63,7 @@ public Operator createOperator(DriverContext driverContext) { checkState(!closed, "Factory is already closed"); OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, ExplainAnalyzeOperator.class.getSimpleName()); - return new ExplainAnalyzeOperator(operatorContext, queryPerformanceFetcher, functionManager, verbose); + return new ExplainAnalyzeOperator(operatorContext, queryPerformanceFetcher, functionAndTypeManager, verbose); } @Override @@ -75,13 +75,13 @@ public void noMoreOperators() @Override public OperatorFactory duplicate() { - return new ExplainAnalyzeOperatorFactory(operatorId, planNodeId, queryPerformanceFetcher, functionManager, verbose); + return new ExplainAnalyzeOperatorFactory(operatorId, planNodeId, queryPerformanceFetcher, functionAndTypeManager, verbose); } } private final OperatorContext operatorContext; private final QueryPerformanceFetcher queryPerformanceFetcher; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final boolean verbose; private boolean finishing; private boolean outputConsumed; @@ -89,12 +89,12 @@ public OperatorFactory duplicate() public ExplainAnalyzeOperator( OperatorContext operatorContext, QueryPerformanceFetcher queryPerformanceFetcher, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, boolean verbose) { this.operatorContext = requireNonNull(operatorContext, "operatorContext is null"); this.queryPerformanceFetcher = requireNonNull(queryPerformanceFetcher, "queryPerformanceFetcher is null"); - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); this.verbose = verbose; } @@ -145,7 +145,7 @@ public Page getOutput() return null; } - String plan = textDistributedPlan(queryInfo.getOutputStage().get().getSubStages().get(0), functionManager, operatorContext.getSession(), verbose); + String plan = textDistributedPlan(queryInfo.getOutputStage().get().getSubStages().get(0), functionAndTypeManager, operatorContext.getSession(), verbose); BlockBuilder builder = VARCHAR.createBlockBuilder(null, 1); VARCHAR.writeString(builder, plan); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/PagesIndex.java b/presto-main/src/main/java/com/facebook/presto/operator/PagesIndex.java index ac4d9ba7b6f7..0beb4e62518d 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/PagesIndex.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/PagesIndex.java @@ -23,7 +23,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.geospatial.Rectangle; import com.facebook.presto.memory.context.LocalMemoryContext; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.SpatialIndexBuilderOperator.SpatialPredicate; @@ -80,7 +80,7 @@ public class PagesIndex private final OrderingCompiler orderingCompiler; private final JoinCompiler joinCompiler; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final boolean groupByUsesEqualTo; private final List types; @@ -96,7 +96,7 @@ public class PagesIndex private PagesIndex( OrderingCompiler orderingCompiler, JoinCompiler joinCompiler, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, boolean groupByUsesEqualTo, List types, int expectedPositions, @@ -104,7 +104,7 @@ private PagesIndex( { this.orderingCompiler = requireNonNull(orderingCompiler, "orderingCompiler is null"); this.joinCompiler = requireNonNull(joinCompiler, "joinCompiler is null"); - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); this.groupByUsesEqualTo = groupByUsesEqualTo; this.types = ImmutableList.copyOf(requireNonNull(types, "types is null")); this.valueAddresses = new LongArrayList(expectedPositions); @@ -140,7 +140,7 @@ public TestingFactory(boolean eagerCompact) @Override public PagesIndex newPagesIndex(List types, int expectedPositions) { - return new PagesIndex(ORDERING_COMPILER, JOIN_COMPILER, MetadataManager.createTestMetadataManager().getFunctionManager(), groupByUsesEqualTo, types, expectedPositions, eagerCompact); + return new PagesIndex(ORDERING_COMPILER, JOIN_COMPILER, MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(), groupByUsesEqualTo, types, expectedPositions, eagerCompact); } } @@ -150,7 +150,7 @@ public static class DefaultFactory private final OrderingCompiler orderingCompiler; private final JoinCompiler joinCompiler; private final boolean eagerCompact; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final boolean groupByUsesEqualTo; @Inject @@ -159,14 +159,14 @@ public DefaultFactory(OrderingCompiler orderingCompiler, JoinCompiler joinCompil this.orderingCompiler = requireNonNull(orderingCompiler, "orderingCompiler is null"); this.joinCompiler = requireNonNull(joinCompiler, "joinCompiler is null"); this.eagerCompact = requireNonNull(featuresConfig, "featuresConfig is null").isPagesIndexEagerCompactionEnabled(); - this.functionManager = requireNonNull(metadata, "metadata is null").getFunctionManager(); + this.functionAndTypeManager = requireNonNull(metadata, "metadata is null").getFunctionAndTypeManager(); this.groupByUsesEqualTo = featuresConfig.isGroupByUsesEqualTo(); } @Override public PagesIndex newPagesIndex(List types, int expectedPositions) { - return new PagesIndex(orderingCompiler, joinCompiler, functionManager, groupByUsesEqualTo, types, expectedPositions, eagerCompact); + return new PagesIndex(orderingCompiler, joinCompiler, functionAndTypeManager, groupByUsesEqualTo, types, expectedPositions, eagerCompact); } } @@ -442,7 +442,7 @@ public PagesHashStrategy createPagesHashStrategy(List joinChannels, Opt joinChannels, hashChannel, Optional.empty(), - functionManager, + functionAndTypeManager, groupByUsesEqualTo); } @@ -512,7 +512,7 @@ public LookupSourceSupplier createLookupSourceSupplier( joinChannels, hashChannel, sortChannel, - functionManager, + functionAndTypeManager, groupByUsesEqualTo); return new JoinHashSupplier( diff --git a/presto-main/src/main/java/com/facebook/presto/operator/ParametricFunctionHelpers.java b/presto-main/src/main/java/com/facebook/presto/operator/ParametricFunctionHelpers.java index 5c7b7cb5c313..b40d8a103508 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/ParametricFunctionHelpers.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/ParametricFunctionHelpers.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.annotations.ImplementationDependency; import java.lang.invoke.MethodHandle; @@ -26,10 +26,10 @@ public class ParametricFunctionHelpers { private ParametricFunctionHelpers() {} - public static MethodHandle bindDependencies(MethodHandle handle, List dependencies, BoundVariables variables, TypeManager typeManager, FunctionManager functionManager) + public static MethodHandle bindDependencies(MethodHandle handle, List dependencies, BoundVariables variables, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { for (ImplementationDependency dependency : dependencies) { - handle = MethodHandles.insertArguments(handle, 0, dependency.resolve(variables, typeManager, functionManager)); + handle = MethodHandles.insertArguments(handle, 0, dependency.resolve(variables, typeManager, functionAndTypeManager)); } return handle; } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/RemoteProjectOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/RemoteProjectOperator.java index 4a04de162cfe..f431b9939d26 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/RemoteProjectOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/RemoteProjectOperator.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.Page; import com.facebook.presto.common.block.Block; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.plan.PlanNodeId; import com.facebook.presto.spi.relation.CallExpression; import com.facebook.presto.spi.relation.ConstantExpression; @@ -40,17 +40,17 @@ public class RemoteProjectOperator implements Operator { private final OperatorContext operatorContext; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final List projections; private final CompletableFuture[] result; private boolean finishing; - private RemoteProjectOperator(OperatorContext operatorContext, FunctionManager functionManager, List projections) + private RemoteProjectOperator(OperatorContext operatorContext, FunctionAndTypeManager functionAndTypeManager, List projections) { this.operatorContext = requireNonNull(operatorContext, "operatorContext is null"); - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); this.projections = ImmutableList.copyOf(requireNonNull(projections, "projections is null")); this.result = new CompletableFuture[projections.size()]; } @@ -80,7 +80,7 @@ public void addInput(Page page) } else if (projection instanceof CallExpression) { CallExpression remoteCall = (CallExpression) projection; - result[channel] = functionManager.executeFunction( + result[channel] = functionAndTypeManager.executeFunction( remoteCall.getFunctionHandle(), page, remoteCall.getArguments().stream() @@ -163,15 +163,15 @@ public static class RemoteProjectOperatorFactory { private final int operatorId; private final PlanNodeId planNodeId; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final List projections; private boolean closed; - public RemoteProjectOperatorFactory(int operatorId, PlanNodeId planNodeId, FunctionManager functionManager, List projections) + public RemoteProjectOperatorFactory(int operatorId, PlanNodeId planNodeId, FunctionAndTypeManager functionAndTypeManager, List projections) { this.operatorId = operatorId; this.planNodeId = requireNonNull(planNodeId, "planNodeId is null"); - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); this.projections = ImmutableList.copyOf(requireNonNull(projections, "projections is null")); } @Override @@ -179,7 +179,7 @@ public Operator createOperator(DriverContext driverContext) { checkState(!closed, "Factory is already closed"); OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, RemoteProjectOperator.class.getSimpleName()); - return new RemoteProjectOperator(operatorContext, functionManager, projections); + return new RemoteProjectOperator(operatorContext, functionAndTypeManager, projections); } @Override @@ -191,7 +191,7 @@ public void noMoreOperators() @Override public OperatorFactory duplicate() { - return new RemoteProjectOperatorFactory(operatorId, planNodeId, functionManager, projections); + return new RemoteProjectOperatorFactory(operatorId, planNodeId, functionAndTypeManager, projections); } } } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/SimplePagesHashStrategy.java b/presto-main/src/main/java/com/facebook/presto/operator/SimplePagesHashStrategy.java index ea28bb3bf89f..939875e746ad 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/SimplePagesHashStrategy.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/SimplePagesHashStrategy.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.PageBuilder; import com.facebook.presto.common.block.Block; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.type.TypeUtils; import com.google.common.collect.ImmutableList; @@ -57,7 +57,7 @@ public SimplePagesHashStrategy( List hashChannels, OptionalInt precomputedHashChannel, Optional sortChannel, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, boolean groupByUsesEqualTo) { this.types = ImmutableList.copyOf(requireNonNull(types, "types is null")); @@ -73,12 +73,12 @@ public SimplePagesHashStrategy( this.precomputedHashChannel = null; } this.sortChannel = requireNonNull(sortChannel, "sortChannel is null"); - requireNonNull(functionManager, "functionManager is null"); + requireNonNull(functionAndTypeManager, "functionManager is null"); this.groupByUsesEqualTo = groupByUsesEqualTo; ImmutableList.Builder distinctFromMethodHandlesBuilder = ImmutableList.builder(); for (Type type : types) { distinctFromMethodHandlesBuilder.add( - functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(IS_DISTINCT_FROM, fromTypes(type, type))).getMethodHandle()); + functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(IS_DISTINCT_FROM, fromTypes(type, type))).getMethodHandle()); } distinctFromMethodHandles = distinctFromMethodHandlesBuilder.build(); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxAggregationFunction.java index 185c4fe1cbca..d6379ef51618 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxAggregationFunction.java @@ -20,7 +20,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlAggregationFunction; import com.facebook.presto.operator.aggregation.AggregationMetadata.AccumulatorStateDescriptor; import com.facebook.presto.operator.aggregation.state.BlockPositionState; @@ -87,11 +87,11 @@ protected AbstractMinMaxAggregationFunction(String name, boolean min) } @Override - public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type type = boundVariables.getTypeVariable("E"); - MethodHandle compareMethodHandle = functionManager.getBuiltInScalarFunctionImplementation( - functionManager.resolveOperator(operatorType, fromTypes(type, type))).getMethodHandle(); + MethodHandle compareMethodHandle = functionAndTypeManager.getBuiltInScalarFunctionImplementation( + functionAndTypeManager.resolveOperator(operatorType, fromTypes(type, type))).getMethodHandle(); return generateAggregation(type, compareMethodHandle); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxNAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxNAggregationFunction.java index 572269c827ec..fb0e465d68fd 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxNAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxNAggregationFunction.java @@ -21,7 +21,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlAggregationFunction; import com.facebook.presto.operator.aggregation.AggregationMetadata.AccumulatorStateDescriptor; import com.facebook.presto.operator.aggregation.state.MinMaxNState; @@ -72,7 +72,7 @@ protected AbstractMinMaxNAggregationFunction(String name, Function inputTypes = boundSignature.getArgumentTypes().stream().map(typeManager::getType).collect(toImmutableList()); @@ -82,13 +82,13 @@ public InternalAggregationFunction specialize(BoundVariables variables, int arit // Build state factory and serializer Class stateClass = concreteImplementation.getStateClass(); - AccumulatorStateSerializer stateSerializer = getAccumulatorStateSerializer(concreteImplementation, variables, typeManager, functionManager, stateClass, classLoader); + AccumulatorStateSerializer stateSerializer = getAccumulatorStateSerializer(concreteImplementation, variables, typeManager, functionAndTypeManager, stateClass, classLoader); AccumulatorStateFactory stateFactory = StateCompiler.generateStateFactory(stateClass, classLoader); // Bind provided dependencies to aggregation method handlers - MethodHandle inputHandle = bindDependencies(concreteImplementation.getInputFunction(), concreteImplementation.getInputDependencies(), variables, typeManager, functionManager); - MethodHandle combineHandle = bindDependencies(concreteImplementation.getCombineFunction(), concreteImplementation.getCombineDependencies(), variables, typeManager, functionManager); - MethodHandle outputHandle = bindDependencies(concreteImplementation.getOutputFunction(), concreteImplementation.getOutputDependencies(), variables, typeManager, functionManager); + MethodHandle inputHandle = bindDependencies(concreteImplementation.getInputFunction(), concreteImplementation.getInputDependencies(), variables, typeManager, functionAndTypeManager); + MethodHandle combineHandle = bindDependencies(concreteImplementation.getCombineFunction(), concreteImplementation.getCombineDependencies(), variables, typeManager, functionAndTypeManager); + MethodHandle outputHandle = bindDependencies(concreteImplementation.getOutputFunction(), concreteImplementation.getOutputDependencies(), variables, typeManager, functionAndTypeManager); // Build metadata of input parameters List parametersMetadata = buildParameterMetadata(concreteImplementation.getInputParameterMetadataTypes(), inputTypes); @@ -131,7 +131,7 @@ public String getDescription() return details.getDescription().orElse(""); } - private AggregationImplementation findMatchingImplementation(Signature boundSignature, BoundVariables variables, TypeManager typeManager, FunctionManager functionManager) + private AggregationImplementation findMatchingImplementation(Signature boundSignature, BoundVariables variables, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Optional foundImplementation = Optional.empty(); if (implementations.getExactImplementations().containsKey(boundSignature)) { @@ -154,13 +154,13 @@ private AggregationImplementation findMatchingImplementation(Signature boundSign return foundImplementation.get(); } - private static AccumulatorStateSerializer getAccumulatorStateSerializer(AggregationImplementation implementation, BoundVariables variables, TypeManager typeManager, FunctionManager functionManager, Class stateClass, DynamicClassLoader classLoader) + private static AccumulatorStateSerializer getAccumulatorStateSerializer(AggregationImplementation implementation, BoundVariables variables, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager, Class stateClass, DynamicClassLoader classLoader) { AccumulatorStateSerializer stateSerializer; Optional stateSerializerFactory = implementation.getStateSerializerFactory(); if (stateSerializerFactory.isPresent()) { try { - MethodHandle factoryHandle = bindDependencies(stateSerializerFactory.get(), implementation.getStateSerializerFactoryDependencies(), variables, typeManager, functionManager); + MethodHandle factoryHandle = bindDependencies(stateSerializerFactory.get(), implementation.getStateSerializerFactoryDependencies(), variables, typeManager, functionAndTypeManager); stateSerializer = (AccumulatorStateSerializer) factoryHandle.invoke(); } catch (Throwable t) { diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/RealAverageAggregation.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/RealAverageAggregation.java index 4d8ad493be17..3e8e4cc6d95a 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/RealAverageAggregation.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/RealAverageAggregation.java @@ -19,7 +19,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlAggregationFunction; import com.facebook.presto.operator.aggregation.AggregationMetadata.AccumulatorStateDescriptor; import com.facebook.presto.operator.aggregation.state.DoubleState; @@ -70,7 +70,7 @@ public String getDescription() } @Override - public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { DynamicClassLoader classLoader = new DynamicClassLoader(AverageAggregations.class.getClassLoader()); Class longStateInterface = LongState.class; diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ReduceAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ReduceAggregationFunction.java index 2cd8b4067905..4e6018bd4d12 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ReduceAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ReduceAggregationFunction.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlAggregationFunction; import com.facebook.presto.operator.aggregation.AggregationMetadata.AccumulatorStateDescriptor; import com.facebook.presto.operator.aggregation.AggregationMetadata.ParameterMetadata; @@ -80,7 +80,7 @@ public String getDescription() } @Override - public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type inputType = boundVariables.getTypeVariable("T"); Type stateType = boundVariables.getTypeVariable("S"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/StatisticalDigestAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/StatisticalDigestAggregationFunction.java index 9283a18bdb25..a6f0317f09ae 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/StatisticalDigestAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/StatisticalDigestAggregationFunction.java @@ -22,7 +22,7 @@ import com.facebook.presto.common.type.TypeSignature; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlAggregationFunction; import com.facebook.presto.operator.aggregation.state.StatisticalDigestState; import com.facebook.presto.operator.aggregation.state.StatisticalDigestStateFactory; @@ -122,7 +122,7 @@ public abstract class StatisticalDigestAggregationFunction } @Override - public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type valueType = boundVariables.getTypeVariable("V"); Type outputType = typeManager.getParameterizedType( diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/arrayagg/ArrayAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/arrayagg/ArrayAggregationFunction.java index 04e7a862098e..875d8aac41a9 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/arrayagg/ArrayAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/arrayagg/ArrayAggregationFunction.java @@ -20,7 +20,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlAggregationFunction; import com.facebook.presto.operator.aggregation.AccumulatorCompiler; import com.facebook.presto.operator.aggregation.AggregationMetadata; @@ -76,7 +76,7 @@ public String getDescription() } @Override - public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type type = boundVariables.getTypeVariable("T"); return generateAggregation(type, legacyArrayAgg, groupMode); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/arrayagg/SetAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/arrayagg/SetAggregationFunction.java index 0a577974a8cc..bb92e8695423 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/arrayagg/SetAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/arrayagg/SetAggregationFunction.java @@ -22,7 +22,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlAggregationFunction; import com.facebook.presto.operator.aggregation.AccumulatorCompiler; import com.facebook.presto.operator.aggregation.AggregationMetadata; @@ -76,7 +76,7 @@ public String getDescription() } @Override - public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type type = boundVariables.getTypeVariable("T"); ArrayType outputType = (ArrayType) typeManager.getParameterizedType(StandardTypes.ARRAY, ImmutableList.of( diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/arrayagg/SetUnionFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/arrayagg/SetUnionFunction.java index bd9cf8b04617..fba486ac7695 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/arrayagg/SetUnionFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/arrayagg/SetUnionFunction.java @@ -22,7 +22,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlAggregationFunction; import com.facebook.presto.operator.aggregation.AccumulatorCompiler; import com.facebook.presto.operator.aggregation.AggregationMetadata; @@ -75,7 +75,7 @@ public String getDescription() } @Override - public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type elementType = boundVariables.getTypeVariable("T"); ArrayType arrayType = (ArrayType) typeManager.getParameterizedType(StandardTypes.ARRAY, ImmutableList.of(TypeSignatureParameter.of(elementType.getTypeSignature()))); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/histogram/Histogram.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/histogram/Histogram.java index 9360fdcc4bb1..e0951b51cd2a 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/histogram/Histogram.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/histogram/Histogram.java @@ -21,7 +21,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlAggregationFunction; import com.facebook.presto.operator.aggregation.AccumulatorCompiler; import com.facebook.presto.operator.aggregation.AggregationMetadata; @@ -73,7 +73,7 @@ public String getDescription() } @Override - public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type keyType = boundVariables.getTypeVariable("K"); Type outputType = typeManager.getParameterizedType(StandardTypes.MAP, ImmutableList.of( diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/minmaxby/AbstractMinMaxBy.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/minmaxby/AbstractMinMaxBy.java index a5f5a000bf7d..612a8a42a509 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/minmaxby/AbstractMinMaxBy.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/minmaxby/AbstractMinMaxBy.java @@ -28,7 +28,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlAggregationFunction; import com.facebook.presto.operator.aggregation.AccumulatorCompiler; import com.facebook.presto.operator.aggregation.AggregationMetadata; @@ -95,14 +95,14 @@ protected AbstractMinMaxBy(boolean min) } @Override - public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type keyType = boundVariables.getTypeVariable("K"); Type valueType = boundVariables.getTypeVariable("V"); - return generateAggregation(valueType, keyType, functionManager); + return generateAggregation(valueType, keyType, functionAndTypeManager); } - private InternalAggregationFunction generateAggregation(Type valueType, Type keyType, FunctionManager functionManager) + private InternalAggregationFunction generateAggregation(Type valueType, Type keyType, FunctionAndTypeManager functionAndTypeManager) { Class stateClazz = getStateClass(keyType.getJavaType(), valueType.getJavaType()); DynamicClassLoader classLoader = new DynamicClassLoader(getClass().getClassLoader()); @@ -136,7 +136,7 @@ private InternalAggregationFunction generateAggregation(Type valueType, Type key CallSiteBinder binder = new CallSiteBinder(); OperatorType operator = min ? LESS_THAN : GREATER_THAN; - MethodHandle compareMethod = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(operator, fromTypes(keyType, keyType))).getMethodHandle(); + MethodHandle compareMethod = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(operator, fromTypes(keyType, keyType))).getMethodHandle(); ClassDefinition definition = new ClassDefinition( a(PUBLIC, FINAL), diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/minmaxby/AbstractMinMaxByNAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/minmaxby/AbstractMinMaxByNAggregationFunction.java index 5f5bb4ac8974..66ac7a005ebc 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/minmaxby/AbstractMinMaxByNAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/minmaxby/AbstractMinMaxByNAggregationFunction.java @@ -21,7 +21,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlAggregationFunction; import com.facebook.presto.operator.aggregation.AbstractMinMaxNAggregationFunction; import com.facebook.presto.operator.aggregation.AccumulatorCompiler; @@ -78,7 +78,7 @@ protected AbstractMinMaxByNAggregationFunction(String name, Function argumen } @Override - protected FunctionHandle getFunctionHandle(BoundVariables boundVariables, FunctionManager functionManager) + protected FunctionHandle getFunctionHandle(BoundVariables boundVariables, FunctionAndTypeManager functionAndTypeManager) { - return functionManager.lookupFunction(name, fromTypeSignatures(applyBoundVariables(argumentTypes, boundVariables))); + return functionAndTypeManager.lookupFunction(name, fromTypeSignatures(applyBoundVariables(argumentTypes, boundVariables))); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/operator/annotations/ImplementationDependency.java b/presto-main/src/main/java/com/facebook/presto/operator/annotations/ImplementationDependency.java index 86a987b0a7c1..f7d9f9bdbf0a 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/annotations/ImplementationDependency.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/annotations/ImplementationDependency.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.type.TypeSignature; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.Convention; import com.facebook.presto.spi.function.FunctionDependency; import com.facebook.presto.spi.function.InvocationConvention; @@ -42,7 +42,7 @@ public interface ImplementationDependency { - Object resolve(BoundVariables boundVariables, TypeManager typeManager, FunctionManager functionManager); + Object resolve(BoundVariables boundVariables, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager); static boolean isImplementationDependencyAnnotation(Annotation annotation) { diff --git a/presto-main/src/main/java/com/facebook/presto/operator/annotations/LiteralImplementationDependency.java b/presto-main/src/main/java/com/facebook/presto/operator/annotations/LiteralImplementationDependency.java index 0d87bd7223b7..5fce8c6c3a9b 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/annotations/LiteralImplementationDependency.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/annotations/LiteralImplementationDependency.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import static java.util.Objects.requireNonNull; @@ -30,7 +30,7 @@ public LiteralImplementationDependency(String literalName) } @Override - public Long resolve(BoundVariables boundVariables, TypeManager typeManager, FunctionManager functionManager) + public Long resolve(BoundVariables boundVariables, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { return boundVariables.getLongVariable(literalName); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/annotations/OperatorImplementationDependency.java b/presto-main/src/main/java/com/facebook/presto/operator/annotations/OperatorImplementationDependency.java index 2d816e15d1e8..2a42ca2700b8 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/annotations/OperatorImplementationDependency.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/annotations/OperatorImplementationDependency.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.function.OperatorType; import com.facebook.presto.common.type.TypeSignature; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.function.InvocationConvention; import com.google.common.collect.ImmutableList; @@ -53,9 +53,9 @@ public List getArgumentTypes() } @Override - protected FunctionHandle getFunctionHandle(BoundVariables boundVariables, FunctionManager functionManager) + protected FunctionHandle getFunctionHandle(BoundVariables boundVariables, FunctionAndTypeManager functionAndTypeManager) { - return functionManager.resolveOperator(operator, fromTypeSignatures(applyBoundVariables(argumentTypes, boundVariables))); + return functionAndTypeManager.resolveOperator(operator, fromTypeSignatures(applyBoundVariables(argumentTypes, boundVariables))); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/operator/annotations/ScalarImplementationDependency.java b/presto-main/src/main/java/com/facebook/presto/operator/annotations/ScalarImplementationDependency.java index 8ef1d950cacc..3af5e526bd66 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/annotations/ScalarImplementationDependency.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/annotations/ScalarImplementationDependency.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.function.InvocationConvention; @@ -32,16 +32,16 @@ protected ScalarImplementationDependency(Optional invocati this.invocationConvention = invocationConvention; } - protected abstract FunctionHandle getFunctionHandle(BoundVariables boundVariables, FunctionManager functionManager); + protected abstract FunctionHandle getFunctionHandle(BoundVariables boundVariables, FunctionAndTypeManager functionAndTypeManager); @Override - public MethodHandle resolve(BoundVariables boundVariables, TypeManager typeManager, FunctionManager functionManager) + public MethodHandle resolve(BoundVariables boundVariables, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { - FunctionHandle functionHandle = getFunctionHandle(boundVariables, functionManager); + FunctionHandle functionHandle = getFunctionHandle(boundVariables, functionAndTypeManager); if (invocationConvention.isPresent()) { - return functionManager.getFunctionInvokerProvider().createFunctionInvoker(functionHandle, invocationConvention).methodHandle(); + return functionAndTypeManager.getFunctionInvokerProvider().createFunctionInvoker(functionHandle, invocationConvention).methodHandle(); } else { - return functionManager.getBuiltInScalarFunctionImplementation(functionHandle).getMethodHandle(); + return functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionHandle).getMethodHandle(); } } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/annotations/TypeImplementationDependency.java b/presto-main/src/main/java/com/facebook/presto/operator/annotations/TypeImplementationDependency.java index 3f1b5e0d85b3..01e17e55990d 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/annotations/TypeImplementationDependency.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/annotations/TypeImplementationDependency.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignature; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import java.util.Objects; @@ -36,7 +36,7 @@ public TypeImplementationDependency(String signature) } @Override - public Type resolve(BoundVariables boundVariables, TypeManager typeManager, FunctionManager functionManager) + public Type resolve(BoundVariables boundVariables, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { return applyBoundVariables(typeManager, signature, boundVariables); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/index/FieldSetFilteringRecordSet.java b/presto-main/src/main/java/com/facebook/presto/operator/index/FieldSetFilteringRecordSet.java index 7703b2623404..f5f5b94b2be0 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/index/FieldSetFilteringRecordSet.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/index/FieldSetFilteringRecordSet.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.function.OperatorType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.RecordCursor; import com.facebook.presto.spi.RecordSet; import com.google.common.collect.ImmutableList; @@ -42,9 +42,9 @@ public class FieldSetFilteringRecordSet private final RecordSet delegate; private final List> fieldSets; - public FieldSetFilteringRecordSet(FunctionManager functionManager, RecordSet delegate, List> fieldSets) + public FieldSetFilteringRecordSet(FunctionAndTypeManager functionAndTypeManager, RecordSet delegate, List> fieldSets) { - requireNonNull(functionManager, "functionManager is null"); + requireNonNull(functionAndTypeManager, "functionManager is null"); this.delegate = requireNonNull(delegate, "delegate is null"); ImmutableList.Builder> fieldSetsBuilder = ImmutableList.builder(); @@ -54,8 +54,8 @@ public FieldSetFilteringRecordSet(FunctionManager functionManager, RecordSet del for (int field : fieldSet) { fieldSetBuilder.add(new Field( field, - functionManager.getBuiltInScalarFunctionImplementation( - functionManager.resolveOperator(OperatorType.EQUAL, fromTypes(columnTypes.get(field), columnTypes.get(field)))).getMethodHandle())); + functionAndTypeManager.getBuiltInScalarFunctionImplementation( + functionAndTypeManager.resolveOperator(OperatorType.EQUAL, fromTypes(columnTypes.get(field), columnTypes.get(field)))).getMethodHandle())); } fieldSetsBuilder.add(fieldSetBuilder.build()); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/AbstractGreatestLeast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/AbstractGreatestLeast.java index 7a3c1f4feda6..7caa787a6d2d 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/AbstractGreatestLeast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/AbstractGreatestLeast.java @@ -28,7 +28,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.function.FunctionKind; @@ -100,13 +100,13 @@ public boolean isDeterministic() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type type = boundVariables.getTypeVariable("E"); checkArgument(type.isOrderable(), "Type must be orderable"); - MethodHandle compareMethod = functionManager.getBuiltInScalarFunctionImplementation( - functionManager.resolveOperator(operatorType, fromTypes(type, type))).getMethodHandle(); + MethodHandle compareMethod = functionAndTypeManager.getBuiltInScalarFunctionImplementation( + functionAndTypeManager.resolveOperator(operatorType, fromTypes(type, type))).getMethodHandle(); List> javaTypes = IntStream.range(0, arity) .mapToObj(i -> type.getJavaType()) diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ApplyFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ApplyFunction.java index 2790a9d8a843..436d5d3db1e3 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ApplyFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ApplyFunction.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.function.FunctionKind; import com.facebook.presto.spi.function.Signature; @@ -84,7 +84,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type argumentType = boundVariables.getTypeVariable("T"); Type returnType = boundVariables.getTypeVariable("U"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayConcatFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayConcatFunction.java index 01c0ed381841..35d0a56234e2 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayConcatFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayConcatFunction.java @@ -21,7 +21,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.function.FunctionKind; @@ -86,7 +86,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { if (arity < 2) { throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "There must be two or more arguments to " + FUNCTION_NAME); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayConstructor.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayConstructor.java index b43cc98aea07..0c69fcf6e6cf 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayConstructor.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayConstructor.java @@ -29,7 +29,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.function.FunctionKind; import com.facebook.presto.spi.function.Signature; @@ -112,7 +112,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Map types = boundVariables.getTypeVariables(); checkArgument(types.size() == 1, "Can only construct arrays from exactly matching types"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayFlattenFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayFlattenFunction.java index 398bb7e11d24..a516f603e454 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayFlattenFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayFlattenFunction.java @@ -21,7 +21,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.function.FunctionKind; import com.facebook.presto.spi.function.Signature; @@ -76,7 +76,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type elementType = boundVariables.getTypeVariable("E"); Type arrayType = typeManager.getParameterizedType(StandardTypes.ARRAY, ImmutableList.of(TypeSignatureParameter.of(elementType.getTypeSignature()))); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayJoin.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayJoin.java index 45998912b1f4..bbb841044ae0 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayJoin.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayJoin.java @@ -24,7 +24,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignature; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.ArgumentProperty; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.ReturnPlaceConvention; @@ -149,11 +149,11 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { return specializeArrayJoin( boundVariables.getTypeVariables(), - functionManager, + functionAndTypeManager, ImmutableList.of(false, false, false), METHOD_HANDLE_STACK, METHOD_HANDLE_PROVIDED_BLOCK); @@ -197,11 +197,11 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { return specializeArrayJoin( boundVariables.getTypeVariables(), - functionManager, + functionAndTypeManager, ImmutableList.of(false, false), METHOD_HANDLE_STACK, METHOD_HANDLE_PROVIDED_BLOCK); @@ -209,7 +209,7 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab private static BuiltInScalarFunctionImplementation specializeArrayJoin( Map types, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, List nullableArguments, MethodHandle methodHandleStack, MethodHandle methodHandleProvidedBlock) @@ -230,7 +230,7 @@ private static BuiltInScalarFunctionImplementation specializeArrayJoin( } else { try { - BuiltInScalarFunctionImplementation castFunction = functionManager.getBuiltInScalarFunctionImplementation(functionManager.lookupCast(CAST, type.getTypeSignature(), VARCHAR_TYPE_SIGNATURE)); + BuiltInScalarFunctionImplementation castFunction = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.lookupCast(CAST, type.getTypeSignature(), VARCHAR_TYPE_SIGNATURE)); MethodHandle getter; Class elementType = type.getJavaType(); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayReduceFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayReduceFunction.java index 81bb76458976..51b6acf8b094 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayReduceFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayReduceFunction.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.function.FunctionKind; import com.facebook.presto.spi.function.Signature; @@ -85,7 +85,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type inputType = boundVariables.getTypeVariable("T"); Type intermediateType = boundVariables.getTypeVariable("S"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArraySubscriptOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArraySubscriptOperator.java index d3fcc03adc4d..5bead2a7de02 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArraySubscriptOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArraySubscriptOperator.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.spi.PrestoException; import com.google.common.collect.ImmutableList; @@ -58,7 +58,7 @@ protected ArraySubscriptOperator() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { checkArgument(boundVariables.getTypeVariables().size() == 1, "Expected one type, got %s", boundVariables.getTypeVariables()); Type elementType = boundVariables.getTypeVariable("E"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayToArrayCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayToArrayCast.java index 6aee3c9b07db..5e1a54b252cb 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayToArrayCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayToArrayCast.java @@ -25,7 +25,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; import com.facebook.presto.metadata.CastType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.function.FunctionMetadata; @@ -70,15 +70,15 @@ private ArrayToArrayCast() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { checkArgument(arity == 1, "Expected arity to be 1"); Type fromType = boundVariables.getTypeVariable("F"); Type toType = boundVariables.getTypeVariable("T"); - FunctionHandle functionHandle = functionManager.lookupCast(CastType.CAST, fromType.getTypeSignature(), toType.getTypeSignature()); - BuiltInScalarFunctionImplementation function = functionManager.getBuiltInScalarFunctionImplementation(functionHandle); - Class castOperatorClass = generateArrayCast(typeManager, functionManager.getFunctionMetadata(functionHandle), function); + FunctionHandle functionHandle = functionAndTypeManager.lookupCast(CastType.CAST, fromType.getTypeSignature(), toType.getTypeSignature()); + BuiltInScalarFunctionImplementation function = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionHandle); + Class castOperatorClass = generateArrayCast(typeManager, functionAndTypeManager.getFunctionMetadata(functionHandle), function); MethodHandle methodHandle = methodHandle(castOperatorClass, "castArray", SqlFunctionProperties.class, Block.class); return new BuiltInScalarFunctionImplementation( false, diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayToElementConcatFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayToElementConcatFunction.java index 21cdd1b2fb8f..21e23682e88e 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayToElementConcatFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayToElementConcatFunction.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.function.FunctionKind; import com.facebook.presto.spi.function.Signature; @@ -78,7 +78,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type type = boundVariables.getTypeVariable("E"); MethodHandle methodHandle; diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayToJsonCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayToJsonCast.java index 7f587b797a97..6652b5c0c255 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayToJsonCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayToJsonCast.java @@ -21,7 +21,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.util.JsonUtil.JsonGeneratorWriter; import com.fasterxml.jackson.core.JsonGenerator; @@ -62,7 +62,7 @@ private ArrayToJsonCast() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { checkArgument(arity == 1, "Expected arity to be 1"); Type type = boundVariables.getTypeVariable("T"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayTransformFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayTransformFunction.java index 4dc9ae984fa2..16c151f3a034 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayTransformFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayTransformFunction.java @@ -30,7 +30,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.function.FunctionKind; import com.facebook.presto.spi.function.Signature; @@ -105,7 +105,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type inputType = boundVariables.getTypeVariable("T"); Type outputType = boundVariables.getTypeVariable("U"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/CastFromUnknownOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/CastFromUnknownOperator.java index 621693ea2736..3ecc51ec4bd6 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/CastFromUnknownOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/CastFromUnknownOperator.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.google.common.collect.ImmutableList; @@ -48,7 +48,7 @@ public CastFromUnknownOperator() @Override public BuiltInScalarFunctionImplementation specialize( BoundVariables boundVariables, int arity, TypeManager typeManager, - FunctionManager functionManager) + FunctionAndTypeManager functionAndTypeManager) { Type toType = boundVariables.getTypeVariable("E"); MethodHandle methodHandle = METHOD_HANDLE_NON_NULL.asType(METHOD_HANDLE_NON_NULL.type().changeReturnType(toType.getJavaType())); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ConcatFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ConcatFunction.java index 367a0ed684a5..42c6c1d51f2a 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ConcatFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ConcatFunction.java @@ -26,7 +26,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignature; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.function.FunctionKind; @@ -108,7 +108,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { if (arity < 2) { throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "There must be two or more concatenation arguments"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ElementToArrayConcatFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ElementToArrayConcatFunction.java index 9f35be09ecc8..6b40a261f0db 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ElementToArrayConcatFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ElementToArrayConcatFunction.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.function.FunctionKind; import com.facebook.presto.spi.function.Signature; @@ -78,7 +78,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type type = boundVariables.getTypeVariable("E"); MethodHandle methodHandle; diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/IdentityCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/IdentityCast.java index c76868ee3a33..5d3a088f54c2 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/IdentityCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/IdentityCast.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.google.common.collect.ImmutableList; @@ -45,7 +45,7 @@ protected IdentityCast() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { checkArgument(boundVariables.getTypeVariables().size() == 1, "Expected only one type"); Type type = boundVariables.getTypeVariable("T"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/InvokeFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/InvokeFunction.java index d30ef6c82455..cb8d49392e57 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/InvokeFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/InvokeFunction.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.function.FunctionKind; import com.facebook.presto.spi.function.Signature; @@ -77,7 +77,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type returnType = boundVariables.getTypeVariable("T"); return new BuiltInScalarFunctionImplementation( diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonStringToArrayCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonStringToArrayCast.java index 5b698afd850b..84274ac27454 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonStringToArrayCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonStringToArrayCast.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.type.StandardTypes; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.function.Signature; import com.facebook.presto.spi.function.SqlFunctionVisibility; @@ -68,8 +68,8 @@ public final SqlFunctionVisibility getVisibility() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { - return JSON_TO_ARRAY.specialize(boundVariables, arity, typeManager, functionManager); + return JSON_TO_ARRAY.specialize(boundVariables, arity, typeManager, functionAndTypeManager); } } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonStringToMapCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonStringToMapCast.java index f3987f3e956c..a5f8c0c4b75e 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonStringToMapCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonStringToMapCast.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.type.StandardTypes; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.function.Signature; import com.facebook.presto.spi.function.SqlFunctionVisibility; @@ -69,8 +69,8 @@ public final SqlFunctionVisibility getVisibility() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { - return JSON_TO_MAP.specialize(boundVariables, arity, typeManager, functionManager); + return JSON_TO_MAP.specialize(boundVariables, arity, typeManager, functionAndTypeManager); } } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonStringToRowCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonStringToRowCast.java index 1fe622fb2e83..c45f555e2555 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonStringToRowCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonStringToRowCast.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.type.StandardTypes; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.function.Signature; import com.facebook.presto.spi.function.SqlFunctionVisibility; @@ -68,8 +68,8 @@ public final SqlFunctionVisibility getVisibility() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { - return JSON_TO_ROW.specialize(boundVariables, arity, typeManager, functionManager); + return JSON_TO_ROW.specialize(boundVariables, arity, typeManager, functionAndTypeManager); } } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToArrayCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToArrayCast.java index fb6ba48edd50..2dc38627fee8 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToArrayCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToArrayCast.java @@ -24,7 +24,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.util.JsonCastException; @@ -67,7 +67,7 @@ private JsonToArrayCast() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { checkArgument(arity == 1, "Expected arity to be 1"); Type type = boundVariables.getTypeVariable("T"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToMapCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToMapCast.java index cba64bc45869..6e80ecfd0cf3 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToMapCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToMapCast.java @@ -24,7 +24,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.util.JsonCastException; @@ -70,7 +70,7 @@ private JsonToMapCast() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { checkArgument(arity == 1, "Expected arity to be 1"); Type keyType = boundVariables.getTypeVariable("K"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToRowCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToRowCast.java index f2478c5e2f5e..2817dc582114 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToRowCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/JsonToRowCast.java @@ -24,7 +24,7 @@ import com.facebook.presto.common.type.StandardTypes; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.util.JsonCastException; @@ -74,7 +74,7 @@ private JsonToRowCast() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { checkArgument(arity == 1, "Expected arity to be 1"); RowType rowType = (RowType) boundVariables.getTypeVariable("T"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapConcatFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapConcatFunction.java index 41c020a2bbe7..82c70932d81e 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapConcatFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapConcatFunction.java @@ -24,7 +24,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.operator.aggregation.TypedSet; import com.facebook.presto.spi.PrestoException; @@ -91,7 +91,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { if (arity < 2) { throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "There must be two or more concatenation arguments to " + FUNCTION_NAME); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapConstructor.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapConstructor.java index 6e15b5b0e232..d20a5c92bfce 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapConstructor.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapConstructor.java @@ -29,7 +29,7 @@ import com.facebook.presto.common.type.TypeSignature; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.function.FunctionKind; @@ -110,18 +110,18 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type keyType = boundVariables.getTypeVariable("K"); Type valueType = boundVariables.getTypeVariable("V"); Type mapType = typeManager.getParameterizedType(MAP, ImmutableList.of(TypeSignatureParameter.of(keyType.getTypeSignature()), TypeSignatureParameter.of(valueType.getTypeSignature()))); - MethodHandle keyNativeHashCode = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(OperatorType.HASH_CODE, fromTypes(keyType))).getMethodHandle(); + MethodHandle keyNativeHashCode = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(OperatorType.HASH_CODE, fromTypes(keyType))).getMethodHandle(); MethodHandle keyBlockHashCode = compose(keyNativeHashCode, nativeValueGetter(keyType)); - MethodHandle keyNativeEquals = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(OperatorType.EQUAL, fromTypes(keyType, keyType))).getMethodHandle(); + MethodHandle keyNativeEquals = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(OperatorType.EQUAL, fromTypes(keyType, keyType))).getMethodHandle(); MethodHandle keyBlockEquals = compose(keyNativeEquals, nativeValueGetter(keyType), nativeValueGetter(keyType)); - MethodHandle keyIndeterminate = functionManager.getBuiltInScalarFunctionImplementation( - functionManager.resolveOperator(INDETERMINATE, fromTypeSignatures((keyType.getTypeSignature())))).getMethodHandle(); + MethodHandle keyIndeterminate = functionAndTypeManager.getBuiltInScalarFunctionImplementation( + functionAndTypeManager.resolveOperator(INDETERMINATE, fromTypeSignatures((keyType.getTypeSignature())))).getMethodHandle(); MethodHandle instanceFactory = constructorMethodHandle(State.class, MapType.class).bindTo(mapType); return new BuiltInScalarFunctionImplementation( diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapElementAtFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapElementAtFunction.java index 612b78007084..8e0f377cc183 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapElementAtFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapElementAtFunction.java @@ -22,7 +22,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.function.FunctionKind; @@ -89,14 +89,14 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type keyType = boundVariables.getTypeVariable("K"); Type valueType = boundVariables.getTypeVariable("V"); - MethodHandle keyNativeHashCode = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(OperatorType.HASH_CODE, fromTypes(keyType))).getMethodHandle(); + MethodHandle keyNativeHashCode = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(OperatorType.HASH_CODE, fromTypes(keyType))).getMethodHandle(); MethodHandle keyBlockHashCode = compose(keyNativeHashCode, nativeValueGetter(keyType)); - MethodHandle keyNativeEquals = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(OperatorType.EQUAL, fromTypes(keyType, keyType))).getMethodHandle(); + MethodHandle keyNativeEquals = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(OperatorType.EQUAL, fromTypes(keyType, keyType))).getMethodHandle(); MethodHandle keyBlockNativeEquals = compose(keyNativeEquals, nativeValueGetter(keyType)); MethodHandle methodHandle; diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapFilterFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapFilterFunction.java index d9334bb427f5..863679839975 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapFilterFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapFilterFunction.java @@ -33,7 +33,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.function.FunctionKind; import com.facebook.presto.spi.function.Signature; @@ -111,7 +111,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type keyType = boundVariables.getTypeVariable("K"); Type valueType = boundVariables.getTypeVariable("V"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapHashCodeOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapHashCodeOperator.java index 0f36a2fb7c52..798f998180ba 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapHashCodeOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapHashCodeOperator.java @@ -19,7 +19,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.google.common.collect.ImmutableList; @@ -50,13 +50,13 @@ private MapHashCodeOperator() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type keyType = boundVariables.getTypeVariable("K"); Type valueType = boundVariables.getTypeVariable("V"); - MethodHandle keyHashCodeFunction = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(HASH_CODE, fromTypes(keyType))).getMethodHandle(); - MethodHandle valueHashCodeFunction = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(HASH_CODE, fromTypes(valueType))).getMethodHandle(); + MethodHandle keyHashCodeFunction = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(HASH_CODE, fromTypes(keyType))).getMethodHandle(); + MethodHandle valueHashCodeFunction = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(HASH_CODE, fromTypes(valueType))).getMethodHandle(); MethodHandle method = METHOD_HANDLE.bindTo(keyHashCodeFunction).bindTo(valueHashCodeFunction).bindTo(keyType).bindTo(valueType); return new BuiltInScalarFunctionImplementation( diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapSubscriptOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapSubscriptOperator.java index 125481af7fce..278fd4150fab 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapSubscriptOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapSubscriptOperator.java @@ -22,7 +22,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.function.FunctionHandle; @@ -72,14 +72,14 @@ public MapSubscriptOperator(boolean legacyMissingKey) } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type keyType = boundVariables.getTypeVariable("K"); Type valueType = boundVariables.getTypeVariable("V"); - MethodHandle keyNativeHashCode = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(OperatorType.HASH_CODE, fromTypes(keyType))).getMethodHandle(); + MethodHandle keyNativeHashCode = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(OperatorType.HASH_CODE, fromTypes(keyType))).getMethodHandle(); MethodHandle keyBlockHashCode = compose(keyNativeHashCode, nativeValueGetter(keyType)); - MethodHandle keyNativeEquals = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(OperatorType.EQUAL, fromTypes(keyType, keyType))).getMethodHandle(); + MethodHandle keyNativeEquals = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(OperatorType.EQUAL, fromTypes(keyType, keyType))).getMethodHandle(); MethodHandle keyBlockNativeEquals = compose(keyNativeEquals, nativeValueGetter(keyType)); MethodHandle methodHandle; @@ -99,7 +99,7 @@ else if (keyType.getJavaType() == Slice.class) { methodHandle = METHOD_HANDLE_OBJECT; } methodHandle = MethodHandles.insertArguments(methodHandle, 0, legacyMissingKey); - MissingKeyExceptionFactory missingKeyExceptionFactory = new MissingKeyExceptionFactory(functionManager, keyType); + MissingKeyExceptionFactory missingKeyExceptionFactory = new MissingKeyExceptionFactory(functionAndTypeManager, keyType); methodHandle = methodHandle.bindTo(missingKeyExceptionFactory).bindTo(keyNativeHashCode).bindTo(keyBlockNativeEquals).bindTo(keyBlockHashCode).bindTo(valueType); methodHandle = methodHandle.asType(methodHandle.type().changeReturnType(Primitives.wrap(valueType.getJavaType()))); @@ -221,13 +221,13 @@ private static class MissingKeyExceptionFactory private final InterpretedFunctionInvoker functionInvoker; private final FunctionHandle castFunction; - public MissingKeyExceptionFactory(FunctionManager functionManager, Type keyType) + public MissingKeyExceptionFactory(FunctionAndTypeManager functionAndTypeManager, Type keyType) { - functionInvoker = new InterpretedFunctionInvoker(functionManager); + functionInvoker = new InterpretedFunctionInvoker(functionAndTypeManager); FunctionHandle castFunction = null; try { - castFunction = functionManager.lookupCast(CAST, keyType.getTypeSignature(), VARCHAR.getTypeSignature()); + castFunction = functionAndTypeManager.lookupCast(CAST, keyType.getTypeSignature(), VARCHAR.getTypeSignature()); } catch (PrestoException ignored) { } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapToJsonCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapToJsonCast.java index 09398a389a1d..22dabc86e291 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapToJsonCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapToJsonCast.java @@ -22,7 +22,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.fasterxml.jackson.core.JsonGenerator; import com.google.common.collect.ImmutableList; @@ -66,7 +66,7 @@ private MapToJsonCast() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { checkArgument(arity == 1, "Expected arity to be 1"); Type keyType = boundVariables.getTypeVariable("K"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapToMapCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapToMapCast.java index 07ad679fbab0..61a5c21ee05b 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapToMapCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapToMapCast.java @@ -22,7 +22,7 @@ import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; import com.facebook.presto.metadata.CastType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.operator.aggregation.TypedSet; import com.facebook.presto.spi.PrestoException; @@ -79,7 +79,7 @@ public MapToMapCast() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { checkArgument(arity == 1, "Expected arity to be 1"); Type fromKeyType = boundVariables.getTypeVariable("FK"); @@ -92,8 +92,8 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab TypeSignatureParameter.of(toKeyType.getTypeSignature()), TypeSignatureParameter.of(toValueType.getTypeSignature()))); - MethodHandle keyProcessor = buildProcessor(functionManager, fromKeyType, toKeyType, true); - MethodHandle valueProcessor = buildProcessor(functionManager, fromValueType, toValueType, false); + MethodHandle keyProcessor = buildProcessor(functionAndTypeManager, fromKeyType, toKeyType, true); + MethodHandle valueProcessor = buildProcessor(functionAndTypeManager, fromValueType, toValueType, false); MethodHandle target = MethodHandles.insertArguments(METHOD_HANDLE, 0, keyProcessor, valueProcessor, toMapType); return new BuiltInScalarFunctionImplementation(true, ImmutableList.of(valueTypeArgumentProperty(RETURN_NULL_ON_NULL)), target); } @@ -102,12 +102,12 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab * The signature of the returned MethodHandle is (Block fromMap, int position, SqlFunctionProperties properties, BlockBuilder mapBlockBuilder)void. * The processor will get the value from fromMap, cast it and write to toBlock. */ - private MethodHandle buildProcessor(FunctionManager functionManager, Type fromType, Type toType, boolean isKey) + private MethodHandle buildProcessor(FunctionAndTypeManager functionAndTypeManager, Type fromType, Type toType, boolean isKey) { MethodHandle getter = nativeValueGetter(fromType); // Adapt cast that takes ([SqlFunctionProperties,] ?) to one that takes (?, SqlFunctionProperties), where ? is the return type of getter. - BuiltInScalarFunctionImplementation castImplementation = functionManager.getBuiltInScalarFunctionImplementation(functionManager.lookupCast(CastType.CAST, fromType.getTypeSignature(), toType.getTypeSignature())); + BuiltInScalarFunctionImplementation castImplementation = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.lookupCast(CastType.CAST, fromType.getTypeSignature(), toType.getTypeSignature())); MethodHandle cast = castImplementation.getMethodHandle(); if (cast.type().parameterArray()[0] != SqlFunctionProperties.class) { cast = MethodHandles.dropArguments(cast, 0, SqlFunctionProperties.class); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapTransformKeyFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapTransformKeyFunction.java index f0f9339782db..b11abdda09c9 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapTransformKeyFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapTransformKeyFunction.java @@ -34,7 +34,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.operator.aggregation.TypedSet; import com.facebook.presto.spi.ErrorCodeSupplier; @@ -121,7 +121,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type keyType = boundVariables.getTypeVariable("K1"); Type transformedKeyType = boundVariables.getTypeVariable("K2"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapTransformValueFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapTransformValueFunction.java index 3d147ee129ce..9b2ce0833968 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapTransformValueFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapTransformValueFunction.java @@ -34,7 +34,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.ErrorCodeSupplier; import com.facebook.presto.spi.PrestoException; @@ -119,7 +119,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type keyType = boundVariables.getTypeVariable("K"); Type valueType = boundVariables.getTypeVariable("V1"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapZipWithFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapZipWithFunction.java index f63ecafc9293..155103d1ea29 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapZipWithFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapZipWithFunction.java @@ -26,7 +26,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.function.FunctionKind; @@ -93,7 +93,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type keyType = boundVariables.getTypeVariable("K"); Type inputValueType1 = boundVariables.getTypeVariable("V1"); @@ -104,9 +104,9 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab ImmutableList.of( TypeSignatureParameter.of(keyType.getTypeSignature()), TypeSignatureParameter.of(outputValueType.getTypeSignature()))); - MethodHandle keyNativeHashCode = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(OperatorType.HASH_CODE, fromTypes(keyType))).getMethodHandle(); + MethodHandle keyNativeHashCode = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(OperatorType.HASH_CODE, fromTypes(keyType))).getMethodHandle(); MethodHandle keyBlockHashCode = compose(keyNativeHashCode, nativeValueGetter(keyType)); - MethodHandle keyNativeEquals = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(OperatorType.EQUAL, fromTypes(keyType, keyType))).getMethodHandle(); + MethodHandle keyNativeEquals = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(OperatorType.EQUAL, fromTypes(keyType, keyType))).getMethodHandle(); MethodHandle keyBlockNativeEquals = compose(keyNativeEquals, nativeValueGetter(keyType)); MethodHandle keyBlockEquals = compose(keyNativeEquals, nativeValueGetter(keyType), nativeValueGetter(keyType)); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ParametricScalar.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ParametricScalar.java index 411381a23dce..90ab08f6aeba 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ParametricScalar.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ParametricScalar.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.operator.ParametricImplementationsGroup; import com.facebook.presto.operator.scalar.annotations.ParametricScalarImplementation; @@ -81,19 +81,19 @@ public ParametricImplementationsGroup getImpleme } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Signature boundSignature = applyBoundVariables(getSignature(), boundVariables, arity); if (implementations.getExactImplementations().containsKey(boundSignature)) { ParametricScalarImplementation implementation = implementations.getExactImplementations().get(boundSignature); - Optional scalarFunctionImplementation = implementation.specialize(boundSignature, boundVariables, typeManager, functionManager); + Optional scalarFunctionImplementation = implementation.specialize(boundSignature, boundVariables, typeManager, functionAndTypeManager); checkCondition(scalarFunctionImplementation.isPresent(), FUNCTION_IMPLEMENTATION_ERROR, String.format("Exact implementation of %s do not match expected java types.", boundSignature.getNameSuffix())); return scalarFunctionImplementation.get(); } BuiltInScalarFunctionImplementation selectedImplementation = null; for (ParametricScalarImplementation implementation : implementations.getSpecializedImplementations()) { - Optional scalarFunctionImplementation = implementation.specialize(boundSignature, boundVariables, typeManager, functionManager); + Optional scalarFunctionImplementation = implementation.specialize(boundSignature, boundVariables, typeManager, functionAndTypeManager); if (scalarFunctionImplementation.isPresent()) { checkCondition(selectedImplementation == null, AMBIGUOUS_FUNCTION_IMPLEMENTATION, "Ambiguous implementation for %s with bindings %s", getSignature(), boundVariables.getTypeVariables()); selectedImplementation = scalarFunctionImplementation.get(); @@ -103,7 +103,7 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab return selectedImplementation; } for (ParametricScalarImplementation implementation : implementations.getGenericImplementations()) { - Optional scalarFunctionImplementation = implementation.specialize(boundSignature, boundVariables, typeManager, functionManager); + Optional scalarFunctionImplementation = implementation.specialize(boundSignature, boundVariables, typeManager, functionAndTypeManager); if (scalarFunctionImplementation.isPresent()) { checkCondition(selectedImplementation == null, AMBIGUOUS_FUNCTION_IMPLEMENTATION, "Ambiguous implementation for %s with bindings %s", getSignature(), boundVariables.getTypeVariables()); selectedImplementation = scalarFunctionImplementation.get(); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/Re2JCastToRegexpFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/Re2JCastToRegexpFunction.java index 2f7fb84d0997..0d3cb0b0bb3a 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/Re2JCastToRegexpFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/Re2JCastToRegexpFunction.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.type.Re2JRegexp; @@ -64,7 +64,7 @@ private Re2JCastToRegexpFunction(String sourceType, int dfaStatesLimit, int dfaR } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { return new BuiltInScalarFunctionImplementation( false, diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowComparisonOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowComparisonOperator.java index e83a0d648505..1d0de3a42eb5 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowComparisonOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowComparisonOperator.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.RowType; import com.facebook.presto.common.type.StandardTypes; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.spi.function.FunctionHandle; import com.google.common.collect.ImmutableList; @@ -45,12 +45,12 @@ protected RowComparisonOperator(OperatorType operatorType) ImmutableList.of(parseTypeSignature("T"), parseTypeSignature("T"))); } - protected List getMethodHandles(RowType type, FunctionManager functionManager, OperatorType operatorType) + protected List getMethodHandles(RowType type, FunctionAndTypeManager functionAndTypeManager, OperatorType operatorType) { ImmutableList.Builder argumentMethods = ImmutableList.builder(); for (Type parameterType : type.getTypeParameters()) { - FunctionHandle operatorHandle = functionManager.resolveOperator(operatorType, fromTypes(parameterType, parameterType)); - argumentMethods.add(functionManager.getBuiltInScalarFunctionImplementation(operatorHandle).getMethodHandle()); + FunctionHandle operatorHandle = functionAndTypeManager.resolveOperator(operatorType, fromTypes(parameterType, parameterType)); + argumentMethods.add(functionAndTypeManager.getBuiltInScalarFunctionImplementation(operatorHandle).getMethodHandle()); } return argumentMethods.build(); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowDistinctFromOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowDistinctFromOperator.java index 6dd7bfda174a..5ffa4367430c 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowDistinctFromOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowDistinctFromOperator.java @@ -18,8 +18,8 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.FunctionInvoker; -import com.facebook.presto.metadata.FunctionManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.ReturnPlaceConvention; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.ScalarImplementationChoice; @@ -61,13 +61,13 @@ private RowDistinctFromOperator() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { ImmutableList.Builder argumentMethods = ImmutableList.builder(); Type type = boundVariables.getTypeVariable("T"); for (Type parameterType : type.getTypeParameters()) { - FunctionHandle operatorHandle = functionManager.resolveOperator(IS_DISTINCT_FROM, fromTypes(parameterType, parameterType)); - FunctionInvoker functionInvoker = functionManager.getFunctionInvokerProvider().createFunctionInvoker( + FunctionHandle operatorHandle = functionAndTypeManager.resolveOperator(IS_DISTINCT_FROM, fromTypes(parameterType, parameterType)); + FunctionInvoker functionInvoker = functionAndTypeManager.getFunctionInvokerProvider().createFunctionInvoker( operatorHandle, Optional.of(new InvocationConvention( ImmutableList.of(NULL_FLAG, NULL_FLAG), diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowEqualOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowEqualOperator.java index fa0b062b8463..1ea9de503cc5 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowEqualOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowEqualOperator.java @@ -19,7 +19,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.spi.function.FunctionHandle; import com.google.common.collect.ImmutableList; @@ -54,7 +54,7 @@ private RowEqualOperator() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { RowType type = (RowType) boundVariables.getTypeVariable("T"); return new BuiltInScalarFunctionImplementation( @@ -64,20 +64,20 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab valueTypeArgumentProperty(RETURN_NULL_ON_NULL)), METHOD_HANDLE .bindTo(type) - .bindTo(resolveFieldEqualOperators(type, functionManager))); + .bindTo(resolveFieldEqualOperators(type, functionAndTypeManager))); } - public static List resolveFieldEqualOperators(RowType rowType, FunctionManager functionManager) + public static List resolveFieldEqualOperators(RowType rowType, FunctionAndTypeManager functionAndTypeManager) { return rowType.getTypeParameters().stream() - .map(type -> resolveEqualOperator(type, functionManager)) + .map(type -> resolveEqualOperator(type, functionAndTypeManager)) .collect(toImmutableList()); } - private static MethodHandle resolveEqualOperator(Type type, FunctionManager functionManager) + private static MethodHandle resolveEqualOperator(Type type, FunctionAndTypeManager functionAndTypeManager) { - FunctionHandle operator = functionManager.resolveOperator(EQUAL, fromTypes(type, type)); - BuiltInScalarFunctionImplementation implementation = functionManager.getBuiltInScalarFunctionImplementation(operator); + FunctionHandle operator = functionAndTypeManager.resolveOperator(EQUAL, fromTypes(type, type)); + BuiltInScalarFunctionImplementation implementation = functionAndTypeManager.getBuiltInScalarFunctionImplementation(operator); return implementation.getMethodHandle(); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowGreaterThanOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowGreaterThanOperator.java index f86701433ee0..a079290963dc 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowGreaterThanOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowGreaterThanOperator.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.google.common.collect.ImmutableList; import java.lang.invoke.MethodHandle; @@ -41,7 +41,7 @@ private RowGreaterThanOperator() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type type = boundVariables.getTypeVariable("T"); return new BuiltInScalarFunctionImplementation( @@ -49,7 +49,7 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab ImmutableList.of( valueTypeArgumentProperty(RETURN_NULL_ON_NULL), valueTypeArgumentProperty(RETURN_NULL_ON_NULL)), - METHOD_HANDLE.bindTo(type).bindTo(getMethodHandles((RowType) type, functionManager, GREATER_THAN))); + METHOD_HANDLE.bindTo(type).bindTo(getMethodHandles((RowType) type, functionAndTypeManager, GREATER_THAN))); } public static boolean greater( diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowGreaterThanOrEqualOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowGreaterThanOrEqualOperator.java index be30acb8b38b..31ce76e36480 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowGreaterThanOrEqualOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowGreaterThanOrEqualOperator.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.google.common.collect.ImmutableList; import java.lang.invoke.MethodHandle; @@ -42,7 +42,7 @@ private RowGreaterThanOrEqualOperator() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type type = boundVariables.getTypeVariable("T"); return new BuiltInScalarFunctionImplementation( @@ -50,7 +50,7 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab ImmutableList.of( valueTypeArgumentProperty(RETURN_NULL_ON_NULL), valueTypeArgumentProperty(RETURN_NULL_ON_NULL)), - METHOD_HANDLE.bindTo(type).bindTo(getMethodHandles((RowType) type, functionManager, GREATER_THAN))); + METHOD_HANDLE.bindTo(type).bindTo(getMethodHandles((RowType) type, functionAndTypeManager, GREATER_THAN))); } public static boolean greaterOrEqual( diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowHashCodeOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowHashCodeOperator.java index 52a53fd4aa2c..63bddbcde4e8 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowHashCodeOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowHashCodeOperator.java @@ -20,7 +20,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.google.common.collect.ImmutableList; @@ -49,7 +49,7 @@ private RowHashCodeOperator() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type type = boundVariables.getTypeVariable("T"); return new BuiltInScalarFunctionImplementation( diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowIndeterminateOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowIndeterminateOperator.java index 56dc12a663c6..509a9fe45f9c 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowIndeterminateOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowIndeterminateOperator.java @@ -25,7 +25,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.sql.gen.CachedInstanceBinder; @@ -69,11 +69,11 @@ private RowIndeterminateOperator() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { checkArgument(arity == 1, "Expected arity to be 1"); Type type = boundVariables.getTypeVariable("T"); - Class indeterminateOperatorClass = generateIndeterminate(type, functionManager); + Class indeterminateOperatorClass = generateIndeterminate(type, functionAndTypeManager); MethodHandle indeterminateMethod = methodHandle(indeterminateOperatorClass, "indeterminate", type.getJavaType(), boolean.class); return new BuiltInScalarFunctionImplementation( false, @@ -81,7 +81,7 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab indeterminateMethod); } - private static Class generateIndeterminate(Type type, FunctionManager functionManager) + private static Class generateIndeterminate(Type type, FunctionAndTypeManager functionAndTypeManager) { CallSiteBinder binder = new CallSiteBinder(); @@ -131,13 +131,13 @@ private static Class generateIndeterminate(Type type, FunctionManager functio .push(true) .gotoLabel(end)); - FunctionHandle functionHandle = functionManager.resolveOperator(INDETERMINATE, fromTypes(fieldTypes.get(i))); + FunctionHandle functionHandle = functionAndTypeManager.resolveOperator(INDETERMINATE, fromTypes(fieldTypes.get(i))); - BuiltInScalarFunctionImplementation function = functionManager.getBuiltInScalarFunctionImplementation(functionHandle); + BuiltInScalarFunctionImplementation function = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionHandle); BytecodeExpression element = constantType(binder, fieldTypes.get(i)).getValue(value, constantInt(i)); ifNullField.ifFalse(new IfStatement("if the field is not null but indeterminate...") - .condition(invokeFunction(scope, cachedInstanceBinder, functionManager.getFunctionMetadata(functionHandle).getName().getFunctionName(), function, element)) + .condition(invokeFunction(scope, cachedInstanceBinder, functionAndTypeManager.getFunctionMetadata(functionHandle).getName().getFunctionName(), function, element)) .ifTrue(new BytecodeBlock() .push(true) .gotoLabel(end))); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowLessThanOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowLessThanOperator.java index e1b77fd4a775..838f0696131a 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowLessThanOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowLessThanOperator.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.google.common.collect.ImmutableList; import java.lang.invoke.MethodHandle; @@ -41,7 +41,7 @@ private RowLessThanOperator() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type type = boundVariables.getTypeVariable("T"); return new BuiltInScalarFunctionImplementation( @@ -49,7 +49,7 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab ImmutableList.of( valueTypeArgumentProperty(RETURN_NULL_ON_NULL), valueTypeArgumentProperty(RETURN_NULL_ON_NULL)), - METHOD_HANDLE.bindTo(type).bindTo(getMethodHandles((RowType) type, functionManager, LESS_THAN))); + METHOD_HANDLE.bindTo(type).bindTo(getMethodHandles((RowType) type, functionAndTypeManager, LESS_THAN))); } public static boolean less( diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowLessThanOrEqualOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowLessThanOrEqualOperator.java index 57cb3f5f2c7c..a31dbdc26c77 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowLessThanOrEqualOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowLessThanOrEqualOperator.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.google.common.collect.ImmutableList; import java.lang.invoke.MethodHandle; @@ -42,7 +42,7 @@ private RowLessThanOrEqualOperator() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type type = boundVariables.getTypeVariable("T"); return new BuiltInScalarFunctionImplementation( @@ -50,7 +50,7 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab ImmutableList.of( valueTypeArgumentProperty(RETURN_NULL_ON_NULL), valueTypeArgumentProperty(RETURN_NULL_ON_NULL)), - METHOD_HANDLE.bindTo(type).bindTo(getMethodHandles((RowType) type, functionManager, LESS_THAN))); + METHOD_HANDLE.bindTo(type).bindTo(getMethodHandles((RowType) type, functionAndTypeManager, LESS_THAN))); } public static boolean lessOrEqual( diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowNotEqualOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowNotEqualOperator.java index 0795c1b74f3c..12d72dc8fddb 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowNotEqualOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowNotEqualOperator.java @@ -19,7 +19,7 @@ import com.facebook.presto.common.type.StandardTypes; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.google.common.collect.ImmutableList; @@ -49,7 +49,7 @@ private RowNotEqualOperator() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { RowType type = (RowType) boundVariables.getTypeVariable("T"); return new BuiltInScalarFunctionImplementation( @@ -59,7 +59,7 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab valueTypeArgumentProperty(RETURN_NULL_ON_NULL)), METHOD_HANDLE .bindTo(type) - .bindTo(RowEqualOperator.resolveFieldEqualOperators(type, functionManager))); + .bindTo(RowEqualOperator.resolveFieldEqualOperators(type, functionAndTypeManager))); } @UsedByGeneratedCode diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowToJsonCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowToJsonCast.java index 8a6455176f5c..53f071daaa88 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowToJsonCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowToJsonCast.java @@ -21,7 +21,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.util.JsonUtil.JsonGeneratorWriter; import com.fasterxml.jackson.core.JsonGenerator; @@ -65,7 +65,7 @@ private RowToJsonCast() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { checkArgument(arity == 1, "Expected arity to be 1"); Type type = boundVariables.getTypeVariable("T"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowToRowCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowToRowCast.java index 61c439313d8c..39d002a0ffbf 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowToRowCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowToRowCast.java @@ -29,7 +29,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; import com.facebook.presto.metadata.CastType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlOperator; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.StandardErrorCode; @@ -77,7 +77,7 @@ private RowToRowCast() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { checkArgument(arity == 1, "Expected arity to be 1"); Type fromType = boundVariables.getTypeVariable("F"); @@ -85,7 +85,7 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab if (fromType.getTypeParameters().size() != toType.getTypeParameters().size()) { throw new PrestoException(StandardErrorCode.INVALID_FUNCTION_ARGUMENT, "the size of fromType and toType must match"); } - Class castOperatorClass = generateRowCast(fromType, toType, functionManager); + Class castOperatorClass = generateRowCast(fromType, toType, functionAndTypeManager); MethodHandle methodHandle = methodHandle(castOperatorClass, "castRow", SqlFunctionProperties.class, Block.class); return new BuiltInScalarFunctionImplementation( false, @@ -93,7 +93,7 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab methodHandle); } - private static Class generateRowCast(Type fromType, Type toType, FunctionManager functionManager) + private static Class generateRowCast(Type fromType, Type toType, FunctionAndTypeManager functionAndTypeManager) { List toTypes = toType.getTypeParameters(); List fromTypes = fromType.getTypeParameters(); @@ -141,8 +141,8 @@ private static Class generateRowCast(Type fromType, Type toType, FunctionMana // loop through to append member blocks for (int i = 0; i < toTypes.size(); i++) { - FunctionHandle functionHandle = functionManager.lookupCast(CastType.CAST, fromTypes.get(i).getTypeSignature(), toTypes.get(i).getTypeSignature()); - BuiltInScalarFunctionImplementation function = functionManager.getBuiltInScalarFunctionImplementation(functionHandle); + FunctionHandle functionHandle = functionAndTypeManager.lookupCast(CastType.CAST, fromTypes.get(i).getTypeSignature(), toTypes.get(i).getTypeSignature()); + BuiltInScalarFunctionImplementation function = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionHandle); Type currentFromType = fromTypes.get(i); if (currentFromType.equals(UNKNOWN)) { body.append(singleRowBlockWriter.invoke("appendNull", BlockBuilder.class).pop()); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/TryCastFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/TryCastFunction.java index d2eff538ac2e..c5730c64cfd8 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/TryCastFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/TryCastFunction.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.ArgumentProperty; import com.facebook.presto.spi.function.FunctionHandle; @@ -77,7 +77,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type fromType = boundVariables.getTypeVariable("F"); Type toType = boundVariables.getTypeVariable("T"); @@ -87,8 +87,8 @@ public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariab MethodHandle tryCastHandle; // the resulting method needs to return a boxed type - FunctionHandle functionHandle = functionManager.lookupCast(CAST, fromType.getTypeSignature(), toType.getTypeSignature()); - BuiltInScalarFunctionImplementation implementation = functionManager.getBuiltInScalarFunctionImplementation(functionHandle); + FunctionHandle functionHandle = functionAndTypeManager.lookupCast(CAST, fromType.getTypeSignature(), toType.getTypeSignature()); + BuiltInScalarFunctionImplementation implementation = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionHandle); argumentProperties = ImmutableList.of(implementation.getArgumentProperty(0)); MethodHandle coercion = implementation.getMethodHandle(); coercion = coercion.asType(methodType(returnType, coercion.type())); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ZipFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ZipFunction.java index 53bdf5101bd2..cae69050223b 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ZipFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ZipFunction.java @@ -22,7 +22,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignature; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.ArgumentProperty; import com.facebook.presto.spi.function.FunctionKind; @@ -100,7 +100,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { List types = this.typeParameters.stream().map(boundVariables::getTypeVariable).collect(toImmutableList()); List argumentProperties = nCopies(types.size(), valueTypeArgumentProperty(RETURN_NULL_ON_NULL)); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ZipWithFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ZipWithFunction.java index 16a18fe60b96..2af719a9975d 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ZipWithFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ZipWithFunction.java @@ -21,7 +21,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.spi.function.FunctionKind; import com.facebook.presto.spi.function.Signature; @@ -84,7 +84,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type leftElementType = boundVariables.getTypeVariable("T"); Type rightElementType = boundVariables.getTypeVariable("U"); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/annotations/ParametricScalarImplementation.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/annotations/ParametricScalarImplementation.java index 3c5e5ee4ec26..f8b401aecae9 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/annotations/ParametricScalarImplementation.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/annotations/ParametricScalarImplementation.java @@ -20,7 +20,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignature; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.ParametricImplementation; import com.facebook.presto.operator.annotations.FunctionsParserHelper; import com.facebook.presto.operator.annotations.ImplementationDependency; @@ -118,7 +118,7 @@ private ParametricScalarImplementation( } } - public Optional specialize(Signature boundSignature, BoundVariables boundVariables, TypeManager typeManager, FunctionManager functionManager) + public Optional specialize(Signature boundSignature, BoundVariables boundVariables, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { List implementationChoices = new ArrayList<>(); for (Map.Entry> entry : specializedTypeParameters.entrySet()) { @@ -151,9 +151,9 @@ public Optional specialize(Signature boundS } for (ParametricScalarImplementationChoice choice : choices) { - MethodHandle boundMethodHandle = bindDependencies(choice.getMethodHandle(), choice.getDependencies(), boundVariables, typeManager, functionManager); + MethodHandle boundMethodHandle = bindDependencies(choice.getMethodHandle(), choice.getDependencies(), boundVariables, typeManager, functionAndTypeManager); Optional boundConstructor = choice.getConstructor().map(constructor -> { - MethodHandle result = bindDependencies(constructor, choice.getConstructorDependencies(), boundVariables, typeManager, functionManager); + MethodHandle result = bindDependencies(constructor, choice.getConstructorDependencies(), boundVariables, typeManager, functionAndTypeManager); checkCondition( result.type().parameterList().isEmpty(), FUNCTION_IMPLEMENTATION_ERROR, diff --git a/presto-main/src/main/java/com/facebook/presto/operator/window/SqlWindowFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/window/SqlWindowFunction.java index 1c8c87b0ee7c..fce7db2b1174 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/window/SqlWindowFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/window/SqlWindowFunction.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; import com.facebook.presto.metadata.BuiltInFunction; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.Signature; import com.facebook.presto.spi.function.SqlFunctionVisibility; @@ -57,7 +57,7 @@ public String getDescription() return supplier.getDescription(); } - public WindowFunctionSupplier specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public WindowFunctionSupplier specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { return supplier; } 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 808f4812e079..85fc7415e979 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 @@ -214,7 +214,7 @@ public void installPlugin(Plugin plugin) for (FunctionNamespaceManagerFactory functionNamespaceManagerFactory : plugin.getFunctionNamespaceManagerFactories()) { log.info("Registering function namespace manager %s", functionNamespaceManagerFactory.getName()); - metadata.getFunctionManager().addFunctionNamespaceFactory(functionNamespaceManagerFactory); + metadata.getFunctionAndTypeManager().addFunctionNamespaceFactory(functionNamespaceManagerFactory); } for (SessionPropertyConfigurationManagerFactory sessionConfigFactory : plugin.getSessionPropertyConfigurationManagerFactories()) { 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 509a765cef2e..a0e0c859eb6d 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 @@ -78,7 +78,7 @@ import com.facebook.presto.metadata.ConnectorMetadataUpdaterManager; import com.facebook.presto.metadata.DiscoveryNodeManager; import com.facebook.presto.metadata.ForNodeManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.HandleJsonModule; import com.facebook.presto.metadata.InternalNodeManager; import com.facebook.presto.metadata.Metadata; @@ -423,7 +423,7 @@ protected void setup(Binder binder) configBinder(binder).bindConfig(StaticCatalogStoreConfig.class); binder.bind(StaticFunctionNamespaceStore.class).in(Scopes.SINGLETON); configBinder(binder).bindConfig(StaticFunctionNamespaceStoreConfig.class); - binder.bind(FunctionManager.class).in(Scopes.SINGLETON); + binder.bind(FunctionAndTypeManager.class).in(Scopes.SINGLETON); binder.bind(MetadataManager.class).in(Scopes.SINGLETON); binder.bind(Metadata.class).to(MetadataManager.class).in(Scopes.SINGLETON); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/InterpretedFunctionInvoker.java b/presto-main/src/main/java/com/facebook/presto/sql/InterpretedFunctionInvoker.java index 4445c5c21dc6..2b92956f2a1a 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/InterpretedFunctionInvoker.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/InterpretedFunctionInvoker.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.NotSupportedException; import com.facebook.presto.common.function.SqlFunctionProperties; import com.facebook.presto.common.type.TimeZoneNotSupportedException; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.ArgumentProperty; import com.facebook.presto.spi.PrestoException; @@ -40,11 +40,11 @@ public class InterpretedFunctionInvoker { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public InterpretedFunctionInvoker(FunctionManager functionManager) + public InterpretedFunctionInvoker(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "registry is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "registry is null"); } public Object invoke(FunctionHandle functionHandle, SqlFunctionProperties properties, Object... arguments) @@ -54,7 +54,7 @@ public Object invoke(FunctionHandle functionHandle, SqlFunctionProperties proper public Object invoke(FunctionHandle functionHandle, SqlFunctionProperties properties, List arguments) { - return invoke(functionManager.getBuiltInScalarFunctionImplementation(functionHandle), properties, arguments); + return invoke(functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionHandle), properties, arguments); } /** diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/AggregationAnalyzer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/AggregationAnalyzer.java index a27b5211adbc..25384ca34767 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/AggregationAnalyzer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/AggregationAnalyzer.java @@ -152,7 +152,7 @@ private AggregationAnalyzer(List groupByExpressions, Scope sourceSco this.orderByScope = orderByScope; this.metadata = metadata; this.analysis = analysis; - this.functionResolution = new FunctionResolution(metadata.getFunctionManager()); + this.functionResolution = new FunctionResolution(metadata.getFunctionAndTypeManager()); this.expressions = groupByExpressions.stream() .map(e -> ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(analysis.getParameters()), e)) .collect(toImmutableList()); @@ -325,14 +325,14 @@ protected Boolean visitInPredicate(InPredicate node, Void context) @Override protected Boolean visitFunctionCall(FunctionCall node, Void context) { - if (metadata.getFunctionManager().getFunctionMetadata(analysis.getFunctionHandle(node)).getFunctionKind() == AGGREGATE) { + if (metadata.getFunctionAndTypeManager().getFunctionMetadata(analysis.getFunctionHandle(node)).getFunctionKind() == AGGREGATE) { if (functionResolution.isCountFunction(analysis.getFunctionHandle(node)) && node.isDistinct()) { warningCollector.add(new PrestoWarning( PERFORMANCE_WARNING, "COUNT(DISTINCT xxx) can be a very expensive operation when the cardinality is high for xxx. In most scenarios, using approx_distinct instead would be enough")); } if (!node.getWindow().isPresent()) { - List aggregateFunctions = extractAggregateFunctions(analysis.getFunctionHandles(), node.getArguments(), metadata.getFunctionManager()); + List aggregateFunctions = extractAggregateFunctions(analysis.getFunctionHandles(), node.getArguments(), metadata.getFunctionAndTypeManager()); List windowFunctions = extractWindowFunctions(node.getArguments()); if (!aggregateFunctions.isEmpty()) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/Analyzer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/Analyzer.java index 7ec1748826a8..dc230a266e6b 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/Analyzer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/Analyzer.java @@ -14,7 +14,7 @@ package com.facebook.presto.sql.analyzer; import com.facebook.presto.Session; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.security.AccessControl; import com.facebook.presto.spi.WarningCollector; @@ -92,9 +92,9 @@ public Analysis analyze(Statement statement, boolean isDescribe) return analysis; } - static void verifyNoAggregateWindowOrGroupingFunctions(Map, FunctionHandle> functionHandles, FunctionManager functionManager, Expression predicate, String clause) + static void verifyNoAggregateWindowOrGroupingFunctions(Map, FunctionHandle> functionHandles, FunctionAndTypeManager functionAndTypeManager, Expression predicate, String clause) { - List aggregates = extractAggregateFunctions(functionHandles, ImmutableList.of(predicate), functionManager); + List aggregates = extractAggregateFunctions(functionHandles, ImmutableList.of(predicate), functionAndTypeManager); List windowExpressions = extractWindowFunctions(ImmutableList.of(predicate)); @@ -110,9 +110,9 @@ static void verifyNoAggregateWindowOrGroupingFunctions(Map } } - static void verifyNoExternalFunctions(Map, FunctionHandle> functionHandles, FunctionManager functionManager, Expression predicate, String clause) + static void verifyNoExternalFunctions(Map, FunctionHandle> functionHandles, FunctionAndTypeManager functionAndTypeManager, Expression predicate, String clause) { - List externalFunctions = extractExternalFunctions(functionHandles, ImmutableList.of(predicate), functionManager); + List externalFunctions = extractExternalFunctions(functionHandles, ImmutableList.of(predicate), functionAndTypeManager); if (!externalFunctions.isEmpty()) { throw new SemanticException(NOT_SUPPORTED, predicate, "External functions in %s is not supported: %s", clause, externalFunctions); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java index d0406a200967..e33987be8837 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java @@ -27,7 +27,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.common.type.VarcharType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.OperatorNotFoundException; import com.facebook.presto.metadata.QualifiedObjectName; @@ -138,7 +138,7 @@ import static com.facebook.presto.common.type.VarbinaryType.VARBINARY; import static com.facebook.presto.common.type.VarcharType.VARCHAR; import static com.facebook.presto.metadata.CastType.CAST; -import static com.facebook.presto.metadata.FunctionManager.qualifyFunctionName; +import static com.facebook.presto.metadata.FunctionAndTypeManager.qualifyFunctionName; import static com.facebook.presto.sql.NodeUtils.getSortItemsFromOrderBy; import static com.facebook.presto.sql.analyzer.Analyzer.verifyNoAggregateWindowOrGroupingFunctions; import static com.facebook.presto.sql.analyzer.Analyzer.verifyNoExternalFunctions; @@ -179,7 +179,7 @@ public class ExpressionAnalyzer private static final int MAX_NUMBER_GROUPING_ARGUMENTS_BIGINT = 63; private static final int MAX_NUMBER_GROUPING_ARGUMENTS_INTEGER = 31; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final TypeManager typeManager; private final Function statementAnalyzerFactory; private final TypeProvider symbolTypes; @@ -205,7 +205,7 @@ public class ExpressionAnalyzer private final WarningCollector warningCollector; private ExpressionAnalyzer( - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, TypeManager typeManager, Function statementAnalyzerFactory, Optional transactionId, @@ -215,7 +215,7 @@ private ExpressionAnalyzer( WarningCollector warningCollector, boolean isDescribe) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.statementAnalyzerFactory = requireNonNull(statementAnalyzerFactory, "statementAnalyzerFactory is null"); this.transactionId = requireNonNull(transactionId, "transactionId is null"); @@ -773,7 +773,7 @@ protected Type visitGenericLiteral(GenericLiteral node, StackableAstVisitorConte if (!JSON.equals(type)) { try { - functionManager.lookupCast(CAST, VARCHAR.getTypeSignature(), type.getTypeSignature()); + functionAndTypeManager.lookupCast(CAST, VARCHAR.getTypeSignature(), type.getTypeSignature()); } catch (IllegalArgumentException e) { throw new SemanticException(TYPE_MISMATCH, node, "No literal form for type %s", type); @@ -907,7 +907,7 @@ protected Type visitFunctionCall(FunctionCall node, StackableAstVisitorContext { ExpressionAnalyzer innerExpressionAnalyzer = new ExpressionAnalyzer( - functionManager, + functionAndTypeManager, typeManager, statementAnalyzerFactory, transactionId, @@ -923,8 +923,8 @@ protected Type visitFunctionCall(FunctionCall node, StackableAstVisitorContext argumentTypes = argumentTypesBuilder.build(); - FunctionHandle function = resolveFunction(transactionId, node, argumentTypes, functionManager); - FunctionMetadata functionMetadata = functionManager.getFunctionMetadata(function); + FunctionHandle function = resolveFunction(transactionId, node, argumentTypes, functionAndTypeManager); + FunctionMetadata functionMetadata = functionAndTypeManager.getFunctionMetadata(function); if (node.getOrderBy().isPresent()) { for (SortItem sortItem : node.getOrderBy().get().getSortItems()) { @@ -1068,7 +1068,7 @@ public Type visitCast(Cast node, StackableAstVisitorContext context) Type value = process(node.getExpression(), context); if (!value.equals(UNKNOWN) && !node.isTypeOnly()) { try { - functionManager.lookupCast(CAST, value.getTypeSignature(), type.getTypeSignature()); + functionAndTypeManager.lookupCast(CAST, value.getTypeSignature(), type.getTypeSignature()); } catch (OperatorNotFoundException e) { throw new SemanticException(TYPE_MISMATCH, node, "Cannot cast %s to %s", value, type); @@ -1305,7 +1305,7 @@ private Type getOperator(StackableAstVisitorContext context, Expression FunctionMetadata operatorMetadata; try { - operatorMetadata = functionManager.getFunctionMetadata(functionManager.resolveOperator(operatorType, fromTypes(argumentTypes.build()))); + operatorMetadata = functionAndTypeManager.getFunctionMetadata(functionAndTypeManager.resolveOperator(operatorType, fromTypes(argumentTypes.build()))); } catch (OperatorNotFoundException e) { throw new SemanticException(TYPE_MISMATCH, node, "%s", e.getMessage()); @@ -1487,10 +1487,10 @@ public List getFunctionInputTypes() } } - public static FunctionHandle resolveFunction(Optional transactionId, FunctionCall node, List argumentTypes, FunctionManager functionManager) + public static FunctionHandle resolveFunction(Optional transactionId, FunctionCall node, List argumentTypes, FunctionAndTypeManager functionAndTypeManager) { try { - return functionManager.resolveFunction(transactionId, qualifyFunctionName(node.getName()), argumentTypes); + return functionAndTypeManager.resolveFunction(transactionId, qualifyFunctionName(node.getName()), argumentTypes); } catch (PrestoException e) { if (e.getErrorCode().getCode() == StandardErrorCode.FUNCTION_NOT_FOUND.toErrorCode().getCode()) { @@ -1617,7 +1617,7 @@ public static ExpressionAnalysis analyzeSqlFunctionExpression( Map argumentTypes) { ExpressionAnalyzer analyzer = ExpressionAnalyzer.createWithoutSubqueries( - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), Optional.empty(), sqlFunctionProperties, @@ -1658,7 +1658,7 @@ private static ExpressionAnalyzer create( WarningCollector warningCollector) { return new ExpressionAnalyzer( - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), node -> new StatementAnalyzer(analysis, metadata, sqlParser, accessControl, session, warningCollector), session.getTransactionId(), @@ -1672,7 +1672,7 @@ private static ExpressionAnalyzer create( public static ExpressionAnalyzer createConstantAnalyzer(Metadata metadata, Session session, List parameters, WarningCollector warningCollector) { return createWithoutSubqueries( - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), session, parameters, @@ -1685,7 +1685,7 @@ public static ExpressionAnalyzer createConstantAnalyzer(Metadata metadata, Sessi public static ExpressionAnalyzer createConstantAnalyzer(Metadata metadata, Session session, List parameters, WarningCollector warningCollector, boolean isDescribe) { return createWithoutSubqueries( - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), session, parameters, @@ -1696,7 +1696,7 @@ public static ExpressionAnalyzer createConstantAnalyzer(Metadata metadata, Sessi } public static ExpressionAnalyzer createWithoutSubqueries( - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, TypeManager typeManager, Session session, List parameters, @@ -1706,7 +1706,7 @@ public static ExpressionAnalyzer createWithoutSubqueries( boolean isDescribe) { return createWithoutSubqueries( - functionManager, + functionAndTypeManager, typeManager, session, TypeProvider.empty(), @@ -1717,7 +1717,7 @@ public static ExpressionAnalyzer createWithoutSubqueries( } public static ExpressionAnalyzer createWithoutSubqueries( - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, TypeManager typeManager, Session session, TypeProvider symbolTypes, @@ -1727,7 +1727,7 @@ public static ExpressionAnalyzer createWithoutSubqueries( boolean isDescribe) { return createWithoutSubqueries( - functionManager, + functionAndTypeManager, typeManager, session.getTransactionId(), session.getSqlFunctionProperties(), @@ -1739,7 +1739,7 @@ public static ExpressionAnalyzer createWithoutSubqueries( } public static ExpressionAnalyzer createWithoutSubqueries( - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, TypeManager typeManager, Optional transactionId, SqlFunctionProperties sqlFunctionProperties, @@ -1750,7 +1750,7 @@ public static ExpressionAnalyzer createWithoutSubqueries( boolean isDescribe) { return new ExpressionAnalyzer( - functionManager, + functionAndTypeManager, typeManager, node -> { throw statementAnalyzerRejection.apply(node); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionTreeUtils.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionTreeUtils.java index 81c35499bec3..c2b5276d40d7 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionTreeUtils.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionTreeUtils.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.type.EnumType; import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.sql.tree.ComparisonExpression; import com.facebook.presto.sql.tree.DefaultExpressionTraversalVisitor; @@ -47,9 +47,9 @@ public final class ExpressionTreeUtils { private ExpressionTreeUtils() {} - static List extractAggregateFunctions(Map, FunctionHandle> functionHandles, Iterable nodes, FunctionManager functionManager) + static List extractAggregateFunctions(Map, FunctionHandle> functionHandles, Iterable nodes, FunctionAndTypeManager functionAndTypeManager) { - return extractExpressions(nodes, FunctionCall.class, isAggregationPredicate(functionHandles, functionManager)); + return extractExpressions(nodes, FunctionCall.class, isAggregationPredicate(functionHandles, functionAndTypeManager)); } static List extractWindowFunctions(Iterable nodes) @@ -57,9 +57,9 @@ static List extractWindowFunctions(Iterable nodes) return extractExpressions(nodes, FunctionCall.class, ExpressionTreeUtils::isWindowFunction); } - static List extractExternalFunctions(Map, FunctionHandle> functionHandles, Iterable nodes, FunctionManager functionManager) + static List extractExternalFunctions(Map, FunctionHandle> functionHandles, Iterable nodes, FunctionAndTypeManager functionAndTypeManager) { - return extractExpressions(nodes, FunctionCall.class, isExternalFunctionPredicate(functionHandles, functionManager)); + return extractExpressions(nodes, FunctionCall.class, isExternalFunctionPredicate(functionHandles, functionAndTypeManager)); } public static List extractExpressions( @@ -69,9 +69,9 @@ public static List extractExpressions( return extractExpressions(nodes, clazz, alwaysTrue()); } - private static Predicate isAggregationPredicate(Map, FunctionHandle> functionHandles, FunctionManager functionManager) + private static Predicate isAggregationPredicate(Map, FunctionHandle> functionHandles, FunctionAndTypeManager functionAndTypeManager) { - return functionCall -> (functionManager.getFunctionMetadata(functionHandles.get(NodeRef.of(functionCall))).getFunctionKind() == AGGREGATE || functionCall.getFilter().isPresent()) + return functionCall -> (functionAndTypeManager.getFunctionMetadata(functionHandles.get(NodeRef.of(functionCall))).getFunctionKind() == AGGREGATE || functionCall.getFilter().isPresent()) && !functionCall.getWindow().isPresent() || functionCall.getOrderBy().isPresent(); } @@ -81,9 +81,9 @@ private static boolean isWindowFunction(FunctionCall functionCall) return functionCall.getWindow().isPresent(); } - private static Predicate isExternalFunctionPredicate(Map, FunctionHandle> functionHandles, FunctionManager functionManager) + private static Predicate isExternalFunctionPredicate(Map, FunctionHandle> functionHandles, FunctionAndTypeManager functionAndTypeManager) { - return functionCall -> functionManager.getFunctionMetadata(functionHandles.get(NodeRef.of(functionCall))).getImplementationType().isExternal(); + return functionCall -> functionAndTypeManager.getFunctionMetadata(functionHandles.get(NodeRef.of(functionCall))).getImplementationType().isExternal(); } private static List extractExpressions( diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/QueryExplainer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/QueryExplainer.java index 3b3a9fe459a9..57e1fa4edd61 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/QueryExplainer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/QueryExplainer.java @@ -126,10 +126,10 @@ public String getPlan(Session session, Statement statement, Type planType, List< switch (planType) { case LOGICAL: Plan plan = getLogicalPlan(session, statement, parameters, warningCollector); - return PlanPrinter.textLogicalPlan(plan.getRoot(), plan.getTypes(), metadata.getFunctionManager(), plan.getStatsAndCosts(), session, 0, verbose); + return PlanPrinter.textLogicalPlan(plan.getRoot(), plan.getTypes(), metadata.getFunctionAndTypeManager(), plan.getStatsAndCosts(), session, 0, verbose); case DISTRIBUTED: SubPlan subPlan = getDistributedPlan(session, statement, parameters, warningCollector); - return PlanPrinter.textDistributedPlan(subPlan, metadata.getFunctionManager(), session, verbose); + return PlanPrinter.textDistributedPlan(subPlan, metadata.getFunctionAndTypeManager(), session, verbose); case IO: return IOPlanPrinter.textIOPlan(getLogicalPlan(session, statement, parameters, warningCollector).getRoot(), metadata, session); } @@ -152,10 +152,10 @@ public String getGraphvizPlan(Session session, Statement statement, Type planTyp switch (planType) { case LOGICAL: Plan plan = getLogicalPlan(session, statement, parameters, warningCollector); - return graphvizLogicalPlan(plan.getRoot(), plan.getTypes(), session, metadata.getFunctionManager()); + return graphvizLogicalPlan(plan.getRoot(), plan.getTypes(), session, metadata.getFunctionAndTypeManager()); case DISTRIBUTED: SubPlan subPlan = getDistributedPlan(session, statement, parameters, warningCollector); - return graphvizDistributedPlan(subPlan, session, metadata.getFunctionManager()); + return graphvizDistributedPlan(subPlan, session, metadata.getFunctionAndTypeManager()); } throw new IllegalArgumentException("Unhandled plan type: " + planType); } @@ -175,7 +175,7 @@ public String getJsonPlan(Session session, Statement statement, Type planType, L return textIOPlan(plan.getRoot(), metadata, session); case LOGICAL: plan = getLogicalPlan(session, statement, parameters, warningCollector); - return jsonLogicalPlan(plan.getRoot(), plan.getTypes(), metadata.getFunctionManager(), plan.getStatsAndCosts(), session); + return jsonLogicalPlan(plan.getRoot(), plan.getTypes(), metadata.getFunctionAndTypeManager(), plan.getStatsAndCosts(), session); case DISTRIBUTED: SubPlan subPlan = getDistributedPlan(session, statement, parameters, warningCollector); return jsonDistributedPlan(subPlan); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java index 19714111d606..dfc205e57a68 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java @@ -598,8 +598,8 @@ protected Scope visitCreateFunction(CreateFunction node, Optional scope) throw new SemanticException(TYPE_MISMATCH, node, "Function implementation type '%s' does not match declared return type '%s'", bodyType, returnType); } - verifyNoAggregateWindowOrGroupingFunctions(analysis.getFunctionHandles(), metadata.getFunctionManager(), returnExpression, "CREATE FUNCTION body"); - verifyNoExternalFunctions(analysis.getFunctionHandles(), metadata.getFunctionManager(), returnExpression, "CREATE FUNCTION body"); + verifyNoAggregateWindowOrGroupingFunctions(analysis.getFunctionHandles(), metadata.getFunctionAndTypeManager(), returnExpression, "CREATE FUNCTION body"); + verifyNoExternalFunctions(analysis.getFunctionHandles(), metadata.getFunctionAndTypeManager(), returnExpression, "CREATE FUNCTION body"); // TODO: Check body contains no SQL invoked functions } @@ -1151,7 +1151,7 @@ protected Scope visitQuerySpecification(QuerySpecification node, Optional // and when aggregation is present, ORDER BY expressions should only be resolvable against // output scope, group by expressions and aggregation expressions. List orderByGroupingOperations = extractExpressions(orderByExpressions, GroupingOperation.class); - List orderByAggregations = extractAggregateFunctions(analysis.getFunctionHandles(), orderByExpressions, metadata.getFunctionManager()); + List orderByAggregations = extractAggregateFunctions(analysis.getFunctionHandles(), orderByExpressions, metadata.getFunctionAndTypeManager()); computeAndAssignOrderByScopeWithAggregation(node.getOrderBy().get(), sourceScope, outputScope, orderByAggregations, groupByExpressions, orderByGroupingOperations); } @@ -1320,7 +1320,7 @@ else if (criteria instanceof JoinOn) { analysis.addCoercion(expression, BOOLEAN, false); } - verifyNoAggregateWindowOrGroupingFunctions(analysis.getFunctionHandles(), metadata.getFunctionManager(), expression, "JOIN clause"); + verifyNoAggregateWindowOrGroupingFunctions(analysis.getFunctionHandles(), metadata.getFunctionAndTypeManager(), expression, "JOIN clause"); analysis.recordSubqueries(node, expressionAnalysis); analysis.setJoinCriteria(node, expression); @@ -1357,7 +1357,7 @@ private Scope analyzeJoinUsing(Join node, List columns, Optional ou private List analyzeWindowFunctions(QuerySpecification node, List expressions) { for (Expression expression : expressions) { - new WindowFunctionValidator(metadata.getFunctionManager()).process(expression, analysis); + new WindowFunctionValidator(metadata.getFunctionAndTypeManager()).process(expression, analysis); } List windowFunctions = extractWindowFunctions(expressions); @@ -1527,7 +1527,7 @@ private List analyzeWindowFunctions(QuerySpecification node, List< analyzeWindowFrame(window.getFrame().get()); } - FunctionKind kind = metadata.getFunctionManager().getFunctionMetadata(analysis.getFunctionHandle(windowFunction)).getFunctionKind(); + FunctionKind kind = metadata.getFunctionAndTypeManager().getFunctionMetadata(analysis.getFunctionHandle(windowFunction)).getFunctionKind(); if (kind != AGGREGATE && kind != WINDOW) { throw new SemanticException(MUST_BE_WINDOW_FUNCTION, node, "Not a window function: %s", windowFunction.getName()); } @@ -1717,7 +1717,7 @@ private List analyzeGroupBy(QuerySpecification node, Scope scope, Li sets.add(ImmutableList.of(ImmutableSet.of(field))); } else { - verifyNoAggregateWindowOrGroupingFunctions(analysis.getFunctionHandles(), metadata.getFunctionManager(), column, "GROUP BY clause"); + verifyNoAggregateWindowOrGroupingFunctions(analysis.getFunctionHandles(), metadata.getFunctionAndTypeManager(), column, "GROUP BY clause"); analysis.recordSubqueries(node, analyzeExpression(column, scope)); complexExpressions.add(column); } @@ -1950,7 +1950,7 @@ public void analyzeWhere(Node node, Scope scope, Expression predicate) { ExpressionAnalysis expressionAnalysis = analyzeExpression(predicate, scope); - verifyNoAggregateWindowOrGroupingFunctions(analysis.getFunctionHandles(), metadata.getFunctionManager(), predicate, "WHERE clause"); + verifyNoAggregateWindowOrGroupingFunctions(analysis.getFunctionHandles(), metadata.getFunctionAndTypeManager(), predicate, "WHERE clause"); analysis.recordSubqueries(node, expressionAnalysis); @@ -1995,7 +1995,7 @@ private List analyzeAggregations( List outputExpressions, List orderByExpressions) { - List aggregates = extractAggregateFunctions(analysis.getFunctionHandles(), Iterables.concat(outputExpressions, orderByExpressions), metadata.getFunctionManager()); + List aggregates = extractAggregateFunctions(analysis.getFunctionHandles(), Iterables.concat(outputExpressions, orderByExpressions), metadata.getFunctionAndTypeManager()); analysis.setAggregates(node, aggregates); return aggregates; } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/WindowFunctionValidator.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/WindowFunctionValidator.java index f5597d8d30e8..b609b59c8bfa 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/WindowFunctionValidator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/WindowFunctionValidator.java @@ -13,7 +13,7 @@ */ package com.facebook.presto.sql.analyzer; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionMetadata; import com.facebook.presto.sql.tree.DefaultExpressionTraversalVisitor; import com.facebook.presto.sql.tree.FunctionCall; @@ -25,11 +25,11 @@ class WindowFunctionValidator extends DefaultExpressionTraversalVisitor { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public WindowFunctionValidator(FunctionManager functionManager) + public WindowFunctionValidator(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override @@ -37,7 +37,7 @@ protected Void visitFunctionCall(FunctionCall functionCall, Analysis analysis) { requireNonNull(analysis, "analysis is null"); - FunctionMetadata functionMetadata = functionManager.getFunctionMetadata(analysis.getFunctionHandle(functionCall)); + FunctionMetadata functionMetadata = functionAndTypeManager.getFunctionMetadata(analysis.getFunctionHandle(functionCall)); if (functionMetadata != null && functionMetadata.getFunctionKind() == WINDOW && !functionCall.getWindow().isPresent()) { throw new SemanticException(WINDOW_REQUIRES_OVER, functionCall, "Window function %s requires an OVER clause", functionMetadata.getName()); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/BytecodeGeneratorContext.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/BytecodeGeneratorContext.java index 2b7cb0116f8d..40addd493315 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/BytecodeGeneratorContext.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/BytecodeGeneratorContext.java @@ -17,7 +17,7 @@ import com.facebook.presto.bytecode.FieldDefinition; import com.facebook.presto.bytecode.Scope; import com.facebook.presto.bytecode.Variable; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation; import com.facebook.presto.spi.relation.RowExpression; import com.facebook.presto.sql.gen.BytecodeUtils.OutputBlockVariableAndType; @@ -34,7 +34,7 @@ public class BytecodeGeneratorContext private final Scope scope; private final CallSiteBinder callSiteBinder; private final CachedInstanceBinder cachedInstanceBinder; - private final FunctionManager manager; + private final FunctionAndTypeManager manager; private final Variable wasNull; public BytecodeGeneratorContext( @@ -42,7 +42,7 @@ public BytecodeGeneratorContext( Scope scope, CallSiteBinder callSiteBinder, CachedInstanceBinder cachedInstanceBinder, - FunctionManager manager) + FunctionAndTypeManager manager) { requireNonNull(rowExpressionCompiler, "bytecodeGenerator is null"); requireNonNull(cachedInstanceBinder, "cachedInstanceBinder is null"); @@ -78,7 +78,7 @@ public BytecodeNode generate(RowExpression expression, Optional output return rowExpressionCompiler.compile(expression, scope, outputBlockVariable, lambdaInterface); } - public FunctionManager getFunctionManager() + public FunctionAndTypeManager getFunctionManager() { return manager; } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/FunctionCallCodeGenerator.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/FunctionCallCodeGenerator.java index 6625384c06dd..37ae88396d42 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/FunctionCallCodeGenerator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/FunctionCallCodeGenerator.java @@ -16,7 +16,7 @@ import com.facebook.presto.bytecode.BytecodeNode; import com.facebook.presto.bytecode.Variable; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.relation.RowExpression; @@ -32,9 +32,9 @@ public class FunctionCallCodeGenerator { public BytecodeNode generateCall(FunctionHandle functionHandle, BytecodeGeneratorContext context, Type returnType, List arguments, Optional outputBlockVariable) { - FunctionManager functionManager = context.getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = context.getFunctionManager(); - BuiltInScalarFunctionImplementation function = functionManager.getBuiltInScalarFunctionImplementation(functionHandle); + BuiltInScalarFunctionImplementation function = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionHandle); List argumentsBytecode = new ArrayList<>(); for (int i = 0; i < arguments.size(); i++) { @@ -49,7 +49,7 @@ public BytecodeNode generateCall(FunctionHandle functionHandle, BytecodeGenerato } return context.generateCall( - functionManager.getFunctionMetadata(functionHandle).getName().getFunctionName(), + functionAndTypeManager.getFunctionMetadata(functionHandle).getName().getFunctionName(), function, argumentsBytecode, outputBlockVariable.map(variable -> new OutputBlockVariableAndType(variable, returnType))); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/InCodeGenerator.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/InCodeGenerator.java index fb2d385b08a5..7618e3f7684f 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/InCodeGenerator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/InCodeGenerator.java @@ -24,7 +24,7 @@ import com.facebook.presto.common.type.DateType; import com.facebook.presto.common.type.IntegerType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.relation.ConstantExpression; @@ -63,11 +63,11 @@ public class InCodeGenerator implements SpecialFormBytecodeGenerator { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public InCodeGenerator(FunctionManager functionManager) + public InCodeGenerator(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } enum SwitchGenerationCase @@ -220,7 +220,7 @@ public BytecodeNode generateExpression(BytecodeGeneratorContext generatorContext .append(switchBuilder.build()); break; case SET_CONTAINS: - Set constantValuesSet = toFastutilHashSet(constantValues, type, functionManager); + Set constantValuesSet = toFastutilHashSet(constantValues, type, functionAndTypeManager); Binding constant = generatorContext.getCallSiteBinder().bind(constantValuesSet, constantValuesSet.getClass()); switchBlock = new BytecodeBlock() diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinCompiler.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinCompiler.java index 96e1cbcb27f1..c81b04380adf 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinCompiler.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinCompiler.java @@ -36,7 +36,7 @@ import com.facebook.presto.common.type.BigintType; import com.facebook.presto.common.type.StandardTypes; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.operator.JoinHash; import com.facebook.presto.operator.JoinHashSupplier; @@ -91,7 +91,7 @@ public class JoinCompiler { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final boolean groupByUsesEqualTo; private final LoadingCache lookupSourceFactories = CacheBuilder.newBuilder() @@ -114,7 +114,7 @@ public LookupSourceSupplierFactory compileLookupSourceFactory(List argumentsBytecode = new ArrayList<>(); argumentsBytecode.add(generateInputReference(callSiteBinder, scope, type, leftBlock, leftBlockPosition)); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/LambdaBytecodeGenerator.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/LambdaBytecodeGenerator.java index 63a61b2e7a99..939bafc98cac 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/LambdaBytecodeGenerator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/LambdaBytecodeGenerator.java @@ -336,7 +336,7 @@ public static Class compileLambdaProvider(LambdaDefini scope, callSiteBinder, cachedInstanceBinder, - metadata.getFunctionManager()); + metadata.getFunctionAndTypeManager()); body.append( generateLambda( diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/NullIfCodeGenerator.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/NullIfCodeGenerator.java index 764e222183f6..8d6eea20c254 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/NullIfCodeGenerator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/NullIfCodeGenerator.java @@ -22,7 +22,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeSignature; import com.facebook.presto.metadata.CastType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.function.FunctionMetadata; @@ -65,9 +65,9 @@ public BytecodeNode generateExpression(BytecodeGeneratorContext generatorContext Type secondType = second.getType(); // if (equal(cast(first as ), cast(second as )) - FunctionManager functionManager = generatorContext.getFunctionManager(); - FunctionHandle equalFunction = functionManager.resolveOperator(EQUAL, fromTypes(firstType, secondType)); - FunctionMetadata equalFunctionMetadata = functionManager.getFunctionMetadata(equalFunction); + FunctionAndTypeManager functionAndTypeManager = generatorContext.getFunctionManager(); + FunctionHandle equalFunction = functionAndTypeManager.resolveOperator(EQUAL, fromTypes(firstType, secondType)); + FunctionMetadata equalFunctionMetadata = functionAndTypeManager.getFunctionMetadata(equalFunction); BuiltInScalarFunctionImplementation equalsFunction = generatorContext.getFunctionManager().getBuiltInScalarFunctionImplementation(equalFunction); BytecodeNode equalsCall = generatorContext.generateCall( EQUAL.name(), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/PageFunctionCompiler.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/PageFunctionCompiler.java index 763a71a52f5e..bd536f479a10 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/PageFunctionCompiler.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/PageFunctionCompiler.java @@ -141,7 +141,7 @@ public PageFunctionCompiler(Metadata metadata, CompilerConfig config) public PageFunctionCompiler(Metadata metadata, int expressionCacheSize) { this.metadata = requireNonNull(metadata, "metadata is null"); - this.determinismEvaluator = new RowExpressionDeterminismEvaluator(metadata.getFunctionManager()); + this.determinismEvaluator = new RowExpressionDeterminismEvaluator(metadata.getFunctionAndTypeManager()); if (expressionCacheSize > 0) { projectionCache = CacheBuilder.newBuilder() diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/RowExpressionCompiler.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/RowExpressionCompiler.java index 2d59f4bb75fe..99d6126b7aeb 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/RowExpressionCompiler.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/RowExpressionCompiler.java @@ -19,7 +19,7 @@ import com.facebook.presto.bytecode.Scope; import com.facebook.presto.bytecode.Variable; import com.facebook.presto.common.function.SqlFunctionProperties; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.function.FunctionMetadata; import com.facebook.presto.spi.function.SqlInvokedScalarFunctionImplementation; @@ -104,8 +104,8 @@ private class Visitor @Override public BytecodeNode visitCall(CallExpression call, Context context) { - FunctionManager functionManager = metadata.getFunctionManager(); - FunctionMetadata functionMetadata = functionManager.getFunctionMetadata(call.getFunctionHandle()); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); + FunctionMetadata functionMetadata = functionAndTypeManager.getFunctionMetadata(call.getFunctionHandle()); BytecodeGeneratorContext generatorContext; switch (functionMetadata.getImplementationType()) { case BUILTIN: @@ -114,10 +114,10 @@ public BytecodeNode visitCall(CallExpression call, Context context) context.getScope(), callSiteBinder, cachedInstanceBinder, - functionManager); + functionAndTypeManager); return (new FunctionCallCodeGenerator()).generateCall(call.getFunctionHandle(), generatorContext, call.getType(), call.getArguments(), context.getOutputBlockVariable()); case SQL: - SqlInvokedScalarFunctionImplementation functionImplementation = (SqlInvokedScalarFunctionImplementation) functionManager.getScalarFunctionImplementation(call.getFunctionHandle()); + SqlInvokedScalarFunctionImplementation functionImplementation = (SqlInvokedScalarFunctionImplementation) functionAndTypeManager.getScalarFunctionImplementation(call.getFunctionHandle()); RowExpression function = getSqlFunctionRowExpression(functionMetadata, functionImplementation, metadata, sqlFunctionProperties, call.getArguments()); // Pre-compile lambda bytecode and update compiled lambda map @@ -140,7 +140,7 @@ public BytecodeNode visitCall(CallExpression call, Context context) context.getScope(), callSiteBinder, cachedInstanceBinder, - functionManager); + functionAndTypeManager); return (new IfCodeGenerator()).generateExpression( generatorContext, @@ -246,7 +246,7 @@ public BytecodeNode visitLambda(LambdaDefinitionExpression lambda, Context conte context.getScope(), callSiteBinder, cachedInstanceBinder, - metadata.getFunctionManager()); + metadata.getFunctionAndTypeManager()); return generateLambda( generatorContext, @@ -298,7 +298,7 @@ public BytecodeNode visitSpecialForm(SpecialFormExpression specialForm, Context break; // functions that require varargs and/or complex types (e.g., lists) case IN: - generator = new InCodeGenerator(metadata.getFunctionManager()); + generator = new InCodeGenerator(metadata.getFunctionAndTypeManager()); break; // optimized implementations (shortcircuiting behavior) case AND: @@ -324,7 +324,7 @@ public BytecodeNode visitSpecialForm(SpecialFormExpression specialForm, Context context.getScope(), callSiteBinder, cachedInstanceBinder, - metadata.getFunctionManager()); + metadata.getFunctionAndTypeManager()); return generator.generateExpression(generatorContext, specialForm.getType(), specialForm.getArguments(), context.getOutputBlockVariable()); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/EffectivePredicateExtractor.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/EffectivePredicateExtractor.java index 23055d80f205..d5159ae4e118 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/EffectivePredicateExtractor.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/EffectivePredicateExtractor.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.expressions.LogicalRowExpressions; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.OperatorNotFoundException; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.plan.AggregationNode; @@ -74,19 +74,19 @@ public class EffectivePredicateExtractor { private final RowExpressionDomainTranslator domainTranslator; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final TypeManager typeManager; - public EffectivePredicateExtractor(RowExpressionDomainTranslator domainTranslator, FunctionManager functionManager, TypeManager typeManager) + public EffectivePredicateExtractor(RowExpressionDomainTranslator domainTranslator, FunctionAndTypeManager functionAndTypeManager, TypeManager typeManager) { this.domainTranslator = requireNonNull(domainTranslator, "domainTranslator is null"); - this.functionManager = functionManager; + this.functionAndTypeManager = functionAndTypeManager; this.typeManager = typeManager; } public RowExpression extract(PlanNode node) { - return node.accept(new Visitor(domainTranslator, functionManager, typeManager), null); + return node.accept(new Visitor(domainTranslator, functionAndTypeManager, typeManager), null); } private static class Visitor @@ -96,15 +96,15 @@ private static class Visitor private final LogicalRowExpressions logicalRowExpressions; private final RowExpressionDeterminismEvaluator determinismEvaluator; private final TypeManager typeManager; - private final FunctionManager functionManger; + private final FunctionAndTypeManager functionManger; - public Visitor(RowExpressionDomainTranslator domainTranslator, FunctionManager functionManager, TypeManager typeManager) + public Visitor(RowExpressionDomainTranslator domainTranslator, FunctionAndTypeManager functionAndTypeManager, TypeManager typeManager) { this.domainTranslator = requireNonNull(domainTranslator, "domainTranslator is null"); this.typeManager = requireNonNull(typeManager); - this.functionManger = requireNonNull(functionManager); - this.determinismEvaluator = new RowExpressionDeterminismEvaluator(functionManager); - this.logicalRowExpressions = new LogicalRowExpressions(determinismEvaluator, new FunctionResolution(functionManager), functionManager); + this.functionManger = requireNonNull(functionAndTypeManager); + this.determinismEvaluator = new RowExpressionDeterminismEvaluator(functionAndTypeManager); + this.logicalRowExpressions = new LogicalRowExpressions(determinismEvaluator, new FunctionResolution(functionAndTypeManager), functionAndTypeManager); } @Override @@ -401,11 +401,11 @@ private RowExpression toEquality(Map.Entry canonicalMap; // Map each known RowExpression to canonical RowExpression private final Set derivedExpressions; private final RowExpressionDeterminismEvaluator determinismEvaluator; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private EqualityInference( Iterable> equalityGroups, Set derivedExpressions, RowExpressionDeterminismEvaluator determinismEvaluator, - FunctionManager functionManager) + FunctionAndTypeManager functionAndTypeManager) { this.determinismEvaluator = determinismEvaluator; - this.functionManager = functionManager; + this.functionAndTypeManager = functionAndTypeManager; ImmutableSetMultimap.Builder setBuilder = ImmutableSetMultimap.builder(); for (Set equalityGroup : equalityGroups) { if (!equalityGroup.isEmpty()) { @@ -217,13 +217,13 @@ public EqualityPartition generateEqualitiesPartitionedBy(Predicate= 2) { for (RowExpression expression : filter(scopeExpressions, not(equalTo(matchingCanonical)))) { - scopeEqualities.add(buildEqualsExpression(functionManager, matchingCanonical, expression)); + scopeEqualities.add(buildEqualsExpression(functionAndTypeManager, matchingCanonical, expression)); } } RowExpression complementCanonical = getCanonical(scopeComplementExpressions); if (scopeComplementExpressions.size() >= 2) { for (RowExpression expression : filter(scopeComplementExpressions, not(equalTo(complementCanonical)))) { - scopeComplementEqualities.add(buildEqualsExpression(functionManager, complementCanonical, expression)); + scopeComplementEqualities.add(buildEqualsExpression(functionAndTypeManager, complementCanonical, expression)); } } @@ -236,7 +236,7 @@ public EqualityPartition generateEqualitiesPartitionedBy(Predicate equalities = new DisjointSet<>(); private final Set derivedExpressions = new LinkedHashSet<>(); - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final NullabilityAnalyzer nullabilityAnalyzer; private final RowExpressionDeterminismEvaluator determinismEvaluator; - public Builder(FunctionManager functionManager, TypeManager typeManager) + public Builder(FunctionAndTypeManager functionAndTypeManager, TypeManager typeManager) { - this.determinismEvaluator = new RowExpressionDeterminismEvaluator(functionManager); - this.functionManager = functionManager; - this.nullabilityAnalyzer = new NullabilityAnalyzer(functionManager, typeManager); + this.determinismEvaluator = new RowExpressionDeterminismEvaluator(functionAndTypeManager); + this.functionAndTypeManager = functionAndTypeManager; + this.nullabilityAnalyzer = new NullabilityAnalyzer(functionAndTypeManager, typeManager); } public Builder(Metadata metadata) { - this(metadata.getFunctionManager(), metadata.getTypeManager()); + this(metadata.getFunctionAndTypeManager(), metadata.getTypeManager()); } /** @@ -356,7 +356,7 @@ private RowExpression normalizeInPredicateToEquality(RowExpression expression) if (size == 1) { RowExpression leftValue = ((SpecialFormExpression) expression).getArguments().get(0); RowExpression rightValue = ((SpecialFormExpression) expression).getArguments().get(1); - return buildEqualsExpression(functionManager, leftValue, rightValue); + return buildEqualsExpression(functionAndTypeManager, leftValue, rightValue); } } return expression; @@ -449,14 +449,14 @@ private void generateMoreEquivalences() public EqualityInference build() { generateMoreEquivalences(); - return new EqualityInference(equalities.getEquivalentClasses(), derivedExpressions, determinismEvaluator, functionManager); + return new EqualityInference(equalities.getEquivalentClasses(), derivedExpressions, determinismEvaluator, functionAndTypeManager); } private boolean isOperation(RowExpression expression, OperatorType type) { if (expression instanceof CallExpression) { CallExpression call = (CallExpression) expression; - Optional expressionOperatorType = functionManager.getFunctionMetadata(call.getFunctionHandle()).getOperatorType(); + Optional expressionOperatorType = functionAndTypeManager.getFunctionMetadata(call.getFunctionHandle()).getOperatorType(); if (expressionOperatorType.isPresent()) { return expressionOperatorType.get() == type; } @@ -485,16 +485,16 @@ private static boolean isInPredicate(RowExpression expression) return false; } - private static CallExpression buildEqualsExpression(FunctionManager functionManager, RowExpression left, RowExpression right) + private static CallExpression buildEqualsExpression(FunctionAndTypeManager functionAndTypeManager, RowExpression left, RowExpression right) { - return binaryOperation(functionManager, EQUAL, left, right); + return binaryOperation(functionAndTypeManager, EQUAL, left, right); } - private static CallExpression binaryOperation(FunctionManager functionManager, OperatorType type, RowExpression left, RowExpression right) + private static CallExpression binaryOperation(FunctionAndTypeManager functionAndTypeManager, OperatorType type, RowExpression left, RowExpression right) { return call( type.getFunctionName().getFunctionName(), - functionManager.resolveOperator(type, fromTypes(left.getType(), right.getType())), + functionAndTypeManager.resolveOperator(type, fromTypes(left.getType(), right.getType())), BOOLEAN, left, right); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionDomainTranslator.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionDomainTranslator.java index 30a66b752302..7feb14fd443a 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionDomainTranslator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionDomainTranslator.java @@ -298,7 +298,7 @@ private Visitor(Metadata metadata, Session session, TypeProvider types) this.literalEncoder = new LiteralEncoder(metadata.getBlockEncodingSerde()); this.session = requireNonNull(session, "session is null"); this.types = requireNonNull(types, "types is null"); - this.functionInvoker = new InterpretedFunctionInvoker(metadata.getFunctionManager()); + this.functionInvoker = new InterpretedFunctionInvoker(metadata.getFunctionAndTypeManager()); } private Type checkedTypeLookup(Expression expression) @@ -661,7 +661,7 @@ private Optional floorValue(Type fromType, Type toType, Object value) private Optional getSaturatedFloorCastOperator(Type fromType, Type toType) { try { - return Optional.of(metadata.getFunctionManager().lookupCast(SATURATED_FLOOR_CAST, fromType.getTypeSignature(), toType.getTypeSignature())); + return Optional.of(metadata.getFunctionAndTypeManager().lookupCast(SATURATED_FLOOR_CAST, fromType.getTypeSignature(), toType.getTypeSignature())); } catch (OperatorNotFoundException e) { return Optional.empty(); @@ -670,7 +670,7 @@ private Optional getSaturatedFloorCastOperator(Type fromType, Ty private int compareOriginalValueToCoerced(Type originalValueType, Object originalValue, Type coercedValueType, Object coercedValue) { - FunctionHandle castToOriginalTypeOperator = metadata.getFunctionManager().lookupCast(CAST, coercedValueType.getTypeSignature(), originalValueType.getTypeSignature()); + FunctionHandle castToOriginalTypeOperator = metadata.getFunctionAndTypeManager().lookupCast(CAST, coercedValueType.getTypeSignature(), originalValueType.getTypeSignature()); Object coercedValueInOriginalType = functionInvoker.invoke(castToOriginalTypeOperator, session.getSqlFunctionProperties(), coercedValue); Block originalValueBlock = Utils.nativeValueToBlock(originalValueType, originalValue); Block coercedValueBlock = Utils.nativeValueToBlock(originalValueType, coercedValueInOriginalType); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionInterpreter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionInterpreter.java index 61f001bea66c..148ad4fd7a9e 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionInterpreter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionInterpreter.java @@ -121,7 +121,7 @@ import static com.facebook.presto.common.type.TypeUtils.writeNativeValue; import static com.facebook.presto.common.type.VarcharType.createVarcharType; import static com.facebook.presto.metadata.CastType.CAST; -import static com.facebook.presto.metadata.FunctionManager.qualifyFunctionName; +import static com.facebook.presto.metadata.FunctionAndTypeManager.qualifyFunctionName; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static com.facebook.presto.sql.analyzer.ConstantExpressionVerifier.verifyExpressionIsConstant; import static com.facebook.presto.sql.analyzer.ExpressionAnalyzer.createConstantAnalyzer; @@ -255,7 +255,7 @@ private ExpressionInterpreter(Expression expression, Metadata metadata, Session this.expressionTypes = ImmutableMap.copyOf(requireNonNull(expressionTypes, "expressionTypes is null")); verify((expressionTypes.containsKey(NodeRef.of(expression)))); this.optimize = optimize; - this.functionInvoker = new InterpretedFunctionInvoker(metadata.getFunctionManager()); + this.functionInvoker = new InterpretedFunctionInvoker(metadata.getFunctionAndTypeManager()); this.legacyRowFieldOrdinalAccess = isLegacyRowFieldOrdinalAccessEnabled(session); this.visitor = new Visitor(); @@ -594,7 +594,7 @@ protected Object visitInPredicate(InPredicate node, Object context) if (valueList.getValues().stream().allMatch(Literal.class::isInstance) && valueList.getValues().stream().noneMatch(NullLiteral.class::isInstance)) { Set objectSet = valueList.getValues().stream().map(expression -> process(expression, context)).collect(Collectors.toSet()); - set = FastutilSetHelper.toFastutilHashSet(objectSet, type(node.getValue()), metadata.getFunctionManager()); + set = FastutilSetHelper.toFastutilHashSet(objectSet, type(node.getValue()), metadata.getFunctionAndTypeManager()); } inListCache.put(valueList, set); } @@ -690,8 +690,8 @@ protected Object visitArithmeticUnary(ArithmeticUnaryExpression node, Object con case PLUS: return value; case MINUS: - FunctionHandle operatorHandle = metadata.getFunctionManager().resolveOperator(OperatorType.NEGATION, fromTypes(types(node.getValue()))); - MethodHandle handle = metadata.getFunctionManager().getBuiltInScalarFunctionImplementation(operatorHandle).getMethodHandle(); + FunctionHandle operatorHandle = metadata.getFunctionAndTypeManager().resolveOperator(OperatorType.NEGATION, fromTypes(types(node.getValue()))); + MethodHandle handle = metadata.getFunctionAndTypeManager().getBuiltInScalarFunctionImplementation(operatorHandle).getMethodHandle(); if (handle.type().parameterCount() > 0 && handle.type().parameterType(0) == SqlFunctionProperties.class) { handle = handle.bindTo(connectorSession.getSqlFunctionProperties()); @@ -808,8 +808,8 @@ protected Object visitNullIfExpression(NullIfExpression node, Object context) Type commonType = metadata.getTypeManager().getCommonSuperType(firstType, secondType).get(); - FunctionHandle firstCast = metadata.getFunctionManager().lookupCast(CAST, firstType.getTypeSignature(), commonType.getTypeSignature()); - FunctionHandle secondCast = metadata.getFunctionManager().lookupCast(CAST, secondType.getTypeSignature(), commonType.getTypeSignature()); + FunctionHandle firstCast = metadata.getFunctionAndTypeManager().lookupCast(CAST, firstType.getTypeSignature(), commonType.getTypeSignature()); + FunctionHandle secondCast = metadata.getFunctionAndTypeManager().lookupCast(CAST, secondType.getTypeSignature(), commonType.getTypeSignature()); // cast(first as ) == cast(second as ) boolean equal = Boolean.TRUE.equals(invokeOperator( @@ -911,11 +911,11 @@ protected Object visitFunctionCall(FunctionCall node, Object context) argumentValues.add(value); argumentTypes.add(type); } - FunctionHandle functionHandle = metadata.getFunctionManager().resolveFunction( + FunctionHandle functionHandle = metadata.getFunctionAndTypeManager().resolveFunction( session.getTransactionId(), qualifyFunctionName(node.getName()), fromTypes(argumentTypes)); - FunctionMetadata functionMetadata = metadata.getFunctionManager().getFunctionMetadata(functionHandle); + FunctionMetadata functionMetadata = metadata.getFunctionAndTypeManager().getFunctionMetadata(functionHandle); if (!functionMetadata.isCalledOnNullInput()) { for (int i = 0; i < argumentValues.size(); i++) { Object value = argumentValues.get(i); @@ -941,7 +941,7 @@ protected Object visitFunctionCall(FunctionCall node, Object context) result = functionInvoker.invoke(functionHandle, session.getSqlFunctionProperties(), argumentValues); break; case SQL: - Expression function = getSqlFunctionExpression(functionMetadata, (SqlInvokedScalarFunctionImplementation) metadata.getFunctionManager().getScalarFunctionImplementation(functionHandle), metadata, session.getSqlFunctionProperties(), node.getArguments()); + Expression function = getSqlFunctionExpression(functionMetadata, (SqlInvokedScalarFunctionImplementation) metadata.getFunctionAndTypeManager().getScalarFunctionImplementation(functionHandle), metadata, session.getSqlFunctionProperties(), node.getArguments()); ExpressionInterpreter functionInterpreter = new ExpressionInterpreter( function, metadata, @@ -1140,7 +1140,7 @@ public Object visitCast(Cast node, Object context) return new Cast(toExpression(value, sourceType), node.getType(), node.isSafe(), node.isTypeOnly()); } - FunctionHandle operator = metadata.getFunctionManager().lookupCast(CAST, sourceType.getTypeSignature(), targetType.getTypeSignature()); + FunctionHandle operator = metadata.getFunctionAndTypeManager().lookupCast(CAST, sourceType.getTypeSignature(), targetType.getTypeSignature()); try { Object castedValue = functionInvoker.invoke(operator, session.getSqlFunctionProperties(), ImmutableList.of(value)); @@ -1291,7 +1291,7 @@ private boolean hasUnresolvedValue(List values) private Object invokeOperator(OperatorType operatorType, List argumentTypes, List argumentValues) { - FunctionHandle operatorHandle = metadata.getFunctionManager().resolveOperator(operatorType, fromTypes(argumentTypes)); + FunctionHandle operatorHandle = metadata.getFunctionAndTypeManager().resolveOperator(operatorType, fromTypes(argumentTypes)); return functionInvoker.invoke(operatorHandle, session.getSqlFunctionProperties(), argumentValues); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/LiteralInterpreter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/LiteralInterpreter.java index 0e27c10f84bc..138d15b14f6f 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/LiteralInterpreter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/LiteralInterpreter.java @@ -185,7 +185,7 @@ private static class LiteralVisitor private LiteralVisitor(Metadata metadata) { this.metadata = metadata; - this.functionInvoker = new InterpretedFunctionInvoker(metadata.getFunctionManager()); + this.functionInvoker = new InterpretedFunctionInvoker(metadata.getFunctionAndTypeManager()); } @Override @@ -251,12 +251,12 @@ protected Object visitGenericLiteral(GenericLiteral node, ConnectorSession sessi } if (JSON.equals(type)) { - FunctionHandle functionHandle = metadata.getFunctionManager().lookupFunction("json_parse", fromTypes(VARCHAR)); + FunctionHandle functionHandle = metadata.getFunctionAndTypeManager().lookupFunction("json_parse", fromTypes(VARCHAR)); return functionInvoker.invoke(functionHandle, session.getSqlFunctionProperties(), ImmutableList.of(utf8Slice(node.getValue()))); } try { - FunctionHandle functionHandle = metadata.getFunctionManager().lookupCast(CAST, VARCHAR.getTypeSignature(), type.getTypeSignature()); + FunctionHandle functionHandle = metadata.getFunctionAndTypeManager().lookupCast(CAST, VARCHAR.getTypeSignature(), type.getTypeSignature()); return functionInvoker.invoke(functionHandle, session.getSqlFunctionProperties(), ImmutableList.of(utf8Slice(node.getValue()))); } catch (IllegalArgumentException e) { 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 19905f15c7ea..93c02e12a8a3 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 @@ -45,7 +45,7 @@ import com.facebook.presto.index.IndexManager; import com.facebook.presto.metadata.AnalyzeTableHandle; import com.facebook.presto.metadata.ConnectorMetadataUpdaterManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.operator.AggregationOperator.AggregationOperatorFactory; import com.facebook.presto.operator.AssignUniqueIdOperator; @@ -404,8 +404,8 @@ public LocalExecutionPlanner( this.tableCommitContextCodec = requireNonNull(tableCommitContextCodec, "tableCommitContextCodec is null"); this.logicalRowExpressions = new LogicalRowExpressions( requireNonNull(determinismEvaluator, "determinismEvaluator is null"), - new FunctionResolution(metadata.getFunctionManager()), - metadata.getFunctionManager()); + new FunctionResolution(metadata.getFunctionAndTypeManager()), + metadata.getFunctionAndTypeManager()); } public LocalExecutionPlan plan( @@ -884,7 +884,7 @@ public PhysicalOperation visitExplainAnalyze(ExplainAnalyzeNode node, LocalExecu context.getNextOperatorId(), node.getId(), analyzeContext.getQueryPerformanceFetcher(), - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), node.isVerbose()); return new PhysicalOperation(operatorFactory, makeLayout(node), context, source); } @@ -1031,9 +1031,9 @@ public PhysicalOperation visitWindow(WindowNode node, LocalExecutionPlanContext arguments.add(source.getLayout().get(argument)); } VariableReferenceExpression variable = entry.getKey(); - FunctionManager functionManager = metadata.getFunctionManager(); - WindowFunctionSupplier windowFunctionSupplier = functionManager.getWindowFunctionImplementation(functionHandle); - Type type = metadata.getType(functionManager.getFunctionMetadata(functionHandle).getReturnType()); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); + WindowFunctionSupplier windowFunctionSupplier = functionAndTypeManager.getWindowFunctionImplementation(functionHandle); + Type type = metadata.getType(functionAndTypeManager.getFunctionMetadata(functionHandle).getReturnType()); windowFunctionsBuilder.add(window(windowFunctionSupplier, type, frameInfo, function.isIgnoreNulls(), arguments.build())); windowFunctionOutputVariablesBuilder.add(variable); } @@ -1402,7 +1402,7 @@ else if (locality.equals(LOCAL)) { OperatorFactory operatorFactory = new RemoteProjectOperatorFactory( context.getNextOperatorId(), planNodeId, - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), projections); return new PhysicalOperation(operatorFactory, outputMappings, context, source); } @@ -1570,7 +1570,7 @@ public PhysicalOperation visitIndexSource(IndexSourceNode node, LocalExecutionPl List remappedProbeKeyChannels = remappedProbeKeyChannelsBuilder.build(); Function probeKeyNormalizer = recordSet -> { if (!overlappingFieldSets.isEmpty()) { - recordSet = new FieldSetFilteringRecordSet(metadata.getFunctionManager(), recordSet, overlappingFieldSets); + recordSet = new FieldSetFilteringRecordSet(metadata.getFunctionAndTypeManager(), recordSet, overlappingFieldSets); } return new MappedRecordSet(recordSet, remappedProbeKeyChannels); }; @@ -1764,7 +1764,7 @@ public PhysicalOperation visitJoin(JoinNode node, LocalExecutionPlanContext cont public PhysicalOperation visitSpatialJoin(SpatialJoinNode node, LocalExecutionPlanContext context) { RowExpression filterExpression = node.getFilter(); - List spatialFunctions = extractSupportedSpatialFunctions(filterExpression, metadata.getFunctionManager()); + List spatialFunctions = extractSupportedSpatialFunctions(filterExpression, metadata.getFunctionAndTypeManager()); for (CallExpression spatialFunction : spatialFunctions) { Optional operation = tryCreateSpatialJoin(context, node, removeExpressionFromFilter(filterExpression, spatialFunction), spatialFunction, Optional.empty(), Optional.empty()); if (operation.isPresent()) { @@ -1772,9 +1772,9 @@ public PhysicalOperation visitSpatialJoin(SpatialJoinNode node, LocalExecutionPl } } - List spatialComparisons = extractSupportedSpatialComparisons(filterExpression, metadata.getFunctionManager()); + List spatialComparisons = extractSupportedSpatialComparisons(filterExpression, metadata.getFunctionAndTypeManager()); for (CallExpression spatialComparison : spatialComparisons) { - FunctionMetadata functionMetadata = metadata.getFunctionManager().getFunctionMetadata(spatialComparison.getFunctionHandle()); + FunctionMetadata functionMetadata = metadata.getFunctionAndTypeManager().getFunctionMetadata(spatialComparison.getFunctionHandle()); checkArgument(functionMetadata.getOperatorType().isPresent() && functionMetadata.getOperatorType().get().isComparisonOperator()); if (functionMetadata.getOperatorType().get() == OperatorType.LESS_THAN || functionMetadata.getOperatorType().get() == OperatorType.LESS_THAN_OR_EQUAL) { // ST_Distance(a, b) <= r @@ -1857,7 +1857,7 @@ private Optional removeExpressionFromFilter(RowExpression filter, private SpatialPredicate spatialTest(CallExpression functionCall, boolean probeFirst, Optional comparisonOperator) { - FunctionMetadata functionMetadata = metadata.getFunctionManager().getFunctionMetadata(functionCall.getFunctionHandle()); + FunctionMetadata functionMetadata = metadata.getFunctionAndTypeManager().getFunctionMetadata(functionCall.getFunctionHandle()); QualifiedFunctionName functionName = functionMetadata.getName(); List argumentTypes = functionMetadata.getArgumentTypes(); Predicate isSpherical = (typeSignature) @@ -2176,7 +2176,7 @@ private JoinBridgeManager createLookupSourceFact probeSource.getLayout(), buildSource.getLayout())); - Optional sortExpressionContext = node.getSortExpressionContext(metadata.getFunctionManager()); + Optional sortExpressionContext = node.getSortExpressionContext(metadata.getFunctionAndTypeManager()); Optional sortChannel = sortExpressionContext .map(SortExpressionContext::getSortExpression) @@ -2863,8 +2863,8 @@ private AccumulatorFactory buildAccumulatorFactory( Aggregation aggregation, boolean spillEnabled) { - FunctionManager functionManager = metadata.getFunctionManager(); - InternalAggregationFunction internalAggregationFunction = functionManager.getAggregateFunctionImplementation(aggregation.getFunctionHandle()); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); + InternalAggregationFunction internalAggregationFunction = functionAndTypeManager.getAggregateFunctionImplementation(aggregation.getFunctionHandle()); List valueChannels = new ArrayList<>(); for (RowExpression argument : aggregation.getArguments()) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/LogicalPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/LogicalPlanner.java index 1efa7a8b7bab..321e13f0aa6c 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/LogicalPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/LogicalPlanner.java @@ -433,7 +433,7 @@ private RelationPlan createTableWriterPlan( TableStatisticAggregation result = statisticsAggregationPlanner.createStatisticsAggregation(statisticsMetadata, columnToVariableMap, true); - StatisticAggregations.Parts aggregations = result.getAggregations().splitIntoPartialAndFinal(variableAllocator, metadata.getFunctionManager()); + StatisticAggregations.Parts aggregations = result.getAggregations().splitIntoPartialAndFinal(variableAllocator, metadata.getFunctionAndTypeManager()); TableFinishNode commitNode = new TableFinishNode( idAllocator.getNextId(), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/NullabilityAnalyzer.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/NullabilityAnalyzer.java index ad9fca7cd2f8..4e358288ab87 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/NullabilityAnalyzer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/NullabilityAnalyzer.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.expressions.DefaultRowExpressionTraversalVisitor; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionMetadata; import com.facebook.presto.spi.relation.CallExpression; import com.facebook.presto.spi.relation.RowExpression; @@ -32,15 +32,15 @@ public final class NullabilityAnalyzer { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final TypeManager typeManager; private final FunctionResolution functionResolution; - public NullabilityAnalyzer(FunctionManager functionManager, TypeManager typeManager) + public NullabilityAnalyzer(FunctionAndTypeManager functionAndTypeManager, TypeManager typeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); this.typeManager = requireNonNull(typeManager, "typeManager is null"); - this.functionResolution = new FunctionResolution(functionManager); + this.functionResolution = new FunctionResolution(functionAndTypeManager); } public boolean mayReturnNullOnNonNullInput(RowExpression expression) @@ -48,20 +48,20 @@ public boolean mayReturnNullOnNonNullInput(RowExpression expression) requireNonNull(expression, "expression is null"); AtomicBoolean result = new AtomicBoolean(false); - expression.accept(new RowExpressionVisitor(functionManager, typeManager, functionResolution), result); + expression.accept(new RowExpressionVisitor(functionAndTypeManager, typeManager, functionResolution), result); return result.get(); } private static class RowExpressionVisitor extends DefaultRowExpressionTraversalVisitor { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final TypeManager typeManager; private final FunctionResolution functionResolution; - public RowExpressionVisitor(FunctionManager functionManager, TypeManager typeManager, FunctionResolution functionResolution) + public RowExpressionVisitor(FunctionAndTypeManager functionAndTypeManager, TypeManager typeManager, FunctionResolution functionResolution) { - this.functionManager = functionManager; + this.functionAndTypeManager = functionAndTypeManager; this.typeManager = typeManager; this.functionResolution = functionResolution; } @@ -69,7 +69,7 @@ public RowExpressionVisitor(FunctionManager functionManager, TypeManager typeMan @Override public Void visitCall(CallExpression call, AtomicBoolean result) { - FunctionMetadata function = functionManager.getFunctionMetadata(call.getFunctionHandle()); + FunctionMetadata function = functionAndTypeManager.getFunctionMetadata(call.getFunctionHandle()); Optional operator = function.getOperatorType(); if (operator.isPresent()) { switch (operator.get()) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanFragmenter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanFragmenter.java index ee48f8078262..48a18e8feda1 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanFragmenter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanFragmenter.java @@ -429,7 +429,7 @@ private SubPlan buildFragment(PlanNode root, FragmentProperties properties, Plan StageExecutionDescriptor.ungroupedExecution(), outputTableWriterFragment, statsAndCosts.getForSubplan(root), - Optional.of(jsonFragmentPlan(root, fragmentVariableTypes, metadata.getFunctionManager(), session))); + Optional.of(jsonFragmentPlan(root, fragmentVariableTypes, metadata.getFunctionAndTypeManager(), session))); return new SubPlan(fragment, properties.getChildren()); } @@ -781,14 +781,14 @@ private TableFinishNode createTemporaryTableWrite( TableMetadata tableMetadata = metadata.getTableMetadata(session, tableHandle); TableStatisticsMetadata statisticsMetadata = metadata.getStatisticsCollectionMetadataForWrite(session, catalogName, tableMetadata.getMetadata()); TableStatisticAggregation statisticsResult = statisticsAggregationPlanner.createStatisticsAggregation(statisticsMetadata, columnNameToVariable, false); - StatisticAggregations.Parts aggregations = statisticsResult.getAggregations().splitIntoPartialAndFinal(variableAllocator, metadata.getFunctionManager()); + StatisticAggregations.Parts aggregations = statisticsResult.getAggregations().splitIntoPartialAndFinal(variableAllocator, metadata.getFunctionAndTypeManager()); PlanNode tableWriterMerge; // Disabled by default. Enable when the column statistics are essential for future runtime adaptive plan optimizations boolean enableStatsCollectionForTemporaryTable = SystemSessionProperties.isEnableStatsCollectionForTemporaryTable(session); if (isTableWriterMergeOperatorEnabled(session)) { - StatisticAggregations.Parts localAggregations = aggregations.getPartialAggregation().splitIntoPartialAndIntermediate(variableAllocator, metadata.getFunctionManager()); + StatisticAggregations.Parts localAggregations = aggregations.getPartialAggregation().splitIntoPartialAndIntermediate(variableAllocator, metadata.getFunctionAndTypeManager()); tableWriterMerge = new TableWriterMergeNode( idAllocator.getNextId(), gatheringExchange( diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java index 865f9fb60776..f7d4e66159a6 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java @@ -239,7 +239,7 @@ public PlanOptimizers( statsCalculator, estimatedExchangesCostCalculator, ImmutableSet.of( - new InlineProjections(metadata.getFunctionManager()), + new InlineProjections(metadata.getFunctionAndTypeManager()), new RemoveRedundantIdentityProjections())); IterativeOptimizer projectionPushDown = new IterativeOptimizer( @@ -315,19 +315,19 @@ public PlanOptimizers( new MultipleDistinctAggregationToMarkDistinct(), new ImplementBernoulliSampleAsFilter(), new MergeLimitWithDistinct(), - new PruneCountAggregationOverScalar(metadata.getFunctionManager()), - new PruneOrderByInAggregation(metadata.getFunctionManager()), + new PruneCountAggregationOverScalar(metadata.getFunctionAndTypeManager()), + new PruneOrderByInAggregation(metadata.getFunctionAndTypeManager()), new RewriteSpatialPartitioningAggregation(metadata))) .build()), simplifyOptimizer, - new UnaliasSymbolReferences(metadata.getFunctionManager()), + new UnaliasSymbolReferences(metadata.getFunctionAndTypeManager()), new IterativeOptimizer( ruleStats, statsCalculator, estimatedExchangesCostCalculator, ImmutableSet.of(new RemoveRedundantIdentityProjections())), new SetFlatteningOptimizer(), - new ImplementIntersectAndExceptAsUnion(metadata.getFunctionManager()), + new ImplementIntersectAndExceptAsUnion(metadata.getFunctionAndTypeManager()), new LimitPushDown(), // Run the LimitPushDown after flattening set operators to make it easier to do the set flattening new PruneUnreferencedOutputs(), inlineProjections, @@ -345,8 +345,8 @@ public PlanOptimizers( ruleStats, statsCalculator, estimatedExchangesCostCalculator, - ImmutableSet.of(new TransformExistsApplyToLateralNode(metadata.getFunctionManager()))), - new TransformQuantifiedComparisonApplyToLateralJoin(metadata.getFunctionManager()), + ImmutableSet.of(new TransformExistsApplyToLateralNode(metadata.getFunctionAndTypeManager()))), + new TransformQuantifiedComparisonApplyToLateralJoin(metadata.getFunctionAndTypeManager()), new IterativeOptimizer( ruleStats, statsCalculator, @@ -355,7 +355,7 @@ public PlanOptimizers( new RemoveUnreferencedScalarLateralNodes(), new TransformUncorrelatedLateralToJoin(), new TransformUncorrelatedInPredicateSubqueryToSemiJoin(), - new TransformCorrelatedScalarAggregationToJoin(metadata.getFunctionManager()), + new TransformCorrelatedScalarAggregationToJoin(metadata.getFunctionAndTypeManager()), new TransformCorrelatedLateralJoinToJoin())), new IterativeOptimizer( ruleStats, @@ -363,7 +363,7 @@ public PlanOptimizers( estimatedExchangesCostCalculator, ImmutableSet.of( new RemoveUnreferencedScalarApplyNodes(), - new TransformCorrelatedInPredicateToJoin(metadata.getFunctionManager()), // must be run after PruneUnreferencedOutputs + new TransformCorrelatedInPredicateToJoin(metadata.getFunctionAndTypeManager()), // must be run after PruneUnreferencedOutputs new TransformCorrelatedScalarSubquery(), // must be run after TransformCorrelatedScalarAggregationToJoin new TransformCorrelatedLateralJoinToJoin(), new ImplementFilteredAggregations())), @@ -372,7 +372,7 @@ public PlanOptimizers( statsCalculator, estimatedExchangesCostCalculator, ImmutableSet.of( - new InlineProjections(metadata.getFunctionManager()), + new InlineProjections(metadata.getFunctionAndTypeManager()), new RemoveRedundantIdentityProjections(), new TransformCorrelatedSingleRowSubqueryToProject())), new CheckSubqueryNodesAreRewritten()); @@ -400,18 +400,18 @@ public PlanOptimizers( estimatedExchangesCostCalculator, ImmutableSet.of( new RemoveRedundantIdentityProjections(), - new PushAggregationThroughOuterJoin(metadata.getFunctionManager()))), + new PushAggregationThroughOuterJoin(metadata.getFunctionAndTypeManager()))), inlineProjections, simplifyRowExpressionOptimizer, // Re-run the SimplifyExpressions to simplify any recomposed expressions from other optimizations projectionPushDown, - new UnaliasSymbolReferences(metadata.getFunctionManager()), // Run again because predicate pushdown and projection pushdown might add more projections + new UnaliasSymbolReferences(metadata.getFunctionAndTypeManager()), // Run again because predicate pushdown and projection pushdown might add more projections new PruneUnreferencedOutputs(), // Make sure to run this before index join. Filtered projections may not have all the columns. new IndexJoinOptimizer(metadata), // Run this after projections and filters have been fully simplified and pushed down new IterativeOptimizer( ruleStats, statsCalculator, estimatedExchangesCostCalculator, - ImmutableSet.of(new SimplifyCountOverConstant(metadata.getFunctionManager()))), + ImmutableSet.of(new SimplifyCountOverConstant(metadata.getFunctionAndTypeManager()))), new LimitPushDown(), // Run LimitPushDown before WindowFilterPushDown new WindowFilterPushDown(metadata), // This must run after PredicatePushDown and LimitPushDown so that it squashes any successive filter nodes and limits new IterativeOptimizer( @@ -462,8 +462,8 @@ public PlanOptimizers( statsCalculator, costCalculator, ImmutableSet.of( - new RewriteFilterWithExternalFunctionToProject(metadata.getFunctionManager()), - new PlanRemotePojections(metadata.getFunctionManager())))); + new RewriteFilterWithExternalFunctionToProject(metadata.getFunctionAndTypeManager()), + new PlanRemotePojections(metadata.getFunctionAndTypeManager())))); // Pass a supplier so that we pickup connector optimizers that are installed later builder.add( @@ -519,7 +519,7 @@ public PlanOptimizers( ImmutableSet.>builder() .add(new RemoveRedundantIdentityProjections()) .addAll(new ExtractSpatialJoins(metadata, splitManager, pageSourceManager).rules()) - .add(new InlineProjections(metadata.getFunctionManager())) + .add(new InlineProjections(metadata.getFunctionAndTypeManager())) .build())); if (!forceSingleNode) { @@ -553,11 +553,11 @@ public PlanOptimizers( ImmutableSet.of(new RemoveEmptyDelete()))); // Run RemoveEmptyDelete after table scan is removed by PickTableLayout/AddExchanges builder.add(predicatePushDown); // Run predicate push down one more time in case we can leverage new information from layouts' effective predicate - builder.add(new RemoveUnsupportedDynamicFilters(metadata.getFunctionManager())); + builder.add(new RemoveUnsupportedDynamicFilters(metadata.getFunctionAndTypeManager())); builder.add(simplifyRowExpressionOptimizer); // Should be always run after PredicatePushDown builder.add(projectionPushDown); builder.add(inlineProjections); - builder.add(new UnaliasSymbolReferences(metadata.getFunctionManager())); // Run unalias after merging projections to simplify projections more efficiently + builder.add(new UnaliasSymbolReferences(metadata.getFunctionAndTypeManager())); // Run unalias after merging projections to simplify projections more efficiently builder.add(new PruneUnreferencedOutputs()); builder.add(new IterativeOptimizer( ruleStats, @@ -566,7 +566,7 @@ public PlanOptimizers( ImmutableSet.>builder() .add(new RemoveRedundantIdentityProjections()) .add(new PushRemoteExchangeThroughAssignUniqueId()) - .add(new InlineProjections(metadata.getFunctionManager())) + .add(new InlineProjections(metadata.getFunctionAndTypeManager())) .build())); // Optimizers above this don't understand local exchanges, so be careful moving this. @@ -580,7 +580,7 @@ public PlanOptimizers( costCalculator, ImmutableSet.of( new PushPartialAggregationThroughJoin(), - new PushPartialAggregationThroughExchange(metadata.getFunctionManager()), + new PushPartialAggregationThroughExchange(metadata.getFunctionAndTypeManager()), new PruneJoinColumns()))); builder.add(new IterativeOptimizer( @@ -602,7 +602,7 @@ public PlanOptimizers( // DO NOT add optimizers that change the plan shape (computations) after this point // Precomputed hashes - this assumes that partitioning will not change - builder.add(new HashGenerationOptimizer(metadata.getFunctionManager())); + builder.add(new HashGenerationOptimizer(metadata.getFunctionAndTypeManager())); builder.add(new MetadataDeleteOptimizer(metadata)); // TODO: consider adding a formal final plan sanitization optimizer that prepares the plan for transmission/execution/logging diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/RowExpressionInterpreter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/RowExpressionInterpreter.java index 4ec55ddcb4d7..cdba3bb8870e 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/RowExpressionInterpreter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/RowExpressionInterpreter.java @@ -25,7 +25,7 @@ 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.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.PrestoException; @@ -127,7 +127,7 @@ public class RowExpressionInterpreter private final Level optimizationLevel; private final InterpretedFunctionInvoker functionInvoker; private final RowExpressionDeterminismEvaluator determinismEvaluator; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final FunctionResolution resolution; private final Visitor visitor; @@ -151,10 +151,10 @@ public RowExpressionInterpreter(RowExpression expression, Metadata metadata, Con this.metadata = requireNonNull(metadata, "metadata is null"); this.session = requireNonNull(session, "session is null"); this.optimizationLevel = optimizationLevel; - this.functionInvoker = new InterpretedFunctionInvoker(metadata.getFunctionManager()); - this.determinismEvaluator = new RowExpressionDeterminismEvaluator(metadata.getFunctionManager()); - this.resolution = new FunctionResolution(metadata.getFunctionManager()); - this.functionManager = metadata.getFunctionManager(); + this.functionInvoker = new InterpretedFunctionInvoker(metadata.getFunctionAndTypeManager()); + this.determinismEvaluator = new RowExpressionDeterminismEvaluator(metadata.getFunctionAndTypeManager()); + this.resolution = new FunctionResolution(metadata.getFunctionAndTypeManager()); + this.functionAndTypeManager = metadata.getFunctionAndTypeManager(); this.visitor = new Visitor(); } @@ -221,7 +221,7 @@ public Object visitCall(CallExpression node, Object context) } FunctionHandle functionHandle = node.getFunctionHandle(); - FunctionMetadata functionMetadata = metadata.getFunctionManager().getFunctionMetadata(node.getFunctionHandle()); + FunctionMetadata functionMetadata = metadata.getFunctionAndTypeManager().getFunctionMetadata(node.getFunctionHandle()); if (!functionMetadata.isCalledOnNullInput()) { for (Object value : argumentValues) { if (value == null) { @@ -273,7 +273,7 @@ public Object visitCall(CallExpression node, Object context) value = functionInvoker.invoke(functionHandle, session.getSqlFunctionProperties(), argumentValues); break; case SQL: - SqlInvokedScalarFunctionImplementation functionImplementation = (SqlInvokedScalarFunctionImplementation) functionManager.getScalarFunctionImplementation(functionHandle); + SqlInvokedScalarFunctionImplementation functionImplementation = (SqlInvokedScalarFunctionImplementation) functionAndTypeManager.getScalarFunctionImplementation(functionHandle); RowExpression function = getSqlFunctionRowExpression(functionMetadata, functionImplementation, metadata, session.getSqlFunctionProperties(), node.getArguments()); RowExpressionInterpreter rowExpressionInterpreter = new RowExpressionInterpreter(function, metadata, session, optimizationLevel); if (optimizationLevel.ordinal() >= EVALUATED.ordinal()) { @@ -368,8 +368,8 @@ else if (Boolean.TRUE.equals(condition)) { Type leftType = node.getArguments().get(0).getType(); Type rightType = node.getArguments().get(1).getType(); Type commonType = metadata.getTypeManager().getCommonSuperType(leftType, rightType).get(); - FunctionHandle firstCast = metadata.getFunctionManager().lookupCast(CAST, leftType.getTypeSignature(), commonType.getTypeSignature()); - FunctionHandle secondCast = metadata.getFunctionManager().lookupCast(CAST, rightType.getTypeSignature(), commonType.getTypeSignature()); + FunctionHandle firstCast = metadata.getFunctionAndTypeManager().lookupCast(CAST, leftType.getTypeSignature(), commonType.getTypeSignature()); + FunctionHandle secondCast = metadata.getFunctionAndTypeManager().lookupCast(CAST, rightType.getTypeSignature(), commonType.getTypeSignature()); // cast(first as ) == cast(second as ) boolean equal = Boolean.TRUE.equals(invokeOperator( @@ -695,16 +695,16 @@ private RowExpression createFailureFunction(RuntimeException exception, Type typ requireNonNull(exception, "Exception is null"); String failureInfo = JsonCodec.jsonCodec(FailureInfo.class).toJson(Failures.toFailure(exception).toFailureInfo()); - FunctionHandle jsonParse = metadata.getFunctionManager().lookupFunction("json_parse", fromTypes(VARCHAR)); + FunctionHandle jsonParse = metadata.getFunctionAndTypeManager().lookupFunction("json_parse", fromTypes(VARCHAR)); Object json = functionInvoker.invoke(jsonParse, session.getSqlFunctionProperties(), utf8Slice(failureInfo)); - FunctionHandle cast = metadata.getFunctionManager().lookupCast(CAST, UNKNOWN.getTypeSignature(), type.getTypeSignature()); + FunctionHandle cast = metadata.getFunctionAndTypeManager().lookupCast(CAST, UNKNOWN.getTypeSignature(), type.getTypeSignature()); if (exception instanceof PrestoException) { long errorCode = ((PrestoException) exception).getErrorCode().getCode(); - FunctionHandle failureFunction = metadata.getFunctionManager().lookupFunction("fail", fromTypes(INTEGER, JSON)); + FunctionHandle failureFunction = metadata.getFunctionAndTypeManager().lookupFunction("fail", fromTypes(INTEGER, JSON)); return call(CAST.name(), cast, type, call("fail", failureFunction, UNKNOWN, constant(errorCode, INTEGER), LiteralEncoder.toRowExpression(json, JSON))); } - FunctionHandle failureFunction = metadata.getFunctionManager().lookupFunction("fail", fromTypes(JSON)); + FunctionHandle failureFunction = metadata.getFunctionAndTypeManager().lookupFunction("fail", fromTypes(JSON)); return call(CAST.name(), cast, type, call("fail", failureFunction, UNKNOWN, LiteralEncoder.toRowExpression(json, JSON))); } @@ -720,7 +720,7 @@ private boolean hasUnresolvedValue(List values) private Object invokeOperator(OperatorType operatorType, List argumentTypes, List argumentValues) { - FunctionHandle operatorHandle = metadata.getFunctionManager().resolveOperator(operatorType, fromTypes(argumentTypes)); + FunctionHandle operatorHandle = metadata.getFunctionAndTypeManager().resolveOperator(operatorType, fromTypes(argumentTypes)); return functionInvoker.invoke(operatorHandle, session.getSqlFunctionProperties(), argumentValues); } @@ -796,14 +796,14 @@ private SpecialCallResult tryHandleCast(CallExpression callExpression, List argumentValues, List argumentTypes, Object context) { - FunctionResolution resolution = new FunctionResolution(metadata.getFunctionManager()); + FunctionResolution resolution = new FunctionResolution(metadata.getFunctionAndTypeManager()); checkArgument(resolution.isLikeFunction(callExpression.getFunctionHandle())); checkArgument(callExpression.getArguments().size() == 2); RowExpression likePatternExpression = callExpression.getArguments().get(1); @@ -920,14 +920,14 @@ private SpecialCallResult tryHandleLike(CallExpression callExpression, List extractSortExpression(Set buildVariables, RowExpression filter, FunctionManager functionManager) + public static Optional extractSortExpression(Set buildVariables, RowExpression filter, FunctionAndTypeManager functionAndTypeManager) { List filterConjuncts = LogicalRowExpressions.extractConjuncts(filter); - SortExpressionVisitor visitor = new SortExpressionVisitor(buildVariables, functionManager); + SortExpressionVisitor visitor = new SortExpressionVisitor(buildVariables, functionAndTypeManager); - DeterminismEvaluator determinismEvaluator = new RowExpressionDeterminismEvaluator(functionManager); + DeterminismEvaluator determinismEvaluator = new RowExpressionDeterminismEvaluator(functionAndTypeManager); List sortExpressionCandidates = filterConjuncts.stream() .filter(determinismEvaluator::isDeterministic) .map(conjunct -> conjunct.accept(visitor, null)) @@ -101,18 +101,18 @@ private static class SortExpressionVisitor implements RowExpressionVisitor, Void> { private final Set buildVariables; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public SortExpressionVisitor(Set buildVariables, FunctionManager functionManager) + public SortExpressionVisitor(Set buildVariables, FunctionAndTypeManager functionAndTypeManager) { this.buildVariables = buildVariables; - this.functionManager = functionManager; + this.functionAndTypeManager = functionAndTypeManager; } @Override public Optional visitCall(CallExpression call, Void context) { - FunctionMetadata functionMetadata = functionManager.getFunctionMetadata(call.getFunctionHandle()); + FunctionMetadata functionMetadata = functionAndTypeManager.getFunctionMetadata(call.getFunctionHandle()); if (!functionMetadata.getOperatorType().map(OperatorType::isComparisonOperator).orElse(false)) { return Optional.empty(); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java index 4a36db5d7253..681c9c67f0a9 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/StatisticsAggregationPlanner.java @@ -14,7 +14,7 @@ package com.facebook.presto.sql.planner; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.operator.aggregation.MaxDataSizeForStats; import com.facebook.presto.operator.aggregation.SumDataSizeForStats; @@ -76,7 +76,7 @@ public TableStatisticAggregation createStatisticsAggregation(TableStatisticsMeta } ImmutableMap.Builder aggregations = ImmutableMap.builder(); - StandardFunctionResolution functionResolution = new FunctionResolution(metadata.getFunctionManager()); + StandardFunctionResolution functionResolution = new FunctionResolution(metadata.getFunctionAndTypeManager()); for (TableStatisticType type : statisticsMetadata.getTableStatistics()) { if (type != ROW_COUNT) { throw new PrestoException(NOT_SUPPORTED, "Table-wide statistic type not supported: " + type); @@ -137,9 +137,9 @@ private ColumnStatisticsAggregation createColumnAggregation(ColumnStatisticType private ColumnStatisticsAggregation createAggregation(String functionName, RowExpression input, Type inputType, Type outputType) { - FunctionManager functionManager = metadata.getFunctionManager(); - FunctionHandle functionHandle = functionManager.lookupFunction(functionName, TypeSignatureProvider.fromTypes(ImmutableList.of(inputType))); - Type resolvedType = metadata.getType(getOnlyElement(functionManager.getFunctionMetadata(functionHandle).getArgumentTypes())); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); + FunctionHandle functionHandle = functionAndTypeManager.lookupFunction(functionName, TypeSignatureProvider.fromTypes(ImmutableList.of(inputType))); + Type resolvedType = metadata.getType(getOnlyElement(functionAndTypeManager.getFunctionMetadata(functionHandle).getArgumentTypes())); verify(resolvedType.equals(inputType), "resolved function input type does not match the input type: %s != %s", resolvedType, inputType); return new ColumnStatisticsAggregation( new AggregationNode.Aggregation( diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/ExtractSpatialJoins.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/ExtractSpatialJoins.java index 3e5e4df7909a..7aac1c286275 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/ExtractSpatialJoins.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/ExtractSpatialJoins.java @@ -26,7 +26,7 @@ import com.facebook.presto.matching.Capture; import com.facebook.presto.matching.Captures; import com.facebook.presto.matching.Pattern; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.QualifiedObjectName; import com.facebook.presto.metadata.Split; @@ -186,14 +186,14 @@ public static final class ExtractSpatialInnerJoin private final Metadata metadata; private final SplitManager splitManager; private final PageSourceManager pageSourceManager; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; public ExtractSpatialInnerJoin(Metadata metadata, SplitManager splitManager, PageSourceManager pageSourceManager) { this.metadata = requireNonNull(metadata, "metadata is null"); this.splitManager = requireNonNull(splitManager, "splitManager is null"); this.pageSourceManager = requireNonNull(pageSourceManager, "pageSourceManager is null"); - this.functionManager = metadata.getFunctionManager(); + this.functionAndTypeManager = metadata.getFunctionAndTypeManager(); } @Override @@ -213,7 +213,7 @@ public Result apply(FilterNode node, Captures captures, Context context) { JoinNode joinNode = captures.get(JOIN); RowExpression filter = node.getPredicate(); - List spatialFunctions = extractSupportedSpatialFunctions(filter, functionManager); + List spatialFunctions = extractSupportedSpatialFunctions(filter, functionAndTypeManager); for (CallExpression spatialFunction : spatialFunctions) { Result result = tryCreateSpatialJoin(context, joinNode, filter, node.getId(), node.getOutputVariables(), spatialFunction, Optional.empty(), metadata, splitManager, pageSourceManager); if (!result.isEmpty()) { @@ -221,7 +221,7 @@ public Result apply(FilterNode node, Captures captures, Context context) } } - List spatialComparisons = extractSupportedSpatialComparisons(filter, functionManager); + List spatialComparisons = extractSupportedSpatialComparisons(filter, functionAndTypeManager); for (CallExpression spatialComparison : spatialComparisons) { Result result = tryCreateSpatialJoin(context, joinNode, filter, node.getId(), node.getOutputVariables(), spatialComparison, metadata, splitManager, pageSourceManager); if (!result.isEmpty()) { @@ -242,14 +242,14 @@ public static final class ExtractSpatialLeftJoin private final Metadata metadata; private final SplitManager splitManager; private final PageSourceManager pageSourceManager; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; public ExtractSpatialLeftJoin(Metadata metadata, SplitManager splitManager, PageSourceManager pageSourceManager) { this.metadata = requireNonNull(metadata, "metadata is null"); this.splitManager = requireNonNull(splitManager, "splitManager is null"); this.pageSourceManager = requireNonNull(pageSourceManager, "pageSourceManager is null"); - this.functionManager = metadata.getFunctionManager(); + this.functionAndTypeManager = metadata.getFunctionAndTypeManager(); } @Override @@ -269,7 +269,7 @@ public Result apply(JoinNode joinNode, Captures captures, Context context) { checkArgument(joinNode.getFilter().isPresent()); RowExpression filter = joinNode.getFilter().get(); - List spatialFunctions = extractSupportedSpatialFunctions(filter, functionManager); + List spatialFunctions = extractSupportedSpatialFunctions(filter, functionAndTypeManager); for (CallExpression spatialFunction : spatialFunctions) { Result result = tryCreateSpatialJoin(context, joinNode, filter, joinNode.getId(), joinNode.getOutputVariables(), spatialFunction, Optional.empty(), metadata, splitManager, pageSourceManager); if (!result.isEmpty()) { @@ -277,7 +277,7 @@ public Result apply(JoinNode joinNode, Captures captures, Context context) } } - List spatialComparisons = extractSupportedSpatialComparisons(filter, functionManager); + List spatialComparisons = extractSupportedSpatialComparisons(filter, functionAndTypeManager); for (CallExpression spatialComparison : spatialComparisons) { Result result = tryCreateSpatialJoin(context, joinNode, filter, joinNode.getId(), joinNode.getOutputVariables(), spatialComparison, metadata, splitManager, pageSourceManager); if (!result.isEmpty()) { @@ -300,7 +300,7 @@ private static Result tryCreateSpatialJoin( SplitManager splitManager, PageSourceManager pageSourceManager) { - FunctionMetadata spatialComparisonMetadata = metadata.getFunctionManager().getFunctionMetadata(spatialComparison.getFunctionHandle()); + FunctionMetadata spatialComparisonMetadata = metadata.getFunctionAndTypeManager().getFunctionMetadata(spatialComparison.getFunctionHandle()); checkArgument(spatialComparison.getArguments().size() == 2 && spatialComparisonMetadata.getOperatorType().isPresent()); PlanNode leftNode = joinNode.getLeft(); PlanNode rightNode = joinNode.getRight(); @@ -334,7 +334,7 @@ private static Result tryCreateSpatialJoin( if (radiusVariables.isEmpty() || (rightVariables.containsAll(radiusVariables) && containsNone(leftVariables, radiusVariables))) { newRadiusVariable = newRadiusVariable(context, radius); OperatorType flippedOperatorType = flip(spatialComparisonMetadata.getOperatorType().get()); - FunctionHandle flippedHandle = getFlippedFunctionHandle(spatialComparison, metadata.getFunctionManager()); + FunctionHandle flippedHandle = getFlippedFunctionHandle(spatialComparison, metadata.getFunctionAndTypeManager()); newComparison = new CallExpression( flippedOperatorType.getOperator(), // TODO verify if this is the correct function displayName flippedHandle, @@ -377,7 +377,7 @@ private static Result tryCreateSpatialJoin( SplitManager splitManager, PageSourceManager pageSourceManager) { - FunctionManager functionManager = metadata.getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); List arguments = spatialFunction.getArguments(); verify(arguments.size() == 2); @@ -439,12 +439,12 @@ else if (alignment < 0) { rightPartitionVariable = Optional.of(context.getVariableAllocator().newVariable("pid", INTEGER)); if (alignment > 0) { - newLeftNode = addPartitioningNodes(context, functionManager, newLeftNode, leftPartitionVariable.get(), kdbTree.get(), newFirstArgument, Optional.empty()); - newRightNode = addPartitioningNodes(context, functionManager, newRightNode, rightPartitionVariable.get(), kdbTree.get(), newSecondArgument, radius); + newLeftNode = addPartitioningNodes(context, functionAndTypeManager, newLeftNode, leftPartitionVariable.get(), kdbTree.get(), newFirstArgument, Optional.empty()); + newRightNode = addPartitioningNodes(context, functionAndTypeManager, newRightNode, rightPartitionVariable.get(), kdbTree.get(), newSecondArgument, radius); } else { - newLeftNode = addPartitioningNodes(context, functionManager, newLeftNode, leftPartitionVariable.get(), kdbTree.get(), newSecondArgument, Optional.empty()); - newRightNode = addPartitioningNodes(context, functionManager, newRightNode, rightPartitionVariable.get(), kdbTree.get(), newFirstArgument, radius); + newLeftNode = addPartitioningNodes(context, functionAndTypeManager, newLeftNode, leftPartitionVariable.get(), kdbTree.get(), newSecondArgument, Optional.empty()); + newRightNode = addPartitioningNodes(context, functionAndTypeManager, newRightNode, rightPartitionVariable.get(), kdbTree.get(), newFirstArgument, radius); } } @@ -648,14 +648,14 @@ private static PlanNode addProjection(Context context, PlanNode node, VariableRe return new ProjectNode(context.getIdAllocator().getNextId(), node, projections.build(), LOCAL); } - private static PlanNode addPartitioningNodes(Context context, FunctionManager functionManager, PlanNode node, VariableReferenceExpression partitionVariable, KdbTree kdbTree, RowExpression geometry, Optional radius) + private static PlanNode addPartitioningNodes(Context context, FunctionAndTypeManager functionAndTypeManager, PlanNode node, VariableReferenceExpression partitionVariable, KdbTree kdbTree, RowExpression geometry, Optional radius) { Assignments.Builder projections = Assignments.builder(); for (VariableReferenceExpression outputVariable : node.getOutputVariables()) { projections.put(outputVariable, outputVariable); } - FunctionHandle castFunctionHandle = functionManager.lookupCast(CAST, VARCHAR.getTypeSignature(), KDB_TREE.getTypeSignature()); + FunctionHandle castFunctionHandle = functionAndTypeManager.lookupCast(CAST, VARCHAR.getTypeSignature(), KDB_TREE.getTypeSignature()); ImmutableList.Builder partitioningArgumentsBuilder = ImmutableList.builder() .add(new CallExpression(CAST.name(), castFunctionHandle, KDB_TREE, ImmutableList.of(Expressions.constant(utf8Slice(KdbTreeUtils.toJson(kdbTree)), VARCHAR)))) @@ -664,7 +664,7 @@ private static PlanNode addPartitioningNodes(Context context, FunctionManager fu List partitioningArguments = partitioningArgumentsBuilder.build(); String spatialPartitionsFunctionName = "spatial_partitions"; - FunctionHandle functionHandle = functionManager.lookupFunction( + FunctionHandle functionHandle = functionAndTypeManager.lookupFunction( spatialPartitionsFunctionName, fromTypes(partitioningArguments.stream() .map(RowExpression::getType).collect(toImmutableList()))); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/InlineProjections.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/InlineProjections.java index 76c135c807b7..a1984062978d 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/InlineProjections.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/InlineProjections.java @@ -17,7 +17,7 @@ import com.facebook.presto.matching.Capture; import com.facebook.presto.matching.Captures; import com.facebook.presto.matching.Pattern; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.plan.Assignments; import com.facebook.presto.spi.plan.Assignments.Builder; import com.facebook.presto.spi.plan.ProjectNode; @@ -73,9 +73,9 @@ public class InlineProjections private final FunctionResolution functionResolution; - public InlineProjections(FunctionManager functionManager) + public InlineProjections(FunctionAndTypeManager functionAndTypeManager) { - this.functionResolution = new FunctionResolution(functionManager); + this.functionResolution = new FunctionResolution(functionAndTypeManager); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/InlineSqlFunctions.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/InlineSqlFunctions.java index e42414f23605..c2e9cb73561a 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/InlineSqlFunctions.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/InlineSqlFunctions.java @@ -34,7 +34,7 @@ import java.util.Set; import static com.facebook.presto.SystemSessionProperties.isInlineSqlFunctions; -import static com.facebook.presto.metadata.FunctionManager.qualifyFunctionName; +import static com.facebook.presto.metadata.FunctionAndTypeManager.qualifyFunctionName; import static com.facebook.presto.sql.analyzer.ExpressionAnalyzer.getExpressionTypes; import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypes; import static com.facebook.presto.sql.relational.SqlFunctionUtils.getSqlFunctionExpression; @@ -115,18 +115,18 @@ public Expression rewriteFunctionCall(FunctionCall node, Void context, Expressio rewrittenArguments.add(treeRewriter.rewrite(argument, context)); } - FunctionHandle functionHandle = metadata.getFunctionManager().resolveFunction( + FunctionHandle functionHandle = metadata.getFunctionAndTypeManager().resolveFunction( session.getTransactionId(), qualifyFunctionName(node.getName()), fromTypes(argumentTypes)); - FunctionMetadata functionMetadata = metadata.getFunctionManager().getFunctionMetadata(functionHandle); + FunctionMetadata functionMetadata = metadata.getFunctionAndTypeManager().getFunctionMetadata(functionHandle); if (functionMetadata.getImplementationType() != FunctionImplementationType.SQL) { return new FunctionCall(node.getName(), rewrittenArguments); } return getSqlFunctionExpression( functionMetadata, - (SqlInvokedScalarFunctionImplementation) metadata.getFunctionManager().getScalarFunctionImplementation(functionHandle), + (SqlInvokedScalarFunctionImplementation) metadata.getFunctionAndTypeManager().getScalarFunctionImplementation(functionHandle), metadata, session.getSqlFunctionProperties(), rewrittenArguments); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PickTableLayout.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PickTableLayout.java index e59e5a5c84c9..8b8f9cc7d4ac 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PickTableLayout.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PickTableLayout.java @@ -264,9 +264,9 @@ private static PlanNode pushPredicateIntoTableScan( { // don't include non-deterministic predicates LogicalRowExpressions logicalRowExpressions = new LogicalRowExpressions( - new RowExpressionDeterminismEvaluator(metadata.getFunctionManager()), - new FunctionResolution(metadata.getFunctionManager()), - metadata.getFunctionManager()); + new RowExpressionDeterminismEvaluator(metadata.getFunctionAndTypeManager()), + new FunctionResolution(metadata.getFunctionAndTypeManager()), + metadata.getFunctionAndTypeManager()); RowExpression deterministicPredicate = logicalRowExpressions.filterDeterministicConjuncts(predicate); DomainTranslator.ExtractionResult decomposedPredicate = domainTranslator.fromPredicate( session.toConnectorSession(), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PlanRemotePojections.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PlanRemotePojections.java index dab5c813a8d2..6b6993613225 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PlanRemotePojections.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PlanRemotePojections.java @@ -15,7 +15,7 @@ import com.facebook.presto.matching.Captures; import com.facebook.presto.matching.Pattern; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionMetadata; import com.facebook.presto.spi.plan.Assignments; import com.facebook.presto.spi.plan.PlanNode; @@ -58,11 +58,11 @@ public class PlanRemotePojections { private static final Pattern PATTERN = project(); - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public PlanRemotePojections(FunctionManager functionManager) + public PlanRemotePojections(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override @@ -79,7 +79,7 @@ public Result apply(ProjectNode node, Captures captures, Rule.Context context) return Result.empty(); } // Fast check for remote functions - if (node.getAssignments().getExpressions().stream().noneMatch(expression -> expression.accept(new ExternalCallExpressionChecker(functionManager), null))) { + if (node.getAssignments().getExpressions().stream().noneMatch(expression -> expression.accept(new ExternalCallExpressionChecker(functionAndTypeManager), null))) { // No remote function return Result.ofPlanNode(new ProjectNode(node.getId(), node.getSource(), node.getAssignments(), LOCAL)); } @@ -97,7 +97,7 @@ public List planRemoteAssignments(Assignments assignments, Pl { ImmutableList.Builder> assignmentProjections = ImmutableList.builder(); for (Map.Entry entry : assignments.getMap().entrySet()) { - List rewritten = entry.getValue().accept(new Visitor(functionManager, variableAllocator), null); + List rewritten = entry.getValue().accept(new Visitor(functionAndTypeManager, variableAllocator), null); if (rewritten.isEmpty()) { assignmentProjections.add(ImmutableList.of(new ProjectionContext(ImmutableMap.of(entry.getKey(), entry.getValue()), false))); } @@ -229,19 +229,19 @@ private static VariableReferenceExpression getAssignedArgument(List, Void> { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final PlanVariableAllocator variableAllocator; - public Visitor(FunctionManager functionManager, PlanVariableAllocator variableAllocator) + public Visitor(FunctionAndTypeManager functionAndTypeManager, PlanVariableAllocator variableAllocator) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); this.variableAllocator = requireNonNull(variableAllocator, "variableAllocator is null"); } @Override public List visitCall(CallExpression call, Void context) { - FunctionMetadata functionMetadata = functionManager.getFunctionMetadata(call.getFunctionHandle()); + FunctionMetadata functionMetadata = functionAndTypeManager.getFunctionMetadata(call.getFunctionHandle()); boolean local = !functionMetadata.getImplementationType().equals(THRIFT); // Break function arguments into local and remote projections first diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneCountAggregationOverScalar.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneCountAggregationOverScalar.java index 08f3fac343d1..c4c8eea0c19e 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneCountAggregationOverScalar.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneCountAggregationOverScalar.java @@ -15,7 +15,7 @@ import com.facebook.presto.matching.Captures; import com.facebook.presto.matching.Pattern; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.StandardFunctionResolution; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.ValuesNode; @@ -42,10 +42,10 @@ public class PruneCountAggregationOverScalar private static final Pattern PATTERN = aggregation(); private final StandardFunctionResolution functionResolution; - public PruneCountAggregationOverScalar(FunctionManager functionManager) + public PruneCountAggregationOverScalar(FunctionAndTypeManager functionAndTypeManager) { - requireNonNull(functionManager, "functionManager is null"); - this.functionResolution = new FunctionResolution(functionManager); + requireNonNull(functionAndTypeManager, "functionManager is null"); + this.functionResolution = new FunctionResolution(functionAndTypeManager); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneOrderByInAggregation.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneOrderByInAggregation.java index e7d113317767..bffed7940290 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneOrderByInAggregation.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PruneOrderByInAggregation.java @@ -15,7 +15,7 @@ import com.facebook.presto.matching.Captures; import com.facebook.presto.matching.Pattern; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.relation.VariableReferenceExpression; import com.facebook.presto.sql.planner.iterative.Rule; @@ -32,11 +32,11 @@ public class PruneOrderByInAggregation implements Rule { private static final Pattern PATTERN = aggregation(); - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public PruneOrderByInAggregation(FunctionManager functionManager) + public PruneOrderByInAggregation(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override @@ -60,7 +60,7 @@ public Result apply(AggregationNode node, Captures captures, Context context) aggregations.put(entry); } // getAggregateFunctionImplementation can be expensive, so check it last. - else if (functionManager.getAggregateFunctionImplementation(aggregation.getFunctionHandle()).isOrderSensitive()) { + else if (functionAndTypeManager.getAggregateFunctionImplementation(aggregation.getFunctionHandle()).isOrderSensitive()) { aggregations.put(entry); } else { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushAggregationThroughOuterJoin.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushAggregationThroughOuterJoin.java index 9f7ad7001e01..09f9a9f4fb44 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushAggregationThroughOuterJoin.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushAggregationThroughOuterJoin.java @@ -19,7 +19,7 @@ import com.facebook.presto.matching.Capture; import com.facebook.presto.matching.Captures; import com.facebook.presto.matching.Pattern; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.Assignments; import com.facebook.presto.spi.plan.Ordering; @@ -104,11 +104,11 @@ public class PushAggregationThroughOuterJoin private static final Pattern PATTERN = aggregation() .with(source().matching(join().capturedAs(JOIN))); - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public PushAggregationThroughOuterJoin(FunctionManager functionManager) + public PushAggregationThroughOuterJoin(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override @@ -329,7 +329,7 @@ private Optional createAggregationOverNull(AggregationNod aggregation.getOrderBy().map(orderBy -> inlineOrderByVariables(sourcesVariableMapping, orderBy)), aggregation.isDistinct(), aggregation.getMask().map(x -> new VariableReferenceExpression(sourcesVariableMapping.get(x).getName(), x.getType()))); - QualifiedFunctionName functionName = functionManager.getFunctionMetadata(overNullAggregation.getFunctionHandle()).getName(); + QualifiedFunctionName functionName = functionAndTypeManager.getFunctionMetadata(overNullAggregation.getFunctionHandle()).getName(); VariableReferenceExpression overNull = variableAllocator.newVariable(functionName.getFunctionName(), aggregationVariable.getType()); aggregationsOverNullBuilder.put(overNull, overNullAggregation); aggregationsVariableMappingBuilder.put(aggregationVariable, overNull); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushPartialAggregationThroughExchange.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushPartialAggregationThroughExchange.java index 99a98fadf760..56d686009134 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushPartialAggregationThroughExchange.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/PushPartialAggregationThroughExchange.java @@ -16,7 +16,7 @@ import com.facebook.presto.matching.Capture; import com.facebook.presto.matching.Captures; import com.facebook.presto.matching.Pattern; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.plan.AggregationNode; @@ -60,11 +60,11 @@ public class PushPartialAggregationThroughExchange implements Rule { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public PushPartialAggregationThroughExchange(FunctionManager functionManager) + public PushPartialAggregationThroughExchange(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } private static final Capture EXCHANGE_NODE = Capture.newCapture(); @@ -86,7 +86,7 @@ public Result apply(AggregationNode aggregationNode, Captures captures, Context { ExchangeNode exchangeNode = captures.get(EXCHANGE_NODE); - boolean decomposable = isDecomposable(aggregationNode, functionManager); + boolean decomposable = isDecomposable(aggregationNode, functionAndTypeManager); if (aggregationNode.getStep().equals(SINGLE) && aggregationNode.hasEmptyGroupingSet() && @@ -203,9 +203,9 @@ private PlanNode split(AggregationNode node, Context context) Map finalAggregation = new HashMap<>(); for (Map.Entry entry : node.getAggregations().entrySet()) { AggregationNode.Aggregation originalAggregation = entry.getValue(); - String functionName = functionManager.getFunctionMetadata(originalAggregation.getFunctionHandle()).getName().getFunctionName(); + String functionName = functionAndTypeManager.getFunctionMetadata(originalAggregation.getFunctionHandle()).getName().getFunctionName(); FunctionHandle functionHandle = originalAggregation.getFunctionHandle(); - InternalAggregationFunction function = functionManager.getAggregateFunctionImplementation(functionHandle); + InternalAggregationFunction function = functionAndTypeManager.getAggregateFunctionImplementation(functionHandle); VariableReferenceExpression intermediateVariable = context.getVariableAllocator().newVariable(functionName, function.getIntermediateType()); checkState(!originalAggregation.getOrderBy().isPresent(), "Aggregate with ORDER BY does not support partial aggregation"); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveUnsupportedDynamicFilters.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveUnsupportedDynamicFilters.java index 839aa2e9a783..eb125d15e1f3 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveUnsupportedDynamicFilters.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveUnsupportedDynamicFilters.java @@ -19,7 +19,7 @@ import com.facebook.presto.expressions.LogicalRowExpressions; import com.facebook.presto.expressions.RowExpressionRewriter; import com.facebook.presto.expressions.RowExpressionTreeRewriter; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.plan.FilterNode; import com.facebook.presto.spi.plan.PlanNode; @@ -73,13 +73,13 @@ public class RemoveUnsupportedDynamicFilters { private final LogicalRowExpressions logicalRowExpressions; - public RemoveUnsupportedDynamicFilters(FunctionManager functionManager) + public RemoveUnsupportedDynamicFilters(FunctionAndTypeManager functionAndTypeManager) { - requireNonNull(functionManager, "functionManager is null"); + requireNonNull(functionAndTypeManager, "functionManager is null"); this.logicalRowExpressions = new LogicalRowExpressions( - new RowExpressionDeterminismEvaluator(functionManager), - new FunctionResolution(functionManager), - functionManager); + new RowExpressionDeterminismEvaluator(functionAndTypeManager), + new FunctionResolution(functionAndTypeManager), + functionAndTypeManager); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/ReorderJoins.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/ReorderJoins.java index 6e27a7873a0f..9d396a62a213 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/ReorderJoins.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/ReorderJoins.java @@ -108,8 +108,8 @@ public ReorderJoins(CostComparator costComparator, Metadata metadata) { this.costComparator = requireNonNull(costComparator, "costComparator is null"); this.metadata = requireNonNull(metadata, "metadata is null"); - this.functionResolution = new FunctionResolution(metadata.getFunctionManager()); - this.determinismEvaluator = new RowExpressionDeterminismEvaluator(metadata.getFunctionManager()); + this.functionResolution = new FunctionResolution(metadata.getFunctionAndTypeManager()); + this.determinismEvaluator = new RowExpressionDeterminismEvaluator(metadata.getFunctionAndTypeManager()); this.joinNodePattern = join().matching( joinNode -> !joinNode.getDistributionType().isPresent() @@ -177,7 +177,7 @@ static class JoinEnumerator this.metadata = requireNonNull(metadata, "metadata is null"); this.allFilterInference = createEqualityInference(metadata, filter); - this.logicalRowExpressions = new LogicalRowExpressions(determinismEvaluator, functionResolution, metadata.getFunctionManager()); + this.logicalRowExpressions = new LogicalRowExpressions(determinismEvaluator, functionResolution, metadata.getFunctionAndTypeManager()); } private JoinEnumerationResult chooseJoinOrder(LinkedHashSet sources, List outputVariables) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RewriteFilterWithExternalFunctionToProject.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RewriteFilterWithExternalFunctionToProject.java index 8d71761d1033..29dc10ffa938 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RewriteFilterWithExternalFunctionToProject.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RewriteFilterWithExternalFunctionToProject.java @@ -15,7 +15,7 @@ import com.facebook.presto.matching.Captures; import com.facebook.presto.matching.Pattern; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.plan.Assignments; import com.facebook.presto.spi.plan.FilterNode; import com.facebook.presto.spi.plan.ProjectNode; @@ -32,11 +32,11 @@ public class RewriteFilterWithExternalFunctionToProject { private static final Pattern PATTERN = filter(); - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public RewriteFilterWithExternalFunctionToProject(FunctionManager functionManager) + public RewriteFilterWithExternalFunctionToProject(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override @@ -48,7 +48,7 @@ public Pattern getPattern() @Override public Result apply(FilterNode node, Captures captures, Context context) { - if (!node.getPredicate().accept(new ExternalCallExpressionChecker(functionManager), null)) { + if (!node.getPredicate().accept(new ExternalCallExpressionChecker(functionAndTypeManager), null)) { // No remote function in predicate return Result.empty(); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RewriteSpatialPartitioningAggregation.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RewriteSpatialPartitioningAggregation.java index 1ef19042b166..a7ee8b9a0839 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RewriteSpatialPartitioningAggregation.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RewriteSpatialPartitioningAggregation.java @@ -81,7 +81,7 @@ public RewriteSpatialPartitioningAggregation(Metadata metadata) private boolean hasSpatialPartitioningAggregation(AggregationNode aggregationNode) { return aggregationNode.getAggregations().values().stream().anyMatch( - aggregation -> metadata.getFunctionManager().getFunctionMetadata(aggregation.getFunctionHandle()).getName().equals(NAME) + aggregation -> metadata.getFunctionAndTypeManager().getFunctionMetadata(aggregation.getFunctionHandle()).getName().equals(NAME) && aggregation.getArguments().size() == 1); } @@ -99,7 +99,7 @@ public Result apply(AggregationNode node, Captures captures, Context context) ImmutableMap.Builder envelopeAssignments = ImmutableMap.builder(); for (Map.Entry entry : node.getAggregations().entrySet()) { Aggregation aggregation = entry.getValue(); - QualifiedFunctionName name = metadata.getFunctionManager().getFunctionMetadata(aggregation.getFunctionHandle()).getName(); + QualifiedFunctionName name = metadata.getFunctionAndTypeManager().getFunctionMetadata(aggregation.getFunctionHandle()).getName(); Type geometryType = metadata.getType(GEOMETRY_TYPE_SIGNATURE); if (name.equals(NAME) && aggregation.getArguments().size() == 1) { RowExpression geometry = getOnlyElement(aggregation.getArguments()); @@ -114,7 +114,7 @@ public Result apply(AggregationNode node, Captures captures, Context context) new Aggregation( new CallExpression( name.getFunctionName(), - metadata.getFunctionManager().lookupFunction(NAME.getFunctionName(), fromTypes(geometryType, INTEGER)), + metadata.getFunctionAndTypeManager().lookupFunction(NAME.getFunctionName(), fromTypes(geometryType, INTEGER)), entry.getKey().getType(), ImmutableList.of( castToRowExpression(asSymbolReference(envelopeVariable)), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/SimplifyCountOverConstant.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/SimplifyCountOverConstant.java index 87f7efb3528a..2f68c5ea4ff7 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/SimplifyCountOverConstant.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/SimplifyCountOverConstant.java @@ -16,7 +16,7 @@ import com.facebook.presto.matching.Capture; import com.facebook.presto.matching.Captures; import com.facebook.presto.matching.Pattern; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.StandardFunctionResolution; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.Assignments; @@ -52,10 +52,10 @@ public class SimplifyCountOverConstant private final StandardFunctionResolution functionResolution; - public SimplifyCountOverConstant(FunctionManager functionManager) + public SimplifyCountOverConstant(FunctionAndTypeManager functionAndTypeManager) { - requireNonNull(functionManager, "functionManager is null"); - this.functionResolution = new FunctionResolution(functionManager); + requireNonNull(functionAndTypeManager, "functionManager is null"); + this.functionResolution = new FunctionResolution(functionAndTypeManager); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/SimplifyRowExpressions.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/SimplifyRowExpressions.java index 6655d5bb5f94..91d241b21a7b 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/SimplifyRowExpressions.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/SimplifyRowExpressions.java @@ -17,7 +17,7 @@ import com.facebook.presto.expressions.LogicalRowExpressions; import com.facebook.presto.expressions.RowExpressionRewriter; import com.facebook.presto.expressions.RowExpressionTreeRewriter; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.relation.CallExpression; @@ -55,7 +55,7 @@ public Rewriter(Metadata metadata) { requireNonNull(metadata, "metadata is null"); this.optimizer = new RowExpressionOptimizer(metadata); - this.logicalExpressionRewriter = new LogicalExpressionRewriter(metadata.getFunctionManager()); + this.logicalExpressionRewriter = new LogicalExpressionRewriter(metadata.getFunctionAndTypeManager()); } @Override @@ -86,11 +86,11 @@ private static class LogicalExpressionRewriter private final FunctionResolution functionResolution; private final LogicalRowExpressions logicalRowExpressions; - public LogicalExpressionRewriter(FunctionManager functionManager) + public LogicalExpressionRewriter(FunctionAndTypeManager functionAndTypeManager) { - requireNonNull(functionManager, "functionManager is null"); - this.functionResolution = new FunctionResolution(functionManager); - this.logicalRowExpressions = new LogicalRowExpressions(new RowExpressionDeterminismEvaluator(functionManager), new FunctionResolution(functionManager), functionManager); + requireNonNull(functionAndTypeManager, "functionManager is null"); + this.functionResolution = new FunctionResolution(functionAndTypeManager); + this.logicalRowExpressions = new LogicalRowExpressions(new RowExpressionDeterminismEvaluator(functionAndTypeManager), new FunctionResolution(functionAndTypeManager), functionAndTypeManager); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TransformCorrelatedInPredicateToJoin.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TransformCorrelatedInPredicateToJoin.java index fd7c7cb37c62..d3a143ea54f3 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TransformCorrelatedInPredicateToJoin.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TransformCorrelatedInPredicateToJoin.java @@ -15,7 +15,7 @@ import com.facebook.presto.matching.Captures; import com.facebook.presto.matching.Pattern; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.StandardFunctionResolution; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.Assignments; @@ -105,10 +105,10 @@ public class TransformCorrelatedInPredicateToJoin private final StandardFunctionResolution functionResolution; - public TransformCorrelatedInPredicateToJoin(FunctionManager functionManager) + public TransformCorrelatedInPredicateToJoin(FunctionAndTypeManager functionAndTypeManager) { - requireNonNull(functionManager, "functionManager is null"); - this.functionResolution = new FunctionResolution(functionManager); + requireNonNull(functionAndTypeManager, "functionManager is null"); + this.functionResolution = new FunctionResolution(functionAndTypeManager); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TransformCorrelatedScalarAggregationToJoin.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TransformCorrelatedScalarAggregationToJoin.java index 025d7645355d..a99886d234d4 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TransformCorrelatedScalarAggregationToJoin.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TransformCorrelatedScalarAggregationToJoin.java @@ -15,7 +15,7 @@ import com.facebook.presto.matching.Captures; import com.facebook.presto.matching.Pattern; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.ProjectNode; @@ -75,11 +75,11 @@ public Pattern getPattern() return PATTERN; } - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public TransformCorrelatedScalarAggregationToJoin(FunctionManager functionManager) + public TransformCorrelatedScalarAggregationToJoin(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override @@ -96,7 +96,7 @@ public Result apply(LateralJoinNode lateralJoinNode, Captures captures, Context return Result.empty(); } - ScalarAggregationToJoinRewriter rewriter = new ScalarAggregationToJoinRewriter(functionManager, context.getVariableAllocator(), context.getIdAllocator(), context.getLookup()); + ScalarAggregationToJoinRewriter rewriter = new ScalarAggregationToJoinRewriter(functionAndTypeManager, context.getVariableAllocator(), context.getIdAllocator(), context.getLookup()); PlanNode rewrittenNode = rewriter.rewriteScalarAggregation(lateralJoinNode, aggregation.get()); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TransformExistsApplyToLateralNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TransformExistsApplyToLateralNode.java index e25415016295..2aaaed14a61c 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TransformExistsApplyToLateralNode.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TransformExistsApplyToLateralNode.java @@ -15,7 +15,7 @@ import com.facebook.presto.matching.Captures; import com.facebook.presto.matching.Pattern; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.StandardFunctionResolution; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.AggregationNode.Aggregation; @@ -87,10 +87,10 @@ public class TransformExistsApplyToLateralNode private final StandardFunctionResolution functionResolution; - public TransformExistsApplyToLateralNode(FunctionManager functionManager) + public TransformExistsApplyToLateralNode(FunctionAndTypeManager functionAndTypeManager) { - requireNonNull(functionManager, "functionManager is null"); - this.functionResolution = new FunctionResolution(functionManager); + requireNonNull(functionAndTypeManager, "functionManager is null"); + this.functionResolution = new FunctionResolution(functionAndTypeManager); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TranslateExpressions.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TranslateExpressions.java index db4b8248945b..9331eaf62790 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TranslateExpressions.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/TranslateExpressions.java @@ -88,11 +88,11 @@ private Map, Type> analyzeCallExpressionTypes(CallExpression .collect(toImmutableList()); ImmutableMap.Builder, Type> builder = ImmutableMap., Type>builder(); if (!lambdaExpressions.isEmpty()) { - List functionTypes = metadata.getFunctionManager().getFunctionMetadata(callExpression.getFunctionHandle()).getArgumentTypes().stream() + List functionTypes = metadata.getFunctionAndTypeManager().getFunctionMetadata(callExpression.getFunctionHandle()).getArgumentTypes().stream() .filter(typeSignature -> typeSignature.getBase().equals(FunctionType.NAME)) .map(typeSignature -> (FunctionType) (metadata.getTypeManager().getType(typeSignature))) .collect(toImmutableList()); - InternalAggregationFunction internalAggregationFunction = metadata.getFunctionManager().getAggregateFunctionImplementation(callExpression.getFunctionHandle()); + InternalAggregationFunction internalAggregationFunction = metadata.getFunctionAndTypeManager().getAggregateFunctionImplementation(callExpression.getFunctionHandle()); List lambdaInterfaces = internalAggregationFunction.getLambdaInterfaces(); verify(lambdaExpressions.size() == functionTypes.size()); verify(lambdaExpressions.size() == lambdaInterfaces.size()); @@ -160,7 +160,7 @@ private Map, Type> analyze(Expression expression, Session se private RowExpression toRowExpression(Expression expression, Session session, Map, Type> types) { - return SqlToRowExpressionTranslator.translate(expression, types, ImmutableMap.of(), metadata.getFunctionManager(), metadata.getTypeManager(), session); + return SqlToRowExpressionTranslator.translate(expression, types, ImmutableMap.of(), metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), session); } private RowExpression removeOriginalExpression(RowExpression expression, Rule.Context context) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddExchanges.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddExchanges.java index 4a909635c50a..a82cdbd77bbe 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddExchanges.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddExchanges.java @@ -243,7 +243,7 @@ public PlanWithProperties visitAggregation(AggregationNode node, PreferredProper { Set partitioningRequirement = ImmutableSet.copyOf(node.getGroupingKeys()); - boolean preferSingleNode = hasSingleNodeExecutionPreference(node, metadata.getFunctionManager()); + boolean preferSingleNode = hasSingleNodeExecutionPreference(node, metadata.getFunctionAndTypeManager()); boolean hasMixedGroupingSets = node.hasEmptyGroupingSet() && node.hasNonEmptyGroupingSet(); PreferredProperties preferredProperties = preferSingleNode ? PreferredProperties.undistributed() : PreferredProperties.any(); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddLocalExchanges.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddLocalExchanges.java index a62b28ccd46e..170767dac512 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddLocalExchanges.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddLocalExchanges.java @@ -290,13 +290,13 @@ public PlanWithProperties visitAggregation(AggregationNode node, StreamPreferred { checkState(node.getStep() == AggregationNode.Step.SINGLE, "step of aggregation is expected to be SINGLE, but it is %s", node.getStep()); - if (hasSingleNodeExecutionPreference(node, metadata.getFunctionManager())) { + if (hasSingleNodeExecutionPreference(node, metadata.getFunctionAndTypeManager())) { return planAndEnforceChildren(node, singleStream(), defaultParallelism(session)); } List groupingKeys = node.getGroupingKeys(); if (node.hasDefaultOutput()) { - checkState(isDecomposable(node, metadata.getFunctionManager())); + checkState(isDecomposable(node, metadata.getFunctionAndTypeManager())); // Put fixed local exchange directly below final aggregation to ensure that final and partial aggregations are separated by exchange (in a local runner mode) // This is required so that default outputs from multiple instances of partial aggregations are passed to a single final aggregation. @@ -503,7 +503,7 @@ public PlanWithProperties visitTableWriter(TableWriterNode originalTableWriterNo .getStatisticsAggregation() .map(aggregations -> aggregations.splitIntoPartialAndIntermediate( variableAllocator, - metadata.getFunctionManager())); + metadata.getFunctionAndTypeManager())); PlanWithProperties tableWriter; diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AggregationNodeUtils.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AggregationNodeUtils.java index fa08e68a5c9a..73083072fdf9 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AggregationNodeUtils.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AggregationNodeUtils.java @@ -13,7 +13,7 @@ */ package com.facebook.presto.sql.planner.optimizations; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.relation.CallExpression; import com.facebook.presto.spi.relation.RowExpression; @@ -37,11 +37,11 @@ public class AggregationNodeUtils { private AggregationNodeUtils() {} - public static AggregationNode.Aggregation count(FunctionManager functionManager) + public static AggregationNode.Aggregation count(FunctionAndTypeManager functionAndTypeManager) { return new AggregationNode.Aggregation( new CallExpression("count", - new FunctionResolution(functionManager).countFunction(), + new FunctionResolution(functionAndTypeManager).countFunction(), BIGINT, ImmutableList.of()), Optional.empty(), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ExpressionEquivalence.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ExpressionEquivalence.java index 15af125ea034..4e26332960a9 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ExpressionEquivalence.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ExpressionEquivalence.java @@ -16,7 +16,7 @@ import com.facebook.presto.Session; import com.facebook.presto.common.function.QualifiedFunctionName; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.function.FunctionHandle; @@ -76,7 +76,7 @@ public ExpressionEquivalence(Metadata metadata, SqlParser sqlParser) { this.metadata = requireNonNull(metadata, "metadata is null"); this.sqlParser = requireNonNull(sqlParser, "sqlParser is null"); - this.canonicalizationVisitor = new CanonicalizationVisitor(metadata.getFunctionManager()); + this.canonicalizationVisitor = new CanonicalizationVisitor(metadata.getFunctionAndTypeManager()); } public boolean areExpressionsEquivalent(Session session, Expression leftExpression, Expression rightExpression, TypeProvider types) @@ -119,17 +119,17 @@ private RowExpression toRowExpression(Session session, Expression expression, Ma WarningCollector.NOOP); // convert to row expression - return translate(expression, expressionTypes, variableInput, metadata.getFunctionManager(), metadata.getTypeManager(), session); + return translate(expression, expressionTypes, variableInput, metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), session); } private static class CanonicalizationVisitor implements RowExpressionVisitor { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public CanonicalizationVisitor(FunctionManager functionManager) + public CanonicalizationVisitor(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override @@ -143,7 +143,7 @@ public RowExpression visitCall(CallExpression call, Void context) .map(expression -> expression.accept(this, context)) .collect(toImmutableList())); - QualifiedFunctionName callName = functionManager.getFunctionMetadata(call.getFunctionHandle()).getName(); + QualifiedFunctionName callName = functionAndTypeManager.getFunctionMetadata(call.getFunctionHandle()).getName(); if (callName.equals(EQUAL.getFunctionName()) || callName.equals(NOT_EQUAL.getFunctionName()) || callName.equals(IS_DISTINCT_FROM.getFunctionName())) { // sort arguments @@ -157,7 +157,7 @@ public RowExpression visitCall(CallExpression call, Void context) if (callName.equals(GREATER_THAN.getFunctionName()) || callName.equals(GREATER_THAN_OR_EQUAL.getFunctionName())) { // convert greater than to less than - FunctionHandle functionHandle = functionManager.resolveOperator( + FunctionHandle functionHandle = functionAndTypeManager.resolveOperator( callName.equals(GREATER_THAN.getFunctionName()) ? LESS_THAN : LESS_THAN_OR_EQUAL, swapPair(fromTypes(call.getArguments().stream().map(RowExpression::getType).collect(toImmutableList())))); return new CallExpression( diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ExternalCallExpressionChecker.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ExternalCallExpressionChecker.java index af63e4dc26d8..abb19090391f 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ExternalCallExpressionChecker.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ExternalCallExpressionChecker.java @@ -13,7 +13,7 @@ */ package com.facebook.presto.sql.planner.optimizations; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionMetadata; import com.facebook.presto.spi.relation.CallExpression; import com.facebook.presto.spi.relation.ConstantExpression; @@ -28,17 +28,17 @@ public class ExternalCallExpressionChecker implements RowExpressionVisitor { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public ExternalCallExpressionChecker(FunctionManager functionManager) + public ExternalCallExpressionChecker(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override public Boolean visitCall(CallExpression call, Void context) { - FunctionMetadata functionMetadata = functionManager.getFunctionMetadata(call.getFunctionHandle()); + FunctionMetadata functionMetadata = functionAndTypeManager.getFunctionMetadata(call.getFunctionHandle()); if (functionMetadata.getImplementationType().isExternal()) { return true; } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/HashGenerationOptimizer.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/HashGenerationOptimizer.java index a7b2a1bc41d1..2dbae0a0b454 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/HashGenerationOptimizer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/HashGenerationOptimizer.java @@ -16,7 +16,7 @@ import com.facebook.presto.Session; import com.facebook.presto.SystemSessionProperties; import com.facebook.presto.common.function.OperatorType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.Assignments; @@ -95,11 +95,11 @@ public class HashGenerationOptimizer public static final long INITIAL_HASH_VALUE = 0; private static final String HASH_CODE = OperatorType.HASH_CODE.getFunctionName().getFunctionName(); - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public HashGenerationOptimizer(FunctionManager functionManager) + public HashGenerationOptimizer(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override @@ -111,7 +111,7 @@ public PlanNode optimize(PlanNode plan, Session session, TypeProvider types, Pla requireNonNull(variableAllocator, "variableAllocator is null"); requireNonNull(idAllocator, "idAllocator is null"); if (SystemSessionProperties.isOptimizeHashGenerationEnabled(session)) { - PlanWithProperties result = plan.accept(new Rewriter(idAllocator, variableAllocator, functionManager), new HashComputationSet()); + PlanWithProperties result = plan.accept(new Rewriter(idAllocator, variableAllocator, functionAndTypeManager), new HashComputationSet()); return result.getNode(); } return plan; @@ -122,13 +122,13 @@ private static class Rewriter { private final PlanNodeIdAllocator idAllocator; private final PlanVariableAllocator variableAllocator; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - private Rewriter(PlanNodeIdAllocator idAllocator, PlanVariableAllocator variableAllocator, FunctionManager functionManager) + private Rewriter(PlanNodeIdAllocator idAllocator, PlanVariableAllocator variableAllocator, FunctionAndTypeManager functionAndTypeManager) { this.idAllocator = requireNonNull(idAllocator, "idAllocator is null"); this.variableAllocator = requireNonNull(variableAllocator, "variableAllocator is null"); - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override @@ -166,7 +166,7 @@ public PlanWithProperties visitAggregation(AggregationNode node, HashComputation Optional groupByHash = Optional.empty(); List groupingKeys = node.getGroupingKeys(); if (!node.isStreamable() && !canSkipHashGeneration(node.getGroupingKeys())) { - groupByHash = computeHash(groupingKeys, functionManager); + groupByHash = computeHash(groupingKeys, functionAndTypeManager); } // aggregation does not pass through preferred hash variables @@ -209,7 +209,7 @@ public PlanWithProperties visitDistinctLimit(DistinctLimitNode node, HashComputa return planSimpleNodeWithProperties(node, parentPreference); } - Optional hashComputation = computeHash(node.getDistinctVariables(), functionManager); + Optional hashComputation = computeHash(node.getDistinctVariables(), functionAndTypeManager); PlanWithProperties child = planAndEnforce( node.getSource(), new HashComputationSet(hashComputation), @@ -233,7 +233,7 @@ public PlanWithProperties visitMarkDistinct(MarkDistinctNode node, HashComputati return planSimpleNodeWithProperties(node, parentPreference); } - Optional hashComputation = computeHash(node.getDistinctVariables(), functionManager); + Optional hashComputation = computeHash(node.getDistinctVariables(), functionAndTypeManager); PlanWithProperties child = planAndEnforce( node.getSource(), new HashComputationSet(hashComputation), @@ -253,7 +253,7 @@ public PlanWithProperties visitRowNumber(RowNumberNode node, HashComputationSet return planSimpleNodeWithProperties(node, parentPreference); } - Optional hashComputation = computeHash(node.getPartitionBy(), functionManager); + Optional hashComputation = computeHash(node.getPartitionBy(), functionAndTypeManager); PlanWithProperties child = planAndEnforce( node.getSource(), new HashComputationSet(hashComputation), @@ -279,7 +279,7 @@ public PlanWithProperties visitTopNRowNumber(TopNRowNumberNode node, HashComputa return planSimpleNodeWithProperties(node, parentPreference); } - Optional hashComputation = computeHash(node.getPartitionBy(), functionManager); + Optional hashComputation = computeHash(node.getPartitionBy(), functionAndTypeManager); PlanWithProperties child = planAndEnforce( node.getSource(), new HashComputationSet(hashComputation), @@ -316,11 +316,11 @@ public PlanWithProperties visitJoin(JoinNode node, HashComputationSet parentPref // join does not pass through preferred hash variables since they take more memory and since // the join node filters, may take more compute - Optional leftHashComputation = computeHash(Lists.transform(clauses, JoinNode.EquiJoinClause::getLeft), functionManager); + Optional leftHashComputation = computeHash(Lists.transform(clauses, JoinNode.EquiJoinClause::getLeft), functionAndTypeManager); PlanWithProperties left = planAndEnforce(node.getLeft(), new HashComputationSet(leftHashComputation), true, new HashComputationSet(leftHashComputation)); VariableReferenceExpression leftHashVariable = left.getRequiredHashVariable(leftHashComputation.get()); - Optional rightHashComputation = computeHash(Lists.transform(clauses, JoinNode.EquiJoinClause::getRight), functionManager); + Optional rightHashComputation = computeHash(Lists.transform(clauses, JoinNode.EquiJoinClause::getRight), functionAndTypeManager); // drop undesired hash variables from build to save memory PlanWithProperties right = planAndEnforce(node.getRight(), new HashComputationSet(rightHashComputation), true, new HashComputationSet(rightHashComputation)); VariableReferenceExpression rightHashVariable = right.getRequiredHashVariable(rightHashComputation.get()); @@ -378,7 +378,7 @@ private PlanWithProperties buildJoinNodeWithPreferredHashes( @Override public PlanWithProperties visitSemiJoin(SemiJoinNode node, HashComputationSet parentPreference) { - Optional sourceHashComputation = computeHash(ImmutableList.of(node.getSourceJoinVariable()), functionManager); + Optional sourceHashComputation = computeHash(ImmutableList.of(node.getSourceJoinVariable()), functionAndTypeManager); PlanWithProperties source = planAndEnforce( node.getSource(), new HashComputationSet(sourceHashComputation), @@ -386,7 +386,7 @@ public PlanWithProperties visitSemiJoin(SemiJoinNode node, HashComputationSet pa new HashComputationSet(sourceHashComputation)); VariableReferenceExpression sourceHashVariable = source.getRequiredHashVariable(sourceHashComputation.get()); - Optional filterHashComputation = computeHash(ImmutableList.of(node.getFilteringSourceJoinVariable()), functionManager); + Optional filterHashComputation = computeHash(ImmutableList.of(node.getFilteringSourceJoinVariable()), functionAndTypeManager); HashComputationSet requiredHashes = new HashComputationSet(filterHashComputation); PlanWithProperties filteringSource = planAndEnforce(node.getFilteringSource(), requiredHashes, true, requiredHashes); VariableReferenceExpression filteringSourceHashVariable = filteringSource.getRequiredHashVariable(filterHashComputation.get()); @@ -425,7 +425,7 @@ public PlanWithProperties visitIndexJoin(IndexJoinNode node, HashComputationSet // join does not pass through preferred hash variables since they take more memory and since // the join node filters, may take more compute - Optional probeHashComputation = computeHash(Lists.transform(clauses, IndexJoinNode.EquiJoinClause::getProbe), functionManager); + Optional probeHashComputation = computeHash(Lists.transform(clauses, IndexJoinNode.EquiJoinClause::getProbe), functionAndTypeManager); PlanWithProperties probe = planAndEnforce( node.getProbeSource(), new HashComputationSet(probeHashComputation), @@ -433,7 +433,7 @@ public PlanWithProperties visitIndexJoin(IndexJoinNode node, HashComputationSet new HashComputationSet(probeHashComputation)); VariableReferenceExpression probeHashVariable = probe.getRequiredHashVariable(probeHashComputation.get()); - Optional indexHashComputation = computeHash(Lists.transform(clauses, IndexJoinNode.EquiJoinClause::getIndex), functionManager); + Optional indexHashComputation = computeHash(Lists.transform(clauses, IndexJoinNode.EquiJoinClause::getIndex), functionAndTypeManager); HashComputationSet requiredHashes = new HashComputationSet(indexHashComputation); PlanWithProperties index = planAndEnforce(node.getIndexSource(), requiredHashes, true, requiredHashes); VariableReferenceExpression indexHashVariable = index.getRequiredHashVariable(indexHashComputation.get()); @@ -464,7 +464,7 @@ public PlanWithProperties visitWindow(WindowNode node, HashComputationSet parent return planSimpleNodeWithProperties(node, parentPreference, true); } - Optional hashComputation = computeHash(node.getPartitionBy(), functionManager); + Optional hashComputation = computeHash(node.getPartitionBy(), functionAndTypeManager); PlanWithProperties child = planAndEnforce( node.getSource(), new HashComputationSet(hashComputation), @@ -501,7 +501,7 @@ public PlanWithProperties visitExchange(ExchangeNode node, HashComputationSet pa partitioningScheme.getPartitioning().getArguments().stream() .map(VariableReferenceExpression.class::cast) .collect(toImmutableList()), - functionManager); + functionAndTypeManager); preference = preference.withHashComputation(partitionVariables); } @@ -845,17 +845,17 @@ public HashComputationSet withHashComputation(Optional hashComp } } - private static Optional computeHash(Iterable fields, FunctionManager functionManager) + private static Optional computeHash(Iterable fields, FunctionAndTypeManager functionAndTypeManager) { requireNonNull(fields, "fields is null"); List variables = ImmutableList.copyOf(fields); if (variables.isEmpty()) { return Optional.empty(); } - return Optional.of(new HashComputation(fields, functionManager)); + return Optional.of(new HashComputation(fields, functionAndTypeManager)); } - public static Optional getHashExpression(FunctionManager functionManager, List variables) + public static Optional getHashExpression(FunctionAndTypeManager functionAndTypeManager, List variables) { if (variables.isEmpty()) { return Optional.empty(); @@ -863,9 +863,9 @@ public static Optional getHashExpression(FunctionManager function RowExpression result = constant(INITIAL_HASH_VALUE, BIGINT); for (VariableReferenceExpression variable : variables) { - RowExpression hashField = call(functionManager, HASH_CODE, BIGINT, variable); + RowExpression hashField = call(functionAndTypeManager, HASH_CODE, BIGINT, variable); hashField = orNullHashCode(hashField); - result = call(functionManager, "combine_hash", BIGINT, result, hashField); + result = call(functionAndTypeManager, "combine_hash", BIGINT, result, hashField); } return Optional.of(result); } @@ -879,13 +879,13 @@ private static RowExpression orNullHashCode(RowExpression expression) private static class HashComputation { private final List fields; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - private HashComputation(Iterable fields, FunctionManager functionManager) + private HashComputation(Iterable fields, FunctionAndTypeManager functionAndTypeManager) { requireNonNull(fields, "fields is null"); this.fields = ImmutableList.copyOf(fields); - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); checkArgument(!this.fields.isEmpty(), "fields can not be empty"); } @@ -899,7 +899,7 @@ public Optional translate(Function availableFields) @@ -918,8 +918,8 @@ private RowExpression getHashExpression() private RowExpression getHashFunctionCall(RowExpression previousHashValue, VariableReferenceExpression variable) { - CallExpression functionCall = call(functionManager, HASH_CODE, BIGINT, variable); - return call(functionManager, "combine_hash", BIGINT, previousHashValue, orNullHashCode(functionCall)); + CallExpression functionCall = call(functionAndTypeManager, HASH_CODE, BIGINT, variable); + return call(functionAndTypeManager, "combine_hash", BIGINT, previousHashValue, orNullHashCode(functionCall)); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ImplementIntersectAndExceptAsUnion.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ImplementIntersectAndExceptAsUnion.java index 5cefa773297e..b0df6665a3c4 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ImplementIntersectAndExceptAsUnion.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ImplementIntersectAndExceptAsUnion.java @@ -16,7 +16,7 @@ import com.facebook.presto.Session; import com.facebook.presto.common.type.StandardTypes; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.function.StandardFunctionResolution; import com.facebook.presto.spi.plan.AggregationNode; @@ -110,11 +110,11 @@ public class ImplementIntersectAndExceptAsUnion implements PlanOptimizer { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public ImplementIntersectAndExceptAsUnion(FunctionManager functionManager) + public ImplementIntersectAndExceptAsUnion(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override @@ -126,7 +126,7 @@ public PlanNode optimize(PlanNode plan, Session session, TypeProvider types, Pla requireNonNull(variableAllocator, "variableAllocator is null"); requireNonNull(idAllocator, "idAllocator is null"); - return SimplePlanRewriter.rewriteWith(new Rewriter(session, functionManager, idAllocator, variableAllocator), plan); + return SimplePlanRewriter.rewriteWith(new Rewriter(session, functionAndTypeManager, idAllocator, variableAllocator), plan); } private static class Rewriter @@ -139,11 +139,11 @@ private static class Rewriter private final PlanNodeIdAllocator idAllocator; private final PlanVariableAllocator variableAllocator; - private Rewriter(Session session, FunctionManager functionManager, PlanNodeIdAllocator idAllocator, PlanVariableAllocator variableAllocator) + private Rewriter(Session session, FunctionAndTypeManager functionAndTypeManager, PlanNodeIdAllocator idAllocator, PlanVariableAllocator variableAllocator) { - requireNonNull(functionManager, "functionManager is null"); + requireNonNull(functionAndTypeManager, "functionManager is null"); this.session = requireNonNull(session, "session is null"); - this.functionResolution = new FunctionResolution(functionManager); + this.functionResolution = new FunctionResolution(functionAndTypeManager); this.idAllocator = requireNonNull(idAllocator, "idAllocator is null"); this.variableAllocator = requireNonNull(variableAllocator, "variableAllocator is null"); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/IndexJoinOptimizer.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/IndexJoinOptimizer.java index b8d1f2a48e3d..708934bbd00d 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/IndexJoinOptimizer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/IndexJoinOptimizer.java @@ -247,9 +247,9 @@ private IndexSourceRewriter(PlanNodeIdAllocator idAllocator, Metadata metadata, this.metadata = requireNonNull(metadata, "metadata is null"); this.domainTranslator = new RowExpressionDomainTranslator(metadata); this.logicalRowExpressions = new LogicalRowExpressions( - new RowExpressionDeterminismEvaluator(metadata.getFunctionManager()), - new FunctionResolution(metadata.getFunctionManager()), - metadata.getFunctionManager()); + new RowExpressionDeterminismEvaluator(metadata.getFunctionAndTypeManager()), + new FunctionResolution(metadata.getFunctionAndTypeManager()), + metadata.getFunctionAndTypeManager()); this.idAllocator = requireNonNull(idAllocator, "idAllocator is null"); this.session = requireNonNull(session, "session is null"); } @@ -364,7 +364,7 @@ public PlanNode visitFilter(FilterNode node, RewriteContext context) public PlanNode visitWindow(WindowNode node, RewriteContext context) { if (!node.getWindowFunctions().values().stream() - .allMatch(function -> metadata.getFunctionManager().getFunctionMetadata(function.getFunctionHandle()).getFunctionKind() == AGGREGATE)) { + .allMatch(function -> metadata.getFunctionAndTypeManager().getFunctionMetadata(function.getFunctionHandle()).getFunctionKind() == AGGREGATE)) { return node; } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MetadataQueryOptimizer.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MetadataQueryOptimizer.java index 02daba42b00f..edd419afb82e 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MetadataQueryOptimizer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MetadataQueryOptimizer.java @@ -124,7 +124,7 @@ public PlanNode visitAggregation(AggregationNode node, RewriteContext cont { // supported functions are only MIN/MAX/APPROX_DISTINCT or distinct aggregates for (Aggregation aggregation : node.getAggregations().values()) { - QualifiedFunctionName functionName = metadata.getFunctionManager().getFunctionMetadata(aggregation.getFunctionHandle()).getName(); + QualifiedFunctionName functionName = metadata.getFunctionAndTypeManager().getFunctionMetadata(aggregation.getFunctionHandle()).getName(); if (!ALLOWED_FUNCTIONS.contains(functionName) && !aggregation.isDistinct()) { return context.defaultRewrite(node); } @@ -218,7 +218,7 @@ private boolean isReducible(AggregationNode node, List 1 || !inputs.containsAll(aggregation.getCall().getArguments())) { @@ -267,7 +267,7 @@ else if (value.getValue() != null) { assignmentsBuilder.put( outputVariable, evaluateMinMax( - metadata.getFunctionManager().getFunctionMetadata(node.getAggregations().get(outputVariable).getFunctionHandle()), + metadata.getFunctionAndTypeManager().getFunctionMetadata(node.getAggregations().get(outputVariable).getFunctionHandle()), inputColumnValues.get(getOnlyElement(aggregation.getArguments())))); } Assignments assignments = assignmentsBuilder.build(); @@ -290,7 +290,7 @@ private RowExpression evaluateMinMax(FunctionMetadata aggregationFunctionMetadat for (List partitionedArguments : Lists.partition(arguments, 100)) { Object reducedValue = evaluateConstantRowExpression( call( - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), scalarFunctionName, returnType, partitionedArguments), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/OptimizeMixedDistinctAggregations.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/OptimizeMixedDistinctAggregations.java index f0d2f7fd14b9..8f9d9cc1c146 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/OptimizeMixedDistinctAggregations.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/OptimizeMixedDistinctAggregations.java @@ -87,7 +87,7 @@ public class OptimizeMixedDistinctAggregations public OptimizeMixedDistinctAggregations(Metadata metadata) { this.metadata = metadata; - this.functionResolution = new FunctionResolution(metadata.getFunctionManager()); + this.functionResolution = new FunctionResolution(metadata.getFunctionAndTypeManager()); } @Override @@ -184,14 +184,14 @@ public PlanNode visitAggregation(AggregationNode node, RewriteContext subExpression instanceof CallExpression && @@ -314,7 +314,7 @@ private boolean isInliningCandidate(RowExpression expression, ProjectNode node) .collect(Collectors.groupingBy(identity(), Collectors.counting())); return dependencies.entrySet().stream() - .allMatch(entry -> (entry.getValue() == 1 && !node.getAssignments().get(entry.getKey()).accept(new ExternalCallExpressionChecker(functionManager), null)) || + .allMatch(entry -> (entry.getValue() == 1 && !node.getAssignments().get(entry.getKey()).accept(new ExternalCallExpressionChecker(functionAndTypeManager), null)) || node.getAssignments().get(entry.getKey()) instanceof ConstantExpression); } @@ -466,7 +466,7 @@ public PlanNode visitJoin(JoinNode node, RewriteContext context) newJoinPredicate = simplifyExpression(newJoinPredicate); // TODO: find a better way to directly optimize FALSE LITERAL in join predicate if (newJoinPredicate.equals(FALSE_CONSTANT)) { - newJoinPredicate = buildEqualsExpression(functionManager, constant(0L, BIGINT), constant(1L, BIGINT)); + newJoinPredicate = buildEqualsExpression(functionAndTypeManager, constant(0L, BIGINT), constant(1L, BIGINT)); } // Create identity projections for all existing symbols @@ -516,7 +516,7 @@ public PlanNode visitJoin(JoinNode node, RewriteContext context) boolean dynamicFilterEnabled = isEnableDynamicFiltering(session); Map dynamicFilters = ImmutableMap.of(); if (dynamicFilterEnabled) { - DynamicFiltersResult dynamicFiltersResult = createDynamicFilters(node, equiJoinClauses, idAllocator, metadata.getFunctionManager()); + DynamicFiltersResult dynamicFiltersResult = createDynamicFilters(node, equiJoinClauses, idAllocator, metadata.getFunctionAndTypeManager()); dynamicFilters = dynamicFiltersResult.getDynamicFilters(); leftPredicate = logicalRowExpressions.combineConjuncts(leftPredicate, logicalRowExpressions.combineConjuncts(dynamicFiltersResult.getPredicates())); } @@ -603,7 +603,7 @@ private static DynamicFiltersResult createDynamicFilters( JoinNode node, List equiJoinClauses, PlanNodeIdAllocator idAllocator, - FunctionManager functionManager) + FunctionAndTypeManager functionAndTypeManager) { Map dynamicFilters = ImmutableMap.of(); List predicates = ImmutableList.of(); @@ -618,7 +618,7 @@ private static DynamicFiltersResult createDynamicFilters( VariableReferenceExpression probeSymbol = clause.getLeft(); VariableReferenceExpression buildSymbol = clause.getRight(); String id = idAllocator.getNextId().toString(); - predicatesBuilder.add(createDynamicFilterExpression(id, probeSymbol, functionManager)); + predicatesBuilder.add(createDynamicFilterExpression(id, probeSymbol, functionAndTypeManager)); dynamicFiltersBuilder.put(id, buildSymbol); } dynamicFilters = dynamicFiltersBuilder.build(); @@ -627,10 +627,10 @@ private static DynamicFiltersResult createDynamicFilters( return new DynamicFiltersResult(dynamicFilters, predicates); } - private static RowExpression createDynamicFilterExpression(String id, VariableReferenceExpression input, FunctionManager functionManager) + private static RowExpression createDynamicFilterExpression(String id, VariableReferenceExpression input, FunctionAndTypeManager functionAndTypeManager) { return call( - functionManager, + functionAndTypeManager, DynamicFilters.DynamicFilterPlaceholderFunction.NAME, BooleanType.BOOLEAN, ImmutableList.of(new ConstantExpression(Slices.utf8Slice(id), VarcharType.VARCHAR), input)); @@ -640,12 +640,12 @@ private static DynamicFiltersResult createDynamicFilters( VariableReferenceExpression probeVariable, VariableReferenceExpression buildVariable, PlanNodeIdAllocator idAllocator, - FunctionManager functionManager) + FunctionAndTypeManager functionAndTypeManager) { ImmutableMap.Builder dynamicFiltersBuilder = ImmutableMap.builder(); ImmutableList.Builder predicatesBuilder = ImmutableList.builder(); String id = idAllocator.getNextId().toString(); - predicatesBuilder.add(createDynamicFilterExpression(id, probeVariable, functionManager)); + predicatesBuilder.add(createDynamicFilterExpression(id, probeVariable, functionAndTypeManager)); dynamicFiltersBuilder.put(id, buildVariable); return new DynamicFiltersResult(dynamicFiltersBuilder.build(), predicatesBuilder.build()); } @@ -930,18 +930,18 @@ private InnerJoinPushDownResult processInnerJoin(RowExpression inheritedPredicat rightEffectivePredicate = logicalRowExpressions.filterDeterministicConjuncts(rightEffectivePredicate); // Generate equality inferences - EqualityInference allInference = new EqualityInference.Builder(functionManager, typeManager) + EqualityInference allInference = new EqualityInference.Builder(functionAndTypeManager, typeManager) .addEqualityInference(inheritedPredicate, leftEffectivePredicate, rightEffectivePredicate, joinPredicate) .build(); - EqualityInference allInferenceWithoutLeftInferred = new EqualityInference.Builder(functionManager, typeManager) + EqualityInference allInferenceWithoutLeftInferred = new EqualityInference.Builder(functionAndTypeManager, typeManager) .addEqualityInference(inheritedPredicate, rightEffectivePredicate, joinPredicate) .build(); - EqualityInference allInferenceWithoutRightInferred = new EqualityInference.Builder(functionManager, typeManager) + EqualityInference allInferenceWithoutRightInferred = new EqualityInference.Builder(functionAndTypeManager, typeManager) .addEqualityInference(inheritedPredicate, leftEffectivePredicate, joinPredicate) .build(); // Sort through conjuncts in inheritedPredicate that were not used for inference - for (RowExpression conjunct : new EqualityInference.Builder(functionManager, typeManager).nonInferrableConjuncts(inheritedPredicate)) { + for (RowExpression conjunct : new EqualityInference.Builder(functionAndTypeManager, typeManager).nonInferrableConjuncts(inheritedPredicate)) { RowExpression leftRewrittenConjunct = allInference.rewriteExpression(conjunct, in(leftVariables)); if (leftRewrittenConjunct != null) { leftPushDownConjuncts.add(leftRewrittenConjunct); @@ -959,7 +959,7 @@ private InnerJoinPushDownResult processInnerJoin(RowExpression inheritedPredicat } // See if we can push the right effective predicate to the left side - for (RowExpression conjunct : new EqualityInference.Builder(functionManager, typeManager).nonInferrableConjuncts(rightEffectivePredicate)) { + for (RowExpression conjunct : new EqualityInference.Builder(functionAndTypeManager, typeManager).nonInferrableConjuncts(rightEffectivePredicate)) { RowExpression rewritten = allInference.rewriteExpression(conjunct, in(leftVariables)); if (rewritten != null) { leftPushDownConjuncts.add(rewritten); @@ -967,7 +967,7 @@ private InnerJoinPushDownResult processInnerJoin(RowExpression inheritedPredicat } // See if we can push the left effective predicate to the right side - for (RowExpression conjunct : new EqualityInference.Builder(functionManager, typeManager).nonInferrableConjuncts(leftEffectivePredicate)) { + for (RowExpression conjunct : new EqualityInference.Builder(functionAndTypeManager, typeManager).nonInferrableConjuncts(leftEffectivePredicate)) { RowExpression rewritten = allInference.rewriteExpression(conjunct, not(in(leftVariables))); if (rewritten != null) { rightPushDownConjuncts.add(rewritten); @@ -975,7 +975,7 @@ private InnerJoinPushDownResult processInnerJoin(RowExpression inheritedPredicat } // See if we can push any parts of the join predicates to either side - for (RowExpression conjunct : new EqualityInference.Builder(functionManager, typeManager).nonInferrableConjuncts(joinPredicate)) { + for (RowExpression conjunct : new EqualityInference.Builder(functionAndTypeManager, typeManager).nonInferrableConjuncts(joinPredicate)) { RowExpression leftRewritten = allInference.rewriteExpression(conjunct, in(leftVariables)); if (leftRewritten != null) { leftPushDownConjuncts.add(leftRewritten); @@ -1050,7 +1050,7 @@ private RowExpression extractJoinPredicate(JoinNode joinNode) private RowExpression toRowExpression(JoinNode.EquiJoinClause equiJoinClause) { - return buildEqualsExpression(functionManager, equiJoinClause.getLeft(), equiJoinClause.getRight()); + return buildEqualsExpression(functionAndTypeManager, equiJoinClause.getLeft(), equiJoinClause.getRight()); } private JoinNode tryNormalizeToOuterToInnerJoin(JoinNode node, RowExpression inheritedPredicate) @@ -1174,7 +1174,7 @@ private Predicate joinEqualityExpression(final Collection operatorType = functionManager.getFunctionMetadata(((CallExpression) expression).getFunctionHandle()).getOperatorType(); + Optional operatorType = functionAndTypeManager.getFunctionMetadata(((CallExpression) expression).getFunctionHandle()).getOperatorType(); if (operatorType.isPresent()) { return operatorType.get().equals(type); } @@ -1203,10 +1203,10 @@ private PlanNode visitNonFilteringSemiJoin(SemiJoinNode node, RewriteContext sourceVariables = node.getSource().getOutputVariables(); List filteringSourceVariables = node.getFilteringSource().getOutputVariables(); @@ -1304,7 +1304,7 @@ private PlanNode visitFilteringSemiJoin(SemiJoinNode node, RewriteContext dynamicFilters = ImmutableMap.of(); if (isEnableDynamicFiltering(session)) { - DynamicFiltersResult dynamicFiltersResult = createDynamicFilters(node.getSourceJoinVariable(), node.getFilteringSourceJoinVariable(), idAllocator, metadata.getFunctionManager()); + DynamicFiltersResult dynamicFiltersResult = createDynamicFilters(node.getSourceJoinVariable(), node.getFilteringSourceJoinVariable(), idAllocator, metadata.getFunctionAndTypeManager()); dynamicFilters = dynamicFiltersResult.getDynamicFilters(); // add filter node on top of probe rewrittenSource = new FilterNode(idAllocator.getNextId(), rewrittenSource, logicalRowExpressions.combineConjuncts(dynamicFiltersResult.getPredicates())); @@ -1332,13 +1332,13 @@ private PlanNode visitFilteringSemiJoin(SemiJoinNode node, RewriteContext nonInferrableConjuncts(RowExpression inheritedPredicate) { - return new EqualityInference.Builder(functionManager, typeManager) + return new EqualityInference.Builder(functionAndTypeManager, typeManager) .nonInferrableConjuncts(inheritedPredicate); } private EqualityInference createEqualityInference(RowExpression... expressions) { - return new EqualityInference.Builder(functionManager, typeManager) + return new EqualityInference.Builder(functionAndTypeManager, typeManager) .addEqualityInference(expressions) .build(); } @@ -1479,11 +1479,11 @@ public PlanNode visitAssignUniqueId(AssignUniqueId node, RewriteContext c AggregationNode.Aggregation aggregation = entry.getValue(); // Allow sub-field pruning to pass through the arbitrary() aggregation - QualifiedFunctionName aggregateName = metadata.getFunctionManager().getFunctionMetadata(aggregation.getCall().getFunctionHandle()).getName(); + QualifiedFunctionName aggregateName = metadata.getFunctionAndTypeManager().getFunctionMetadata(aggregation.getCall().getFunctionHandle()).getName(); if (ARBITRARY_AGGREGATE_FUNCTION.equals(aggregateName)) { checkState(aggregation.getArguments().get(0) instanceof VariableReferenceExpression); context.get().addAssignment(variable, (VariableReferenceExpression) aggregation.getArguments().get(0)); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ScalarAggregationToJoinRewriter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ScalarAggregationToJoinRewriter.java index 166888cecdcb..526948663e11 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ScalarAggregationToJoinRewriter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ScalarAggregationToJoinRewriter.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.type.BooleanType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.AggregationNode.Aggregation; import com.facebook.presto.spi.plan.Assignments; @@ -63,10 +63,10 @@ public class ScalarAggregationToJoinRewriter private final Lookup lookup; private final PlanNodeDecorrelator planNodeDecorrelator; - public ScalarAggregationToJoinRewriter(FunctionManager functionManager, PlanVariableAllocator variableAllocator, PlanNodeIdAllocator idAllocator, Lookup lookup) + public ScalarAggregationToJoinRewriter(FunctionAndTypeManager functionAndTypeManager, PlanVariableAllocator variableAllocator, PlanNodeIdAllocator idAllocator, Lookup lookup) { - requireNonNull(functionManager, "metadata is null"); - this.functionResolution = new FunctionResolution(functionManager); + requireNonNull(functionAndTypeManager, "metadata is null"); + this.functionResolution = new FunctionResolution(functionAndTypeManager); this.variableAllocator = requireNonNull(variableAllocator, "variableAllocator is null"); this.idAllocator = requireNonNull(idAllocator, "idAllocator is null"); this.lookup = requireNonNull(lookup, "lookup is null"); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformQuantifiedComparisonApplyToLateralJoin.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformQuantifiedComparisonApplyToLateralJoin.java index 3c52d6eb217f..b378095fb101 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformQuantifiedComparisonApplyToLateralJoin.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformQuantifiedComparisonApplyToLateralJoin.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.type.BigintType; import com.facebook.presto.common.type.BooleanType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.function.StandardFunctionResolution; import com.facebook.presto.spi.plan.AggregationNode; @@ -81,10 +81,10 @@ public class TransformQuantifiedComparisonApplyToLateralJoin { private final StandardFunctionResolution functionResolution; - public TransformQuantifiedComparisonApplyToLateralJoin(FunctionManager functionManager) + public TransformQuantifiedComparisonApplyToLateralJoin(FunctionAndTypeManager functionAndTypeManager) { - requireNonNull(functionManager, "functionManager is null"); - this.functionResolution = new FunctionResolution(functionManager); + requireNonNull(functionAndTypeManager, "functionManager is null"); + this.functionResolution = new FunctionResolution(functionAndTypeManager); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/UnaliasSymbolReferences.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/UnaliasSymbolReferences.java index 6334003fce3d..7671611d2b0b 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/UnaliasSymbolReferences.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/UnaliasSymbolReferences.java @@ -15,7 +15,7 @@ import com.facebook.presto.Session; import com.facebook.presto.common.block.SortOrder; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.Assignments; @@ -117,11 +117,11 @@ public class UnaliasSymbolReferences implements PlanOptimizer { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public UnaliasSymbolReferences(FunctionManager functionManager) + public UnaliasSymbolReferences(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override @@ -133,7 +133,7 @@ public PlanNode optimize(PlanNode plan, Session session, TypeProvider types, Pla requireNonNull(variableAllocator, "variableAllocator is null"); requireNonNull(idAllocator, "idAllocator is null"); - return SimplePlanRewriter.rewriteWith(new Rewriter(types, functionManager), plan); + return SimplePlanRewriter.rewriteWith(new Rewriter(types, functionAndTypeManager), plan); } private static class Rewriter @@ -143,10 +143,10 @@ private static class Rewriter private final TypeProvider types; private final RowExpressionDeterminismEvaluator determinismEvaluator; - private Rewriter(TypeProvider types, FunctionManager functionManager) + private Rewriter(TypeProvider types, FunctionAndTypeManager functionAndTypeManager) { this.types = types; - this.determinismEvaluator = new RowExpressionDeterminismEvaluator(functionManager); + this.determinismEvaluator = new RowExpressionDeterminismEvaluator(functionAndTypeManager); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/WindowFilterPushDown.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/WindowFilterPushDown.java index 087252697894..f842c347cdf7 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/WindowFilterPushDown.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/WindowFilterPushDown.java @@ -19,7 +19,7 @@ import com.facebook.presto.common.predicate.TupleDomain; import com.facebook.presto.common.predicate.ValueSet; import com.facebook.presto.expressions.LogicalRowExpressions; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.function.FunctionHandle; @@ -70,9 +70,9 @@ public WindowFilterPushDown(Metadata metadata) this.metadata = requireNonNull(metadata, "metadata is null"); this.domainTranslator = new RowExpressionDomainTranslator(metadata); this.logicalRowExpressions = new LogicalRowExpressions( - new RowExpressionDeterminismEvaluator(metadata.getFunctionManager()), - new FunctionResolution(metadata.getFunctionManager()), - metadata.getFunctionManager()); + new RowExpressionDeterminismEvaluator(metadata.getFunctionAndTypeManager()), + new FunctionResolution(metadata.getFunctionAndTypeManager()), + metadata.getFunctionAndTypeManager()); } @Override @@ -111,7 +111,7 @@ public PlanNode visitWindow(WindowNode node, RewriteContext context) checkState(node.getWindowFunctions().size() == 1, "WindowFilterPushdown requires that WindowNodes contain exactly one window function"); PlanNode rewrittenSource = context.rewrite(node.getSource()); - if (canReplaceWithRowNumber(node, metadata.getFunctionManager())) { + if (canReplaceWithRowNumber(node, metadata.getFunctionAndTypeManager())) { return new RowNumberNode(idAllocator.getNextId(), rewrittenSource, node.getPartitionBy(), @@ -139,7 +139,7 @@ public PlanNode visitLimit(LimitNode node, RewriteContext context) } source = rowNumberNode; } - else if (source instanceof WindowNode && canOptimizeWindowFunction((WindowNode) source, metadata.getFunctionManager()) && isOptimizeTopNRowNumber(session)) { + else if (source instanceof WindowNode && canOptimizeWindowFunction((WindowNode) source, metadata.getFunctionAndTypeManager()) && isOptimizeTopNRowNumber(session)) { WindowNode windowNode = (WindowNode) source; // verify that unordered row_number window functions are replaced by RowNumberNode verify(windowNode.getOrderingScheme().isPresent()); @@ -168,7 +168,7 @@ public PlanNode visitFilter(FilterNode node, RewriteContext context) return rewriteFilterSource(node, source, rowNumberVariable, upperBound.getAsInt()); } } - else if (source instanceof WindowNode && canOptimizeWindowFunction((WindowNode) source, metadata.getFunctionManager()) && isOptimizeTopNRowNumber(session)) { + else if (source instanceof WindowNode && canOptimizeWindowFunction((WindowNode) source, metadata.getFunctionAndTypeManager()) && isOptimizeTopNRowNumber(session)) { WindowNode windowNode = (WindowNode) source; VariableReferenceExpression rowNumberVariable = getOnlyElement(windowNode.getCreatedVariable()); OptionalInt upperBound = extractUpperBound(tupleDomain, rowNumberVariable); @@ -268,24 +268,24 @@ private TopNRowNumberNode convertToTopNRowNumber(WindowNode windowNode, int limi Optional.empty()); } - private static boolean canReplaceWithRowNumber(WindowNode node, FunctionManager functionManager) + private static boolean canReplaceWithRowNumber(WindowNode node, FunctionAndTypeManager functionAndTypeManager) { - return canOptimizeWindowFunction(node, functionManager) && !node.getOrderingScheme().isPresent(); + return canOptimizeWindowFunction(node, functionAndTypeManager) && !node.getOrderingScheme().isPresent(); } - private static boolean canOptimizeWindowFunction(WindowNode node, FunctionManager functionManager) + private static boolean canOptimizeWindowFunction(WindowNode node, FunctionAndTypeManager functionAndTypeManager) { if (node.getWindowFunctions().size() != 1) { return false; } VariableReferenceExpression rowNumberVariable = getOnlyElement(node.getWindowFunctions().keySet()); - return isRowNumberMetadata(functionManager, functionManager.getFunctionMetadata(node.getWindowFunctions().get(rowNumberVariable).getFunctionHandle())); + return isRowNumberMetadata(functionAndTypeManager, functionAndTypeManager.getFunctionMetadata(node.getWindowFunctions().get(rowNumberVariable).getFunctionHandle())); } - private static boolean isRowNumberMetadata(FunctionManager functionManager, FunctionMetadata functionMetadata) + private static boolean isRowNumberMetadata(FunctionAndTypeManager functionAndTypeManager, FunctionMetadata functionMetadata) { - FunctionHandle rowNumberFunction = functionManager.lookupFunction("row_number", ImmutableList.of()); - return functionMetadata.equals(functionManager.getFunctionMetadata(rowNumberFunction)); + FunctionHandle rowNumberFunction = functionAndTypeManager.lookupFunction("row_number", ImmutableList.of()); + return functionMetadata.equals(functionAndTypeManager.getFunctionMetadata(rowNumberFunction)); } } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java index a3d572d5623b..92152ec4491a 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java @@ -13,7 +13,7 @@ */ package com.facebook.presto.sql.planner.plan; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeId; import com.facebook.presto.spi.relation.RowExpression; @@ -266,10 +266,10 @@ public Optional getFilter() return filter; } - public Optional getSortExpressionContext(FunctionManager functionManager) + public Optional getSortExpressionContext(FunctionAndTypeManager functionAndTypeManager) { return filter - .flatMap(filter -> extractSortExpression(ImmutableSet.copyOf(right.getOutputVariables()), filter, functionManager)); + .flatMap(filter -> extractSortExpression(ImmutableSet.copyOf(right.getOutputVariables()), filter, functionAndTypeManager)); } @JsonProperty diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/StatisticAggregations.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/StatisticAggregations.java index 619b91421afe..ce8d62e09575 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/StatisticAggregations.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/StatisticAggregations.java @@ -13,7 +13,7 @@ */ package com.facebook.presto.sql.planner.plan; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.plan.AggregationNode.Aggregation; @@ -58,27 +58,27 @@ public List getGroupingVariables() return groupingVariables; } - public Parts splitIntoPartialAndFinal(PlanVariableAllocator variableAllocator, FunctionManager functionManager) + public Parts splitIntoPartialAndFinal(PlanVariableAllocator variableAllocator, FunctionAndTypeManager functionAndTypeManager) { - return split(variableAllocator, functionManager, false); + return split(variableAllocator, functionAndTypeManager, false); } - public Parts splitIntoPartialAndIntermediate(PlanVariableAllocator variableAllocator, FunctionManager functionManager) + public Parts splitIntoPartialAndIntermediate(PlanVariableAllocator variableAllocator, FunctionAndTypeManager functionAndTypeManager) { - return split(variableAllocator, functionManager, true); + return split(variableAllocator, functionAndTypeManager, true); } - private Parts split(PlanVariableAllocator variableAllocator, FunctionManager functionManager, boolean intermediate) + private Parts split(PlanVariableAllocator variableAllocator, FunctionAndTypeManager functionAndTypeManager, boolean intermediate) { ImmutableMap.Builder finalOrIntermediateAggregations = ImmutableMap.builder(); ImmutableMap.Builder partialAggregations = ImmutableMap.builder(); for (Map.Entry entry : aggregations.entrySet()) { Aggregation originalAggregation = entry.getValue(); FunctionHandle functionHandle = originalAggregation.getFunctionHandle(); - InternalAggregationFunction function = functionManager.getAggregateFunctionImplementation(functionHandle); + InternalAggregationFunction function = functionAndTypeManager.getAggregateFunctionImplementation(functionHandle); // create partial aggregation - VariableReferenceExpression partialVariable = variableAllocator.newVariable(functionManager.getFunctionMetadata(functionHandle).getName().getFunctionName(), function.getIntermediateType()); + VariableReferenceExpression partialVariable = variableAllocator.newVariable(functionAndTypeManager.getFunctionMetadata(functionHandle).getName().getFunctionName(), function.getIntermediateType()); partialAggregations.put(partialVariable, new Aggregation( new CallExpression( originalAggregation.getCall().getDisplayName(), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/planPrinter/PlanPrinter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/planPrinter/PlanPrinter.java index 7e38ef3accad..e748c5468292 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/planPrinter/PlanPrinter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/planPrinter/PlanPrinter.java @@ -27,7 +27,7 @@ import com.facebook.presto.execution.TaskInfo; import com.facebook.presto.expressions.DynamicFilters.DynamicFilterExtractResult; import com.facebook.presto.expressions.LogicalRowExpressions; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.OperatorNotFoundException; import com.facebook.presto.operator.StageExecutionDescriptor; import com.facebook.presto.spi.ColumnHandle; @@ -142,7 +142,7 @@ public class PlanPrinter { private final PlanRepresentation representation; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final LogicalRowExpressions logicalRowExpressions; private final Function formatter; @@ -150,22 +150,22 @@ private PlanPrinter( PlanNode planRoot, TypeProvider types, Optional stageExecutionStrategy, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, StatsAndCosts estimatedStatsAndCosts, Session session, Optional> stats) { requireNonNull(planRoot, "planRoot is null"); requireNonNull(types, "types is null"); - requireNonNull(functionManager, "functionManager is null"); + requireNonNull(functionAndTypeManager, "functionManager is null"); requireNonNull(estimatedStatsAndCosts, "estimatedStatsAndCosts is null"); requireNonNull(stats, "stats is null"); - this.functionManager = functionManager; + this.functionAndTypeManager = functionAndTypeManager; this.logicalRowExpressions = new LogicalRowExpressions( - new RowExpressionDeterminismEvaluator(functionManager), - new FunctionResolution(functionManager), - functionManager); + new RowExpressionDeterminismEvaluator(functionAndTypeManager), + new FunctionResolution(functionAndTypeManager), + functionAndTypeManager); Optional totalCpuTime = stats.map(s -> new Duration(s.values().stream() .mapToLong(planNode -> planNode.getPlanNodeCpuTime().toMillis()) @@ -177,7 +177,7 @@ private PlanPrinter( this.representation = new PlanRepresentation(planRoot, types, totalCpuTime, totalScheduledTime); - RowExpressionFormatter rowExpressionFormatter = new RowExpressionFormatter(functionManager); + RowExpressionFormatter rowExpressionFormatter = new RowExpressionFormatter(functionAndTypeManager); ConnectorSession connectorSession = requireNonNull(session, "session is null").toConnectorSession(); this.formatter = rowExpression -> rowExpressionFormatter.formatRowExpression(connectorSession, rowExpression); @@ -195,52 +195,52 @@ public String toJson() return new JsonRenderer().render(representation); } - public static String jsonFragmentPlan(PlanNode root, Set variables, FunctionManager functionManager, Session session) + public static String jsonFragmentPlan(PlanNode root, Set variables, FunctionAndTypeManager functionAndTypeManager, Session session) { TypeProvider typeProvider = TypeProvider.fromVariables(variables); - return new PlanPrinter(root, typeProvider, Optional.empty(), functionManager, StatsAndCosts.empty(), session, Optional.empty()).toJson(); + return new PlanPrinter(root, typeProvider, Optional.empty(), functionAndTypeManager, StatsAndCosts.empty(), session, Optional.empty()).toJson(); } - public static String textLogicalPlan(PlanNode plan, TypeProvider types, FunctionManager functionManager, StatsAndCosts estimatedStatsAndCosts, Session session, int level) + public static String textLogicalPlan(PlanNode plan, TypeProvider types, FunctionAndTypeManager functionAndTypeManager, StatsAndCosts estimatedStatsAndCosts, Session session, int level) { - return new PlanPrinter(plan, types, Optional.empty(), functionManager, estimatedStatsAndCosts, session, Optional.empty()).toText(false, level); + return new PlanPrinter(plan, types, Optional.empty(), functionAndTypeManager, estimatedStatsAndCosts, session, Optional.empty()).toText(false, level); } public static String textLogicalPlan( PlanNode plan, TypeProvider types, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, StatsAndCosts estimatedStatsAndCosts, Session session, int level, boolean verbose) { - return textLogicalPlan(plan, types, Optional.empty(), functionManager, estimatedStatsAndCosts, session, Optional.empty(), level, verbose); + return textLogicalPlan(plan, types, Optional.empty(), functionAndTypeManager, estimatedStatsAndCosts, session, Optional.empty(), level, verbose); } public static String textLogicalPlan( PlanNode plan, TypeProvider types, Optional stageExecutionStrategy, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, StatsAndCosts estimatedStatsAndCosts, Session session, Optional> stats, int level, boolean verbose) { - return new PlanPrinter(plan, types, stageExecutionStrategy, functionManager, estimatedStatsAndCosts, session, stats).toText(verbose, level); + return new PlanPrinter(plan, types, stageExecutionStrategy, functionAndTypeManager, estimatedStatsAndCosts, session, stats).toText(verbose, level); } - public static String textDistributedPlan(StageInfo outputStageInfo, FunctionManager functionManager, Session session, boolean verbose) + public static String textDistributedPlan(StageInfo outputStageInfo, FunctionAndTypeManager functionAndTypeManager, Session session, boolean verbose) { StringBuilder builder = new StringBuilder(); List allStages = getAllStages(Optional.of(outputStageInfo)); Map aggregatedStats = aggregateStageStats(allStages); for (StageInfo stageInfo : allStages) { builder.append(formatFragment( - functionManager, + functionAndTypeManager, session, stageInfo.getPlan().get(), Optional.of(stageInfo), @@ -251,12 +251,12 @@ public static String textDistributedPlan(StageInfo outputStageInfo, FunctionMana return builder.toString(); } - public static String textDistributedPlan(SubPlan plan, FunctionManager functionManager, Session session, boolean verbose) + public static String textDistributedPlan(SubPlan plan, FunctionAndTypeManager functionAndTypeManager, Session session, boolean verbose) { StringBuilder builder = new StringBuilder(); for (PlanFragment fragment : plan.getAllFragments()) { builder.append(formatFragment( - functionManager, + functionAndTypeManager, session, fragment, Optional.empty(), @@ -267,10 +267,10 @@ public static String textDistributedPlan(SubPlan plan, FunctionManager functionM return builder.toString(); } - public static String textPlanFragment(PlanFragment fragment, FunctionManager functionManager, Session session, boolean verbose) + public static String textPlanFragment(PlanFragment fragment, FunctionAndTypeManager functionAndTypeManager, Session session, boolean verbose) { return formatFragment( - functionManager, + functionAndTypeManager, session, fragment, Optional.empty(), @@ -281,23 +281,23 @@ public static String textPlanFragment(PlanFragment fragment, FunctionManager fun public static String jsonLogicalPlan( PlanNode plan, TypeProvider types, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, StatsAndCosts estimatedStatsAndCosts, Session session) { - return jsonLogicalPlan(plan, types, Optional.empty(), functionManager, estimatedStatsAndCosts, session, Optional.empty()); + return jsonLogicalPlan(plan, types, Optional.empty(), functionAndTypeManager, estimatedStatsAndCosts, session, Optional.empty()); } public static String jsonLogicalPlan( PlanNode plan, TypeProvider types, Optional stageExecutionStrategy, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, StatsAndCosts estimatedStatsAndCosts, Session session, Optional> stats) { - return new PlanPrinter(plan, types, stageExecutionStrategy, functionManager, estimatedStatsAndCosts, session, stats).toJson(); + return new PlanPrinter(plan, types, stageExecutionStrategy, functionAndTypeManager, estimatedStatsAndCosts, session, stats).toJson(); } public static String jsonDistributedPlan(StageInfo outputStageInfo) @@ -326,7 +326,7 @@ private static String formatJsonFragmentList(List fragments) } private static String formatFragment( - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, Session session, PlanFragment fragment, Optional stageInfo, @@ -385,7 +385,7 @@ private static String formatFragment( fragment.getRoot(), typeProvider, Optional.of(fragment.getStageExecutionDescriptor()), - functionManager, + functionAndTypeManager, fragment.getStatsAndCosts(), session, planNodeStats, @@ -396,7 +396,7 @@ private static String formatFragment( return builder.toString(); } - public static String graphvizLogicalPlan(PlanNode plan, TypeProvider types, Session session, FunctionManager functionManager) + public static String graphvizLogicalPlan(PlanNode plan, TypeProvider types, Session session, FunctionAndTypeManager functionAndTypeManager) { // TODO: This should move to something like GraphvizRenderer PlanFragment fragment = new PlanFragment( @@ -410,12 +410,12 @@ public static String graphvizLogicalPlan(PlanNode plan, TypeProvider types, Sess false, StatsAndCosts.empty(), Optional.empty()); - return GraphvizPrinter.printLogical(ImmutableList.of(fragment), session, functionManager); + return GraphvizPrinter.printLogical(ImmutableList.of(fragment), session, functionAndTypeManager); } - public static String graphvizDistributedPlan(SubPlan plan, Session session, FunctionManager functionManager) + public static String graphvizDistributedPlan(SubPlan plan, Session session, FunctionAndTypeManager functionAndTypeManager) { - return GraphvizPrinter.printDistributed(plan, session, functionManager); + return GraphvizPrinter.printDistributed(plan, session, functionAndTypeManager); } private class Visitor @@ -472,7 +472,7 @@ public Void visitJoin(JoinNode node, Void context) .collect(Collectors.joining(", ", "{", "}"))); } - node.getSortExpressionContext(functionManager) + node.getSortExpressionContext(functionAndTypeManager) .ifPresent(sortContext -> nodeOutput.appendDetails("SortExpression[%s]", formatter.apply(sortContext.getSortExpression()))); node.getLeft().accept(this, context); node.getRight().accept(this, context); @@ -591,7 +591,7 @@ private String formatAggregation(AggregationNode.Aggregation aggregation) { StringBuilder builder = new StringBuilder(); builder.append("\""); - builder.append(functionManager.getFunctionMetadata(aggregation.getFunctionHandle()).getName()); + builder.append(functionAndTypeManager.getFunctionMetadata(aggregation.getFunctionHandle()).getName()); builder.append("\""); builder.append("("); if (aggregation.isDistinct()) { @@ -1210,7 +1210,7 @@ private String formatDomain(Domain domain) for (Range range : ranges.getOrderedRanges()) { StringBuilder builder = new StringBuilder(); if (range.isSingleValue()) { - String value = castToVarchar(type, range.getSingleValue(), functionManager, session); + String value = castToVarchar(type, range.getSingleValue(), functionAndTypeManager, session); builder.append('[').append(value).append(']'); } else { @@ -1220,7 +1220,7 @@ private String formatDomain(Domain domain) builder.append(""); } else { - builder.append(castToVarchar(type, range.getLow().getValue(), functionManager, session)); + builder.append(castToVarchar(type, range.getLow().getValue(), functionAndTypeManager, session)); } builder.append(", "); @@ -1229,7 +1229,7 @@ private String formatDomain(Domain domain) builder.append(""); } else { - builder.append(castToVarchar(type, range.getHigh().getValue(), functionManager, session)); + builder.append(castToVarchar(type, range.getHigh().getValue(), functionAndTypeManager, session)); } builder.append((range.getHigh().getBound() == Marker.Bound.EXACTLY) ? ']' : ')'); @@ -1238,7 +1238,7 @@ private String formatDomain(Domain domain) } }, discreteValues -> discreteValues.getValues().stream() - .map(value -> castToVarchar(type, value, functionManager, session)) + .map(value -> castToVarchar(type, value, functionAndTypeManager, session)) .sorted() // Sort so the values will be printed in predictable order .forEach(parts::add), allOrNone -> { @@ -1292,15 +1292,15 @@ public NodeRepresentation addNode(PlanNode rootNode, String name, String identif } } - private static String castToVarchar(Type type, Object value, FunctionManager functionManager, Session session) + private static String castToVarchar(Type type, Object value, FunctionAndTypeManager functionAndTypeManager, Session session) { if (value == null) { return "NULL"; } try { - FunctionHandle cast = functionManager.lookupCast(CAST, type.getTypeSignature(), VARCHAR.getTypeSignature()); - Slice coerced = (Slice) new InterpretedFunctionInvoker(functionManager).invoke(cast, session.getSqlFunctionProperties(), value); + FunctionHandle cast = functionAndTypeManager.lookupCast(CAST, type.getTypeSignature(), VARCHAR.getTypeSignature()); + Slice coerced = (Slice) new InterpretedFunctionInvoker(functionAndTypeManager).invoke(cast, session.getSqlFunctionProperties(), value); return coerced.toStringUtf8(); } catch (OperatorNotFoundException e) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/planPrinter/RowExpressionFormatter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/planPrinter/RowExpressionFormatter.java index 69cd5de2ab47..453120c164ca 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/planPrinter/RowExpressionFormatter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/planPrinter/RowExpressionFormatter.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.block.Block; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.function.FunctionMetadataManager; import com.facebook.presto.spi.function.StandardFunctionResolution; @@ -42,10 +42,10 @@ public final class RowExpressionFormatter private final FunctionMetadataManager functionMetadataManager; private final StandardFunctionResolution standardFunctionResolution; - public RowExpressionFormatter(FunctionManager functionManager) + public RowExpressionFormatter(FunctionAndTypeManager functionAndTypeManager) { - this.functionMetadataManager = requireNonNull(functionManager, "function manager is null"); - this.standardFunctionResolution = new FunctionResolution(functionManager); + this.functionMetadataManager = requireNonNull(functionAndTypeManager, "function manager is null"); + this.standardFunctionResolution = new FunctionResolution(functionAndTypeManager); } public String formatRowExpression(ConnectorSession session, RowExpression expression) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/CheckUnsupportedExternalFunctions.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/CheckUnsupportedExternalFunctions.java index a7a87b0b3ec3..030910fa2686 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/CheckUnsupportedExternalFunctions.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/CheckUnsupportedExternalFunctions.java @@ -14,7 +14,7 @@ package com.facebook.presto.sql.planner.sanity; import com.facebook.presto.Session; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.WarningCollector; @@ -40,7 +40,7 @@ public class CheckUnsupportedExternalFunctions @Override public void validate(PlanNode planNode, Session session, Metadata metadata, SqlParser sqlParser, TypeProvider types, WarningCollector warningCollector) { - planNode.accept(new Visitor(metadata.getFunctionManager()), null); + planNode.accept(new Visitor(metadata.getFunctionAndTypeManager()), null); } private static class Visitor @@ -48,9 +48,9 @@ private static class Visitor { private final ExternalCallExpressionChecker externalCallExpressionChecker; - Visitor(FunctionManager functionManager) + Visitor(FunctionAndTypeManager functionAndTypeManager) { - this.externalCallExpressionChecker = new ExternalCallExpressionChecker(requireNonNull(functionManager, "functionManager is null")); + this.externalCallExpressionChecker = new ExternalCallExpressionChecker(requireNonNull(functionAndTypeManager, "functionManager is null")); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/TypeValidator.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/TypeValidator.java index 2237ccc7e5c9..18d034bf76e7 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/TypeValidator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/TypeValidator.java @@ -158,7 +158,7 @@ private void checkWindowFunctions(Map aggregations) { for (Map.Entry entry : aggregations.entrySet()) { - verifyTypeSignature(entry.getKey(), metadata.getFunctionManager().getFunctionMetadata(entry.getValue().getFunctionHandle()).getReturnType()); + verifyTypeSignature(entry.getKey(), metadata.getFunctionAndTypeManager().getFunctionMetadata(entry.getValue().getFunctionHandle()).getReturnType()); } } @@ -181,7 +181,7 @@ private void checkAggregation(Map aggr for (Map.Entry entry : aggregations.entrySet()) { VariableReferenceExpression variable = entry.getKey(); Aggregation aggregation = entry.getValue(); - FunctionMetadata functionMetadata = metadata.getFunctionManager().getFunctionMetadata(aggregation.getFunctionHandle()); + FunctionMetadata functionMetadata = metadata.getFunctionAndTypeManager().getFunctionMetadata(aggregation.getFunctionHandle()); verifyTypeSignature( variable, functionMetadata.getReturnType()); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/VerifyProjectionLocality.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/VerifyProjectionLocality.java index 4e06c9303ba4..d27d30aa71c8 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/VerifyProjectionLocality.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/VerifyProjectionLocality.java @@ -14,7 +14,7 @@ package com.facebook.presto.sql.planner.sanity; import com.facebook.presto.Session; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.plan.PlanNode; @@ -36,7 +36,7 @@ public class VerifyProjectionLocality @Override public void validate(PlanNode planNode, Session session, Metadata metadata, SqlParser sqlParser, TypeProvider types, WarningCollector warningCollector) { - planNode.accept(new Visitor(metadata.getFunctionManager()), null); + planNode.accept(new Visitor(metadata.getFunctionAndTypeManager()), null); } private static class Visitor @@ -44,9 +44,9 @@ private static class Visitor { private final ExternalCallExpressionChecker externalCallExpressionChecker; - Visitor(FunctionManager functionManager) + Visitor(FunctionAndTypeManager functionAndTypeManager) { - this.externalCallExpressionChecker = new ExternalCallExpressionChecker(requireNonNull(functionManager, "functionManager is null")); + this.externalCallExpressionChecker = new ExternalCallExpressionChecker(requireNonNull(functionAndTypeManager, "functionManager is null")); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/Expressions.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/Expressions.java index d12877b3701a..7e94d27fcb34 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/Expressions.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/Expressions.java @@ -14,7 +14,7 @@ package com.facebook.presto.sql.relational; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.relation.CallExpression; import com.facebook.presto.spi.relation.ConstantExpression; @@ -66,14 +66,14 @@ public static CallExpression call(String displayName, FunctionHandle functionHan return new CallExpression(displayName, functionHandle, returnType, arguments); } - public static CallExpression call(FunctionManager functionManager, String name, Type returnType, RowExpression... arguments) + public static CallExpression call(FunctionAndTypeManager functionAndTypeManager, String name, Type returnType, RowExpression... arguments) { - return call(functionManager, name, returnType, ImmutableList.copyOf(arguments)); + return call(functionAndTypeManager, name, returnType, ImmutableList.copyOf(arguments)); } - public static CallExpression call(FunctionManager functionManager, String name, Type returnType, List arguments) + public static CallExpression call(FunctionAndTypeManager functionAndTypeManager, String name, Type returnType, List arguments) { - FunctionHandle functionHandle = functionManager.lookupFunction(name, fromTypes(arguments.stream().map(RowExpression::getType).collect(toImmutableList()))); + FunctionHandle functionHandle = functionAndTypeManager.lookupFunction(name, fromTypes(arguments.stream().map(RowExpression::getType).collect(toImmutableList()))); return call(name, functionHandle, returnType, arguments); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java index a602006b2bc8..f5e50f581cd7 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.function.QualifiedFunctionName; import com.facebook.presto.common.type.CharType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.function.StandardFunctionResolution; import com.facebook.presto.sql.tree.ArithmeticBinaryExpression; @@ -55,17 +55,17 @@ public final class FunctionResolution implements StandardFunctionResolution { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public FunctionResolution(FunctionManager functionManager) + public FunctionResolution(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override public FunctionHandle notFunction() { - return functionManager.lookupFunction("not", fromTypes(BOOLEAN)); + return functionAndTypeManager.lookupFunction("not", fromTypes(BOOLEAN)); } public boolean isNotFunction(FunctionHandle functionHandle) @@ -76,59 +76,59 @@ public boolean isNotFunction(FunctionHandle functionHandle) @Override public FunctionHandle likeVarcharFunction() { - return functionManager.lookupFunction("LIKE", fromTypes(VARCHAR, LIKE_PATTERN)); + return functionAndTypeManager.lookupFunction("LIKE", fromTypes(VARCHAR, LIKE_PATTERN)); } @Override public FunctionHandle likeCharFunction(Type valueType) { checkArgument(valueType instanceof CharType, "Expected CHAR value type"); - return functionManager.lookupFunction("LIKE", fromTypes(valueType, LIKE_PATTERN)); + return functionAndTypeManager.lookupFunction("LIKE", fromTypes(valueType, LIKE_PATTERN)); } public boolean isLikeFunction(FunctionHandle functionHandle) { - return functionManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedFunctionName.of(DEFAULT_NAMESPACE, "LIKE")); + return functionAndTypeManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedFunctionName.of(DEFAULT_NAMESPACE, "LIKE")); } public FunctionHandle likePatternFunction() { - return functionManager.lookupFunction("LIKE_PATTERN", fromTypes(VARCHAR, VARCHAR)); + return functionAndTypeManager.lookupFunction("LIKE_PATTERN", fromTypes(VARCHAR, VARCHAR)); } @Override public boolean isCastFunction(FunctionHandle functionHandle) { - return functionManager.getFunctionMetadata(functionHandle).getOperatorType().equals(Optional.of(OperatorType.CAST)); + return functionAndTypeManager.getFunctionMetadata(functionHandle).getOperatorType().equals(Optional.of(OperatorType.CAST)); } public boolean isTryCastFunction(FunctionHandle functionHandle) { - return functionManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedFunctionName.of(DEFAULT_NAMESPACE, "TRY_CAST")); + return functionAndTypeManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedFunctionName.of(DEFAULT_NAMESPACE, "TRY_CAST")); } public boolean isArrayConstructor(FunctionHandle functionHandle) { - return functionManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedFunctionName.of(DEFAULT_NAMESPACE, ARRAY_CONSTRUCTOR)); + return functionAndTypeManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedFunctionName.of(DEFAULT_NAMESPACE, ARRAY_CONSTRUCTOR)); } @Override public FunctionHandle betweenFunction(Type valueType, Type lowerBoundType, Type upperBoundType) { - return functionManager.lookupFunction(BETWEEN.getFunctionName().getFunctionName(), fromTypes(valueType, lowerBoundType, upperBoundType)); + return functionAndTypeManager.lookupFunction(BETWEEN.getFunctionName().getFunctionName(), fromTypes(valueType, lowerBoundType, upperBoundType)); } @Override public boolean isBetweenFunction(FunctionHandle functionHandle) { - return functionManager.getFunctionMetadata(functionHandle).getOperatorType().equals(Optional.of(BETWEEN)); + return functionAndTypeManager.getFunctionMetadata(functionHandle).getOperatorType().equals(Optional.of(BETWEEN)); } @Override public FunctionHandle arithmeticFunction(OperatorType operator, Type leftType, Type rightType) { checkArgument(operator.isArithmeticOperator(), format("unexpected arithmetic type %s", operator)); - return functionManager.resolveOperator(operator, fromTypes(leftType, rightType)); + return functionAndTypeManager.resolveOperator(operator, fromTypes(leftType, rightType)); } public FunctionHandle arithmeticFunction(ArithmeticBinaryExpression.Operator operator, Type leftType, Type rightType) @@ -159,33 +159,33 @@ public FunctionHandle arithmeticFunction(ArithmeticBinaryExpression.Operator ope @Override public boolean isArithmeticFunction(FunctionHandle functionHandle) { - Optional operatorType = functionManager.getFunctionMetadata(functionHandle).getOperatorType(); + Optional operatorType = functionAndTypeManager.getFunctionMetadata(functionHandle).getOperatorType(); return operatorType.isPresent() && operatorType.get().isArithmeticOperator(); } @Override public FunctionHandle negateFunction(Type type) { - return functionManager.lookupFunction(NEGATION.getFunctionName().getFunctionName(), fromTypes(type)); + return functionAndTypeManager.lookupFunction(NEGATION.getFunctionName().getFunctionName(), fromTypes(type)); } @Override public boolean isNegateFunction(FunctionHandle functionHandle) { - return functionManager.getFunctionMetadata(functionHandle).getOperatorType().equals(Optional.of(NEGATION)); + return functionAndTypeManager.getFunctionMetadata(functionHandle).getOperatorType().equals(Optional.of(NEGATION)); } @Override public FunctionHandle arrayConstructor(List argumentTypes) { - return functionManager.lookupFunction(ARRAY_CONSTRUCTOR, fromTypes(argumentTypes)); + return functionAndTypeManager.lookupFunction(ARRAY_CONSTRUCTOR, fromTypes(argumentTypes)); } @Override public FunctionHandle comparisonFunction(OperatorType operator, Type leftType, Type rightType) { checkArgument(operator.isComparisonOperator(), format("unexpected comparison type %s", operator)); - return functionManager.resolveOperator(operator, fromTypes(leftType, rightType)); + return functionAndTypeManager.resolveOperator(operator, fromTypes(leftType, rightType)); } public FunctionHandle comparisonFunction(ComparisonExpression.Operator operator, Type leftType, Type rightType) @@ -223,76 +223,76 @@ public FunctionHandle comparisonFunction(ComparisonExpression.Operator operator, @Override public boolean isComparisonFunction(FunctionHandle functionHandle) { - Optional operatorType = functionManager.getFunctionMetadata(functionHandle).getOperatorType(); + Optional operatorType = functionAndTypeManager.getFunctionMetadata(functionHandle).getOperatorType(); return operatorType.isPresent() && operatorType.get().isComparisonOperator(); } @Override public FunctionHandle subscriptFunction(Type baseType, Type indexType) { - return functionManager.lookupFunction(SUBSCRIPT.getFunctionName().getFunctionName(), fromTypes(baseType, indexType)); + return functionAndTypeManager.lookupFunction(SUBSCRIPT.getFunctionName().getFunctionName(), fromTypes(baseType, indexType)); } @Override public boolean isSubscriptFunction(FunctionHandle functionHandle) { - return functionManager.getFunctionMetadata(functionHandle).getOperatorType().equals(Optional.of(SUBSCRIPT)); + return functionAndTypeManager.getFunctionMetadata(functionHandle).getOperatorType().equals(Optional.of(SUBSCRIPT)); } public FunctionHandle tryFunction(Type returnType) { - return functionManager.lookupFunction("$internal$try", fromTypes(returnType)); + return functionAndTypeManager.lookupFunction("$internal$try", fromTypes(returnType)); } public boolean isTryFunction(FunctionHandle functionHandle) { - return functionManager.getFunctionMetadata(functionHandle).getName().equals("$internal$try"); + return functionAndTypeManager.getFunctionMetadata(functionHandle).getName().equals("$internal$try"); } public boolean isFailFunction(FunctionHandle functionHandle) { - return functionManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedFunctionName.of(DEFAULT_NAMESPACE, "fail")); + return functionAndTypeManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedFunctionName.of(DEFAULT_NAMESPACE, "fail")); } @Override public boolean isCountFunction(FunctionHandle functionHandle) { - return functionManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedFunctionName.of(DEFAULT_NAMESPACE, "count")); + return functionAndTypeManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedFunctionName.of(DEFAULT_NAMESPACE, "count")); } @Override public FunctionHandle countFunction() { - return functionManager.lookupFunction("count", ImmutableList.of()); + return functionAndTypeManager.lookupFunction("count", ImmutableList.of()); } @Override public FunctionHandle countFunction(Type valueType) { - return functionManager.lookupFunction("count", fromTypes(valueType)); + return functionAndTypeManager.lookupFunction("count", fromTypes(valueType)); } @Override public boolean isMaxFunction(FunctionHandle functionHandle) { - return functionManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedFunctionName.of(DEFAULT_NAMESPACE, "max")); + return functionAndTypeManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedFunctionName.of(DEFAULT_NAMESPACE, "max")); } @Override public FunctionHandle maxFunction(Type valueType) { - return functionManager.lookupFunction("max", fromTypes(valueType)); + return functionAndTypeManager.lookupFunction("max", fromTypes(valueType)); } @Override public boolean isMinFunction(FunctionHandle functionHandle) { - return functionManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedFunctionName.of(DEFAULT_NAMESPACE, "min")); + return functionAndTypeManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedFunctionName.of(DEFAULT_NAMESPACE, "min")); } @Override public FunctionHandle minFunction(Type valueType) { - return functionManager.lookupFunction("min", fromTypes(valueType)); + return functionAndTypeManager.lookupFunction("min", fromTypes(valueType)); } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/RowExpressionDeterminismEvaluator.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/RowExpressionDeterminismEvaluator.java index a7fa75a1e407..fdeb51304c5e 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/RowExpressionDeterminismEvaluator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/RowExpressionDeterminismEvaluator.java @@ -13,7 +13,7 @@ */ package com.facebook.presto.sql.relational; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.function.FunctionHandle; @@ -35,33 +35,33 @@ public class RowExpressionDeterminismEvaluator implements DeterminismEvaluator { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; @Inject public RowExpressionDeterminismEvaluator(Metadata metadata) { - this(requireNonNull(metadata, "metadata is null").getFunctionManager()); + this(requireNonNull(metadata, "metadata is null").getFunctionAndTypeManager()); } - public RowExpressionDeterminismEvaluator(FunctionManager functionManager) + public RowExpressionDeterminismEvaluator(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override public boolean isDeterministic(RowExpression expression) { - return expression.accept(new Visitor(functionManager), null); + return expression.accept(new Visitor(functionAndTypeManager), null); } private static class Visitor implements RowExpressionVisitor { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; - public Visitor(FunctionManager functionManager) + public Visitor(FunctionAndTypeManager functionAndTypeManager) { - this.functionManager = functionManager; + this.functionAndTypeManager = functionAndTypeManager; } @Override @@ -81,7 +81,7 @@ public Boolean visitCall(CallExpression call, Void context) { FunctionHandle functionHandle = call.getFunctionHandle(); try { - if (!functionManager.getFunctionMetadata(functionHandle).isDeterministic()) { + if (!functionAndTypeManager.getFunctionMetadata(functionHandle).isDeterministic()) { return false; } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/RowExpressionDomainTranslator.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/RowExpressionDomainTranslator.java index 7f37e440dc73..dda905b2a4a7 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/RowExpressionDomainTranslator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/RowExpressionDomainTranslator.java @@ -27,7 +27,7 @@ import com.facebook.presto.common.predicate.ValueSet; import com.facebook.presto.common.type.Type; import com.facebook.presto.expressions.LogicalRowExpressions; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.OperatorNotFoundException; import com.facebook.presto.spi.ConnectorSession; @@ -94,7 +94,7 @@ public final class RowExpressionDomainTranslator implements DomainTranslator { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final LogicalRowExpressions logicalRowExpressions; private final StandardFunctionResolution functionResolution; private final Metadata metadata; @@ -103,9 +103,9 @@ public final class RowExpressionDomainTranslator public RowExpressionDomainTranslator(Metadata metadata) { this.metadata = requireNonNull(metadata, "metadata is null"); - this.functionManager = metadata.getFunctionManager(); - this.logicalRowExpressions = new LogicalRowExpressions(new RowExpressionDeterminismEvaluator(functionManager), new FunctionResolution(functionManager), functionManager); - this.functionResolution = new FunctionResolution(functionManager); + this.functionAndTypeManager = metadata.getFunctionAndTypeManager(); + this.logicalRowExpressions = new LogicalRowExpressions(new RowExpressionDeterminismEvaluator(functionAndTypeManager), new FunctionResolution(functionAndTypeManager), functionAndTypeManager); + this.functionResolution = new FunctionResolution(functionAndTypeManager); } @Override @@ -169,7 +169,7 @@ private RowExpression processRange(Type type, Range range, RowExpression referen // specialize the range with BETWEEN expression if possible b/c it is currently more efficient return call( BETWEEN.name(), - functionManager.resolveOperator(BETWEEN, fromTypes(reference.getType(), type, type)), + functionAndTypeManager.resolveOperator(BETWEEN, fromTypes(reference.getType(), type, type)), BOOLEAN, reference, toRowExpression(range.getLow().getValue(), type), @@ -302,7 +302,7 @@ private static class Visitor private final InterpretedFunctionInvoker functionInvoker; private final Metadata metadata; private final ConnectorSession session; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final LogicalRowExpressions logicalRowExpressions; private final DeterminismEvaluator determinismEvaluator; private final StandardFunctionResolution resolution; @@ -310,13 +310,13 @@ private static class Visitor private Visitor(Metadata metadata, ConnectorSession session, ColumnExtractor columnExtractor) { - this.functionInvoker = new InterpretedFunctionInvoker(metadata.getFunctionManager()); + this.functionInvoker = new InterpretedFunctionInvoker(metadata.getFunctionAndTypeManager()); this.metadata = metadata; this.session = session; - this.functionManager = metadata.getFunctionManager(); - this.logicalRowExpressions = new LogicalRowExpressions(new RowExpressionDeterminismEvaluator(functionManager), new FunctionResolution(functionManager), functionManager); - this.determinismEvaluator = new RowExpressionDeterminismEvaluator(functionManager); - this.resolution = new FunctionResolution(functionManager); + this.functionAndTypeManager = metadata.getFunctionAndTypeManager(); + this.logicalRowExpressions = new LogicalRowExpressions(new RowExpressionDeterminismEvaluator(functionAndTypeManager), new FunctionResolution(functionAndTypeManager), functionAndTypeManager); + this.determinismEvaluator = new RowExpressionDeterminismEvaluator(functionAndTypeManager); + this.resolution = new FunctionResolution(functionAndTypeManager); this.columnExtractor = requireNonNull(columnExtractor, "columnExtractor is null"); } @@ -335,7 +335,7 @@ public ExtractionResult visitSpecialForm(SpecialFormExpression node, Boolean ImmutableList.Builder disjuncts = ImmutableList.builder(); for (RowExpression expression : values) { - disjuncts.add(call(EQUAL.name(), functionManager.resolveOperator(EQUAL, fromTypes(target.getType(), expression.getType())), BOOLEAN, target, expression)); + disjuncts.add(call(EQUAL.name(), functionAndTypeManager.resolveOperator(EQUAL, fromTypes(target.getType(), expression.getType())), BOOLEAN, target, expression)); } ExtractionResult extractionResult = or(disjuncts.build()).accept(this, complement); @@ -403,7 +403,7 @@ public ExtractionResult visitCall(CallExpression node, Boolean complement) binaryOperator(LESS_THAN_OR_EQUAL, node.getArguments().get(0), node.getArguments().get(2))).accept(this, complement); } - FunctionMetadata functionMetadata = metadata.getFunctionManager().getFunctionMetadata(node.getFunctionHandle()); + FunctionMetadata functionMetadata = metadata.getFunctionAndTypeManager().getFunctionMetadata(node.getFunctionHandle()); if (functionMetadata.getOperatorType().map(OperatorType::isComparisonOperator).orElse(false)) { Optional optionalNormalized = toNormalizedSimpleComparison(functionMetadata.getOperatorType().get(), node.getArguments().get(0), node.getArguments().get(1)); if (!optionalNormalized.isPresent()) { @@ -555,7 +555,7 @@ private RowExpression binaryOperator(OperatorType operatorType, RowExpression le { return call( operatorType.name(), - metadata.getFunctionManager().resolveOperator(operatorType, fromTypes(left.getType(), right.getType())), + metadata.getFunctionAndTypeManager().resolveOperator(operatorType, fromTypes(left.getType(), right.getType())), BOOLEAN, left, right); @@ -570,7 +570,7 @@ private Optional floorValue(Type fromType, Type toType, Object value) private Optional getSaturatedFloorCastOperator(Type fromType, Type toType) { try { - return Optional.of(metadata.getFunctionManager().lookupCast(SATURATED_FLOOR_CAST, fromType.getTypeSignature(), toType.getTypeSignature())); + return Optional.of(metadata.getFunctionAndTypeManager().lookupCast(SATURATED_FLOOR_CAST, fromType.getTypeSignature(), toType.getTypeSignature())); } catch (OperatorNotFoundException e) { return Optional.empty(); @@ -579,7 +579,7 @@ private Optional getSaturatedFloorCastOperator(Type fromType, Ty private int compareOriginalValueToCoerced(Type originalValueType, Object originalValue, Type coercedValueType, Object coercedValue) { - FunctionHandle castToOriginalTypeOperator = metadata.getFunctionManager().lookupCast(CAST, coercedValueType.getTypeSignature(), originalValueType.getTypeSignature()); + FunctionHandle castToOriginalTypeOperator = metadata.getFunctionAndTypeManager().lookupCast(CAST, coercedValueType.getTypeSignature(), originalValueType.getTypeSignature()); Object coercedValueInOriginalType = functionInvoker.invoke(castToOriginalTypeOperator, session.getSqlFunctionProperties(), coercedValue); Block originalValueBlock = Utils.nativeValueToBlock(originalValueType, originalValue); Block coercedValueBlock = Utils.nativeValueToBlock(originalValueType, coercedValueInOriginalType); @@ -833,7 +833,7 @@ private RowExpression in(RowExpression value, List inList) private RowExpression binaryOperator(OperatorType operatorType, RowExpression left, RowExpression right) { - return call(operatorType.name(), functionManager.resolveOperator(operatorType, fromTypes(left.getType(), right.getType())), BOOLEAN, left, right); + return call(operatorType.name(), functionAndTypeManager.resolveOperator(operatorType, fromTypes(left.getType(), right.getType())), BOOLEAN, left, right); } private RowExpression greaterThan(RowExpression left, RowExpression right) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlFunctionUtils.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlFunctionUtils.java index 05967ad563a2..1dbd40fb9391 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlFunctionUtils.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlFunctionUtils.java @@ -93,7 +93,7 @@ public static RowExpression getSqlFunctionRowExpression(FunctionMetadata functio lambdaCaptureDesugaredExpression, analyzeSqlFunctionExpression(metadata, sqlFunctionProperties, lambdaCaptureDesugaredExpression, variableAllocator.getTypes().allTypes()).getExpressionTypes(), ImmutableMap.of(), - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), Optional.empty(), Optional.empty(), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java index 6ef5c3aaeb02..43fb47931408 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java @@ -23,7 +23,7 @@ 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.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.relation.ConstantExpression; import com.facebook.presto.spi.relation.LambdaDefinitionExpression; import com.facebook.presto.spi.relation.RowExpression; @@ -108,7 +108,7 @@ import static com.facebook.presto.common.type.VarcharType.createVarcharType; import static com.facebook.presto.metadata.CastType.CAST; import static com.facebook.presto.metadata.CastType.TRY_CAST; -import static com.facebook.presto.metadata.FunctionManager.qualifyFunctionName; +import static com.facebook.presto.metadata.FunctionAndTypeManager.qualifyFunctionName; import static com.facebook.presto.spi.relation.SpecialFormExpression.Form.AND; import static com.facebook.presto.spi.relation.SpecialFormExpression.Form.BIND; import static com.facebook.presto.spi.relation.SpecialFormExpression.Form.COALESCE; @@ -151,18 +151,18 @@ public static RowExpression translate( Expression expression, Map, Type> types, Map layout, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, TypeManager typeManager, Session session) { - return translate(expression, types, layout, functionManager, typeManager, Optional.of(session.getUser()), session.getTransactionId(), session.getSqlFunctionProperties()); + return translate(expression, types, layout, functionAndTypeManager, typeManager, Optional.of(session.getUser()), session.getTransactionId(), session.getSqlFunctionProperties()); } public static RowExpression translate( Expression expression, Map, Type> types, Map layout, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, TypeManager typeManager, Optional user, Optional transactionId, @@ -172,7 +172,7 @@ public static RowExpression translate( types, layout, typeManager, - functionManager, + functionAndTypeManager, user, transactionId, sqlFunctionProperties); @@ -187,7 +187,7 @@ private static class Visitor private final Map, Type> types; private final Map layout; private final TypeManager typeManager; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final Optional user; private final Optional transactionId; private final SqlFunctionProperties sqlFunctionProperties; @@ -197,7 +197,7 @@ private Visitor( Map, Type> types, Map layout, TypeManager typeManager, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, Optional user, Optional transactionId, SqlFunctionProperties sqlFunctionProperties) @@ -205,11 +205,11 @@ private Visitor( this.types = ImmutableMap.copyOf(requireNonNull(types, "types is null")); this.layout = layout; this.typeManager = typeManager; - this.functionManager = functionManager; + this.functionAndTypeManager = functionAndTypeManager; this.user = user; this.transactionId = transactionId; this.sqlFunctionProperties = sqlFunctionProperties; - this.functionResolution = new FunctionResolution(functionManager); + this.functionResolution = new FunctionResolution(functionAndTypeManager); } private Type getType(Expression node) @@ -323,14 +323,14 @@ else if (BIGINT.equals(type)) { if (JSON.equals(type)) { return call( "json_parse", - functionManager.lookupFunction("json_parse", fromTypes(VARCHAR)), + functionAndTypeManager.lookupFunction("json_parse", fromTypes(VARCHAR)), getType(node), constant(utf8Slice(node.getValue()), VARCHAR)); } return call( CAST.name(), - functionManager.lookupCast(CAST, VARCHAR.getTypeSignature(), getType(node).getTypeSignature()), + functionAndTypeManager.lookupCast(CAST, VARCHAR.getTypeSignature(), getType(node).getTypeSignature()), getType(node), constant(utf8Slice(node.getValue()), VARCHAR)); } @@ -407,7 +407,7 @@ protected RowExpression visitFunctionCall(FunctionCall node, Void context) .map(TypeSignatureProvider::new) .collect(toImmutableList()); - return call(node.getName().toString(), functionManager.resolveFunction(transactionId, qualifyFunctionName(node.getName()), argumentTypes), getType(node), arguments); + return call(node.getName().toString(), functionAndTypeManager.resolveFunction(transactionId, qualifyFunctionName(node.getName()), argumentTypes), getType(node), arguments); } @Override @@ -479,7 +479,7 @@ protected RowExpression visitArithmeticUnary(ArithmeticUnaryExpression node, Voi case MINUS: return call( NEGATION.name(), - functionManager.resolveOperator(NEGATION, fromTypes(expression.getType())), + functionAndTypeManager.resolveOperator(NEGATION, fromTypes(expression.getType())), getType(node), expression); } @@ -510,10 +510,10 @@ protected RowExpression visitCast(Cast node, Void context) RowExpression value = process(node.getExpression(), context); if (node.isSafe()) { - return call(TRY_CAST.name(), functionManager.lookupCast(TRY_CAST, value.getType().getTypeSignature(), getType(node).getTypeSignature()), getType(node), value); + return call(TRY_CAST.name(), functionAndTypeManager.lookupCast(TRY_CAST, value.getType().getTypeSignature(), getType(node).getTypeSignature()), getType(node), value); } - return call(CAST.name(), functionManager.lookupCast(CAST, value.getType().getTypeSignature(), getType(node).getTypeSignature()), getType(node), value); + return call(CAST.name(), functionAndTypeManager.lookupCast(CAST, value.getType().getTypeSignature(), getType(node).getTypeSignature()), getType(node), value); } @Override @@ -689,7 +689,7 @@ protected RowExpression visitBetweenPredicate(BetweenPredicate node, Void contex return call( BETWEEN.name(), - functionManager.resolveOperator(BETWEEN, fromTypes(value.getType(), min.getType(), max.getType())), + functionAndTypeManager.resolveOperator(BETWEEN, fromTypes(value.getType(), min.getType(), max.getType())), BOOLEAN, value, min, @@ -707,7 +707,7 @@ protected RowExpression visitLikePredicate(LikePredicate node, Void context) return likeFunctionCall(value, call("LIKE_PATTERN", functionResolution.likePatternFunction(), LIKE_PATTERN, pattern, escape)); } - return likeFunctionCall(value, call(CAST.name(), functionManager.lookupCast(CAST, VARCHAR.getTypeSignature(), LIKE_PATTERN.getTypeSignature()), LIKE_PATTERN, pattern)); + return likeFunctionCall(value, call(CAST.name(), functionAndTypeManager.lookupCast(CAST, VARCHAR.getTypeSignature(), LIKE_PATTERN.getTypeSignature()), LIKE_PATTERN, pattern)); } private RowExpression likeFunctionCall(RowExpression value, RowExpression pattern) @@ -741,7 +741,7 @@ protected RowExpression visitSubscriptExpression(SubscriptExpression node, Void } return call( SUBSCRIPT.name(), - functionManager.resolveOperator(SUBSCRIPT, fromTypes(base.getType(), index.getType())), + functionAndTypeManager.resolveOperator(SUBSCRIPT, fromTypes(base.getType(), index.getType())), getType(node), base, index); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/rewrite/ShowQueriesRewrite.java b/presto-main/src/main/java/com/facebook/presto/sql/rewrite/ShowQueriesRewrite.java index fea4fa8af52a..876b4644431f 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/rewrite/ShowQueriesRewrite.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/rewrite/ShowQueriesRewrite.java @@ -99,7 +99,7 @@ import static com.facebook.presto.connector.informationSchema.InformationSchemaMetadata.TABLE_TABLES; import static com.facebook.presto.connector.informationSchema.InformationSchemaMetadata.TABLE_TABLE_PRIVILEGES; import static com.facebook.presto.metadata.BuiltInFunctionNamespaceManager.DEFAULT_NAMESPACE; -import static com.facebook.presto.metadata.FunctionManager.qualifyFunctionName; +import static com.facebook.presto.metadata.FunctionAndTypeManager.qualifyFunctionName; import static com.facebook.presto.metadata.MetadataListing.listCatalogs; import static com.facebook.presto.metadata.MetadataListing.listSchemas; import static com.facebook.presto.metadata.MetadataUtil.createCatalogSchemaName; @@ -497,7 +497,7 @@ protected Node visitShowCreate(ShowCreate node, Void context) protected Node visitShowCreateFunction(ShowCreateFunction node, Void context) { QualifiedFunctionName functionName = qualifyFunctionName(node.getName()); - Collection functions = metadata.getFunctionManager().getFunctions(session.getTransactionId(), functionName); + Collection functions = metadata.getFunctionAndTypeManager().getFunctions(session.getTransactionId(), functionName); if (node.getParameterTypes().isPresent()) { List parameterTypes = node.getParameterTypes().get().stream() .map(TypeSignature::parseTypeSignature) 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 05b5435f5a57..db13a4e84c54 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 @@ -400,7 +400,7 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig, transactionManager, new RowExpressionDomainTranslator(metadata), new RowExpressionPredicateCompiler(metadata), - new RowExpressionDeterminismEvaluator(metadata.getFunctionManager()), + new RowExpressionDeterminismEvaluator(metadata.getFunctionAndTypeManager()), new FilterStatsCalculator(metadata, scalarStatsCalculator, statsNormalizer), blockEncodingManager); @@ -624,7 +624,7 @@ public void createCatalog(String catalogName, String connectorName, Map properties) { - metadata.getFunctionManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties); + metadata.getFunctionAndTypeManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties); } public LocalQueryRunner printPlan() @@ -765,7 +765,7 @@ public List createDrivers(Session session, @Language("SQL") String sql, private List createDrivers(Session session, Plan plan, OutputFactory outputFactory, TaskContext taskContext) { if (printPlan) { - System.out.println(PlanPrinter.textLogicalPlan(plan.getRoot(), plan.getTypes(), metadata.getFunctionManager(), plan.getStatsAndCosts(), session, 0, false)); + System.out.println(PlanPrinter.textLogicalPlan(plan.getRoot(), plan.getTypes(), metadata.getFunctionAndTypeManager(), plan.getStatsAndCosts(), session, 0, false)); } SubPlan subplan = createSubPlans(session, plan, true); diff --git a/presto-main/src/main/java/com/facebook/presto/testing/TestingConnectorContext.java b/presto-main/src/main/java/com/facebook/presto/testing/TestingConnectorContext.java index d746dfeddc85..f3297dd3d723 100644 --- a/presto-main/src/main/java/com/facebook/presto/testing/TestingConnectorContext.java +++ b/presto-main/src/main/java/com/facebook/presto/testing/TestingConnectorContext.java @@ -23,7 +23,7 @@ import com.facebook.presto.cost.FilterStatsCalculator; import com.facebook.presto.cost.ScalarStatsCalculator; import com.facebook.presto.cost.StatsNormalizer; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.InMemoryNodeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.MetadataManager; @@ -58,14 +58,14 @@ public class TestingConnectorContext { private final NodeManager nodeManager = new ConnectorAwareNodeManager(new InMemoryNodeManager(), "testenv", new ConnectorId("test")); private final TypeManager typeManager = new TypeRegistry(); - private final FunctionManager functionManager = new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); - private final StandardFunctionResolution functionResolution = new FunctionResolution(functionManager); + private final FunctionAndTypeManager functionAndTypeManager = new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + private final StandardFunctionResolution functionResolution = new FunctionResolution(functionAndTypeManager); private final PageSorter pageSorter = new PagesIndexPageSorter(new PagesIndex.TestingFactory(false)); private final PageIndexerFactory pageIndexerFactory = new GroupByHashPageIndexerFactory(new JoinCompiler(MetadataManager.createTestMetadataManager(), new FeaturesConfig())); private final Metadata metadata = MetadataManager.createTestMetadataManager(); private final DomainTranslator domainTranslator = new RowExpressionDomainTranslator(metadata); private final PredicateCompiler predicateCompiler = new RowExpressionPredicateCompiler(metadata); - private final DeterminismEvaluator determinismEvaluator = new RowExpressionDeterminismEvaluator(functionManager); + private final DeterminismEvaluator determinismEvaluator = new RowExpressionDeterminismEvaluator(functionAndTypeManager); private final FilterStatsCalculatorService filterStatsCalculatorService = new ConnectorFilterStatsCalculatorService(new FilterStatsCalculator(metadata, new ScalarStatsCalculator(metadata), new StatsNormalizer())); private final BlockEncodingSerde blockEncodingSerde = new BlockEncodingManager(); @@ -84,7 +84,7 @@ public TypeManager getTypeManager() @Override public FunctionMetadataManager getFunctionMetadataManager() { - return functionManager; + return functionAndTypeManager; } @Override @@ -137,7 +137,7 @@ public DeterminismEvaluator getDeterminismEvaluator() @Override public String formatRowExpression(ConnectorSession session, RowExpression expression) { - return new RowExpressionFormatter(functionManager).formatRowExpression(session, expression); + return new RowExpressionFormatter(functionAndTypeManager).formatRowExpression(session, expression); } }; } diff --git a/presto-main/src/main/java/com/facebook/presto/testing/TestingEnvironment.java b/presto-main/src/main/java/com/facebook/presto/testing/TestingEnvironment.java index d02ffad5db7b..b84b19b5e864 100644 --- a/presto-main/src/main/java/com/facebook/presto/testing/TestingEnvironment.java +++ b/presto-main/src/main/java/com/facebook/presto/testing/TestingEnvironment.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.function.OperatorType; import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.type.TypeRegistry; @@ -30,10 +30,10 @@ public class TestingEnvironment private TestingEnvironment() {} public static final TypeManager TYPE_MANAGER = new TypeRegistry(); - public static final FunctionManager FUNCTION_MANAGER = new FunctionManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); + public static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = new FunctionAndTypeManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); public static MethodHandle getOperatorMethodHandle(OperatorType operatorType, Type... parameterTypes) { - return FUNCTION_MANAGER.getBuiltInScalarFunctionImplementation(FUNCTION_MANAGER.resolveOperator(operatorType, fromTypes(parameterTypes))).getMethodHandle(); + return FUNCTION_AND_TYPE_MANAGER.getBuiltInScalarFunctionImplementation(FUNCTION_AND_TYPE_MANAGER.resolveOperator(operatorType, fromTypes(parameterTypes))).getMethodHandle(); } } diff --git a/presto-main/src/main/java/com/facebook/presto/type/MapParametricType.java b/presto-main/src/main/java/com/facebook/presto/type/MapParametricType.java index 8ede483f7383..868cfff5f26d 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/MapParametricType.java +++ b/presto-main/src/main/java/com/facebook/presto/type/MapParametricType.java @@ -20,7 +20,7 @@ import com.facebook.presto.common.type.StandardTypes; import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeParameter; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import java.lang.invoke.MethodHandle; import java.util.List; @@ -44,10 +44,10 @@ public String getName() @Override public Type createType(List parameters) { - throw new IllegalStateException("MapType creation requires map operator MethodHandle. Please use `createType(functionManager, parameters)` instead"); + throw new IllegalStateException("MapType creation requires map operator MethodHandle. Please use `createType(functionAndTypeManager, parameters)` instead"); } - public Type createType(FunctionManager functionManager, List parameters) + public Type createType(FunctionAndTypeManager functionAndTypeManager, List parameters) { checkArgument(parameters.size() == 2, "Expected two parameters, got %s", parameters); TypeParameter firstParameter = parameters.get(0); @@ -59,9 +59,9 @@ public Type createType(FunctionManager functionManager, List para Type keyType = firstParameter.getType(); Type valueType = secondParameter.getType(); - MethodHandle keyNativeEquals = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(OperatorType.EQUAL, fromTypes(keyType, keyType))).getMethodHandle(); + MethodHandle keyNativeEquals = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(OperatorType.EQUAL, fromTypes(keyType, keyType))).getMethodHandle(); MethodHandle keyBlockEquals = compose(keyNativeEquals, nativeValueGetter(keyType), nativeValueGetter(keyType)); - MethodHandle keyNativeHashCode = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(OperatorType.HASH_CODE, fromTypes(keyType))).getMethodHandle(); + MethodHandle keyNativeHashCode = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(OperatorType.HASH_CODE, fromTypes(keyType))).getMethodHandle(); MethodHandle keyBlockHashCode = compose(keyNativeHashCode, nativeValueGetter(keyType)); return new MapType( keyType, diff --git a/presto-main/src/main/java/com/facebook/presto/type/TypeRegistry.java b/presto-main/src/main/java/com/facebook/presto/type/TypeRegistry.java index 4d23bd746ef5..d078c5c8c193 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/TypeRegistry.java +++ b/presto-main/src/main/java/com/facebook/presto/type/TypeRegistry.java @@ -26,7 +26,7 @@ import com.facebook.presto.common.type.TypeSignature; import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.common.type.VarcharType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.type.khyperloglog.KHyperLogLogType; import com.facebook.presto.type.setdigest.SetDigestType; @@ -103,7 +103,7 @@ public final class TypeRegistry private final ConcurrentMap parametricTypes = new ConcurrentHashMap<>(); private final FeaturesConfig featuresConfig; - private FunctionManager functionManager; + private FunctionAndTypeManager functionAndTypeManager; private final LoadingCache parametricTypeCache; @@ -169,10 +169,10 @@ public TypeRegistry(Set types, FeaturesConfig featuresConfig) .build(CacheLoader.from(this::instantiateParametricType)); } - public void setFunctionManager(FunctionManager functionManager) + public void setFunctionManager(FunctionAndTypeManager functionAndTypeManager) { - checkState(this.functionManager == null, "TypeRegistry can only be associated with a single FunctionManager"); - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + checkState(this.functionAndTypeManager == null, "TypeRegistry can only be associated with a single FunctionManager"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); } @Override @@ -211,7 +211,7 @@ private Type instantiateParametricType(TypeSignature signature) throw new IllegalArgumentException("Unknown type " + signature); } else if (parametricType instanceof MapParametricType) { - return ((MapParametricType) parametricType).createType(functionManager, parameters); + return ((MapParametricType) parametricType).createType(functionAndTypeManager, parameters); } Type instantiatedType = parametricType.createType(parameters); diff --git a/presto-main/src/main/java/com/facebook/presto/util/FastutilSetHelper.java b/presto-main/src/main/java/com/facebook/presto/util/FastutilSetHelper.java index 50ce05b8191f..a4938061539f 100644 --- a/presto-main/src/main/java/com/facebook/presto/util/FastutilSetHelper.java +++ b/presto-main/src/main/java/com/facebook/presto/util/FastutilSetHelper.java @@ -14,7 +14,7 @@ package com.facebook.presto.util; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.PrestoException; import it.unimi.dsi.fastutil.Hash; import it.unimi.dsi.fastutil.booleans.BooleanOpenHashSet; @@ -43,23 +43,23 @@ public final class FastutilSetHelper private FastutilSetHelper() {} @SuppressWarnings("unchecked") - public static Set toFastutilHashSet(Set set, Type type, FunctionManager functionManager) + public static Set toFastutilHashSet(Set set, Type type, FunctionAndTypeManager functionAndTypeManager) { // 0.25 as the load factor is chosen because the argument set is assumed to be small (<10000), // and the return set is assumed to be read-heavy. // The performance of InCodeGenerator heavily depends on the load factor being small. Class javaElementType = type.getJavaType(); if (javaElementType == long.class) { - return new LongOpenCustomHashSet((Collection) set, 0.25f, new LongStrategy(functionManager, type)); + return new LongOpenCustomHashSet((Collection) set, 0.25f, new LongStrategy(functionAndTypeManager, type)); } if (javaElementType == double.class) { - return new DoubleOpenCustomHashSet((Collection) set, 0.25f, new DoubleStrategy(functionManager, type)); + return new DoubleOpenCustomHashSet((Collection) set, 0.25f, new DoubleStrategy(functionAndTypeManager, type)); } if (javaElementType == boolean.class) { return new BooleanOpenHashSet((Collection) set, 0.25f); } else if (!type.getJavaType().isPrimitive()) { - return new ObjectOpenCustomHashSet(set, 0.25f, new ObjectStrategy(functionManager, type)); + return new ObjectOpenCustomHashSet(set, 0.25f, new ObjectStrategy(functionAndTypeManager, type)); } else { throw new UnsupportedOperationException("Unsupported native type in set: " + type.getJavaType() + " with type " + type.getTypeSignature()); @@ -92,10 +92,10 @@ private static final class LongStrategy private final MethodHandle hashCodeHandle; private final MethodHandle equalsHandle; - private LongStrategy(FunctionManager functionManager, Type type) + private LongStrategy(FunctionAndTypeManager functionAndTypeManager, Type type) { - hashCodeHandle = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(HASH_CODE, fromTypes(type))).getMethodHandle(); - equalsHandle = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(EQUAL, fromTypes(type, type))).getMethodHandle(); + hashCodeHandle = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(HASH_CODE, fromTypes(type))).getMethodHandle(); + equalsHandle = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(EQUAL, fromTypes(type, type))).getMethodHandle(); } @Override @@ -134,10 +134,10 @@ private static final class DoubleStrategy private final MethodHandle hashCodeHandle; private final MethodHandle equalsHandle; - private DoubleStrategy(FunctionManager functionManager, Type type) + private DoubleStrategy(FunctionAndTypeManager functionAndTypeManager, Type type) { - hashCodeHandle = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(HASH_CODE, fromTypes(type))).getMethodHandle(); - equalsHandle = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(EQUAL, fromTypes(type, type))).getMethodHandle(); + hashCodeHandle = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(HASH_CODE, fromTypes(type))).getMethodHandle(); + equalsHandle = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(EQUAL, fromTypes(type, type))).getMethodHandle(); } @Override @@ -176,12 +176,12 @@ private static final class ObjectStrategy private final MethodHandle hashCodeHandle; private final MethodHandle equalsHandle; - private ObjectStrategy(FunctionManager functionManager, Type type) + private ObjectStrategy(FunctionAndTypeManager functionAndTypeManager, Type type) { - hashCodeHandle = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(HASH_CODE, fromTypes(type))) + hashCodeHandle = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(HASH_CODE, fromTypes(type))) .getMethodHandle() .asType(MethodType.methodType(long.class, Object.class)); - equalsHandle = functionManager.getBuiltInScalarFunctionImplementation(functionManager.resolveOperator(EQUAL, fromTypes(type, type))) + equalsHandle = functionAndTypeManager.getBuiltInScalarFunctionImplementation(functionAndTypeManager.resolveOperator(EQUAL, fromTypes(type, type))) .getMethodHandle() .asType(MethodType.methodType(Boolean.class, Object.class, Object.class)); } diff --git a/presto-main/src/main/java/com/facebook/presto/util/GraphvizPrinter.java b/presto-main/src/main/java/com/facebook/presto/util/GraphvizPrinter.java index 93a343165376..8a3c881a83b3 100644 --- a/presto-main/src/main/java/com/facebook/presto/util/GraphvizPrinter.java +++ b/presto-main/src/main/java/com/facebook/presto/util/GraphvizPrinter.java @@ -14,7 +14,7 @@ package com.facebook.presto.util; import com.facebook.presto.Session; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.AggregationNode.Aggregation; @@ -141,7 +141,7 @@ private enum NodeType private GraphvizPrinter() {} - public static String printLogical(List fragments, Session session, FunctionManager functionManager) + public static String printLogical(List fragments, Session session, FunctionAndTypeManager functionAndTypeManager) { Map fragmentsById = Maps.uniqueIndex(fragments, PlanFragment::getId); PlanNodeIdGenerator idGenerator = new PlanNodeIdGenerator(); @@ -150,7 +150,7 @@ public static String printLogical(List fragments, Session session, output.append("digraph logical_plan {\n"); for (PlanFragment fragment : fragments) { - printFragmentNodes(output, fragment, idGenerator, session, functionManager); + printFragmentNodes(output, fragment, idGenerator, session, functionAndTypeManager); } for (PlanFragment fragment : fragments) { @@ -162,7 +162,7 @@ public static String printLogical(List fragments, Session session, return output.toString(); } - public static String printDistributed(SubPlan plan, Session session, FunctionManager functionManager) + public static String printDistributed(SubPlan plan, Session session, FunctionAndTypeManager functionAndTypeManager) { List fragments = plan.getAllFragments(); Map fragmentsById = Maps.uniqueIndex(fragments, PlanFragment::getId); @@ -171,7 +171,7 @@ public static String printDistributed(SubPlan plan, Session session, FunctionMan StringBuilder output = new StringBuilder(); output.append("digraph distributed_plan {\n"); - printSubPlan(plan, fragmentsById, idGenerator, output, session, functionManager); + printSubPlan(plan, fragmentsById, idGenerator, output, session, functionAndTypeManager); output.append("}\n"); @@ -184,18 +184,18 @@ private static void printSubPlan( PlanNodeIdGenerator idGenerator, StringBuilder output, Session session, - FunctionManager functionManager) + FunctionAndTypeManager functionAndTypeManager) { PlanFragment fragment = plan.getFragment(); - printFragmentNodes(output, fragment, idGenerator, session, functionManager); + printFragmentNodes(output, fragment, idGenerator, session, functionAndTypeManager); fragment.getRoot().accept(new EdgePrinter(output, fragmentsById, idGenerator), null); for (SubPlan child : plan.getChildren()) { - printSubPlan(child, fragmentsById, idGenerator, output, session, functionManager); + printSubPlan(child, fragmentsById, idGenerator, output, session, functionAndTypeManager); } } - private static void printFragmentNodes(StringBuilder output, PlanFragment fragment, PlanNodeIdGenerator idGenerator, Session session, FunctionManager functionManager) + private static void printFragmentNodes(StringBuilder output, PlanFragment fragment, PlanNodeIdGenerator idGenerator, Session session, FunctionAndTypeManager functionAndTypeManager) { String clusterId = "cluster_" + fragment.getId(); output.append("subgraph ") @@ -207,7 +207,7 @@ private static void printFragmentNodes(StringBuilder output, PlanFragment fragme .append('\n'); PlanNode plan = fragment.getRoot(); - plan.accept(new NodePrinter(output, idGenerator, session, functionManager), null); + plan.accept(new NodePrinter(output, idGenerator, session, functionAndTypeManager), null); output.append("}") .append('\n'); @@ -221,11 +221,11 @@ private static class NodePrinter private final PlanNodeIdGenerator idGenerator; private final Function formatter; - public NodePrinter(StringBuilder output, PlanNodeIdGenerator idGenerator, Session session, FunctionManager functionManager) + public NodePrinter(StringBuilder output, PlanNodeIdGenerator idGenerator, Session session, FunctionAndTypeManager functionAndTypeManager) { this.output = output; this.idGenerator = idGenerator; - RowExpressionFormatter rowExpressionFormatter = new RowExpressionFormatter(functionManager); + RowExpressionFormatter rowExpressionFormatter = new RowExpressionFormatter(functionAndTypeManager); ConnectorSession connectorSession = requireNonNull(session, "session is null").toConnectorSession(); this.formatter = rowExpression -> rowExpressionFormatter.formatRowExpression(connectorSession, rowExpression); } diff --git a/presto-main/src/main/java/com/facebook/presto/util/SpatialJoinUtils.java b/presto-main/src/main/java/com/facebook/presto/util/SpatialJoinUtils.java index 5ad4ae2a8946..582679505034 100644 --- a/presto-main/src/main/java/com/facebook/presto/util/SpatialJoinUtils.java +++ b/presto-main/src/main/java/com/facebook/presto/util/SpatialJoinUtils.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.function.OperatorType; import com.facebook.presto.common.function.QualifiedFunctionName; import com.facebook.presto.expressions.LogicalRowExpressions; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.function.FunctionMetadata; import com.facebook.presto.spi.relation.CallExpression; @@ -70,18 +70,18 @@ private SpatialJoinUtils() {} *

* Doesn't check or guarantee anything about function arguments. */ - public static List extractSupportedSpatialFunctions(RowExpression filterExpression, FunctionManager functionManager) + public static List extractSupportedSpatialFunctions(RowExpression filterExpression, FunctionAndTypeManager functionAndTypeManager) { return LogicalRowExpressions.extractConjuncts(filterExpression).stream() .filter(CallExpression.class::isInstance) .map(CallExpression.class::cast) - .filter(call -> isSupportedSpatialFunction(call, functionManager)) + .filter(call -> isSupportedSpatialFunction(call, functionAndTypeManager)) .collect(toImmutableList()); } - private static boolean isSupportedSpatialFunction(CallExpression call, FunctionManager functionManager) + private static boolean isSupportedSpatialFunction(CallExpression call, FunctionAndTypeManager functionAndTypeManager) { - String functionName = functionManager.getFunctionMetadata(call.getFunctionHandle()).getName().getFunctionName().toLowerCase(ENGLISH); + String functionName = functionAndTypeManager.getFunctionMetadata(call.getFunctionHandle()).getName().getFunctionName().toLowerCase(ENGLISH); return ALLOWED_SPATIAL_JOIN_FUNCTIONS.contains(functionName); } @@ -95,45 +95,45 @@ private static boolean isSupportedSpatialFunction(CallExpression call, FunctionM * Doesn't check or guarantee anything about ST_Distance functions arguments * or the other side of the comparison. */ - public static List extractSupportedSpatialComparisons(RowExpression filterExpression, FunctionManager functionManager) + public static List extractSupportedSpatialComparisons(RowExpression filterExpression, FunctionAndTypeManager functionAndTypeManager) { return LogicalRowExpressions.extractConjuncts(filterExpression).stream() .filter(CallExpression.class::isInstance) .map(CallExpression.class::cast) - .filter(call -> new FunctionResolution(functionManager).isComparisonFunction(call.getFunctionHandle())) - .filter(call -> isSupportedSpatialComparison(call, functionManager)) + .filter(call -> new FunctionResolution(functionAndTypeManager).isComparisonFunction(call.getFunctionHandle())) + .filter(call -> isSupportedSpatialComparison(call, functionAndTypeManager)) .collect(toImmutableList()); } - private static boolean isSupportedSpatialComparison(CallExpression expression, FunctionManager functionManager) + private static boolean isSupportedSpatialComparison(CallExpression expression, FunctionAndTypeManager functionAndTypeManager) { - FunctionMetadata metadata = functionManager.getFunctionMetadata(expression.getFunctionHandle()); + FunctionMetadata metadata = functionAndTypeManager.getFunctionMetadata(expression.getFunctionHandle()); checkArgument(metadata.getOperatorType().isPresent() && metadata.getOperatorType().get().isComparisonOperator()); switch (metadata.getOperatorType().get()) { case LESS_THAN: case LESS_THAN_OR_EQUAL: - return isSTDistance(expression.getArguments().get(0), functionManager); + return isSTDistance(expression.getArguments().get(0), functionAndTypeManager); case GREATER_THAN: case GREATER_THAN_OR_EQUAL: - return isSTDistance(expression.getArguments().get(1), functionManager); + return isSTDistance(expression.getArguments().get(1), functionAndTypeManager); default: return false; } } - private static boolean isSTDistance(RowExpression expression, FunctionManager functionManager) + private static boolean isSTDistance(RowExpression expression, FunctionAndTypeManager functionAndTypeManager) { - return expression instanceof CallExpression && functionManager.getFunctionMetadata(((CallExpression) expression).getFunctionHandle()).getName().equals(ST_DISTANCE); + return expression instanceof CallExpression && functionAndTypeManager.getFunctionMetadata(((CallExpression) expression).getFunctionHandle()).getName().equals(ST_DISTANCE); } - public static FunctionHandle getFlippedFunctionHandle(CallExpression callExpression, FunctionManager functionManager) + public static FunctionHandle getFlippedFunctionHandle(CallExpression callExpression, FunctionAndTypeManager functionAndTypeManager) { - FunctionMetadata callExpressionMetadata = functionManager.getFunctionMetadata(callExpression.getFunctionHandle()); + FunctionMetadata callExpressionMetadata = functionAndTypeManager.getFunctionMetadata(callExpression.getFunctionHandle()); checkArgument(callExpressionMetadata.getOperatorType().isPresent()); OperatorType operatorType = flip(callExpressionMetadata.getOperatorType().get()); List typeProviderList = fromTypes(callExpression.getArguments().stream().map(RowExpression::getType).collect(toImmutableList())); checkArgument(typeProviderList.size() == 2, "Expected there to be only two arguments in type provider"); - return functionManager.resolveOperator( + return functionAndTypeManager.resolveOperator( operatorType, ImmutableList.of(typeProviderList.get(1), typeProviderList.get(0))); } diff --git a/presto-main/src/test/java/com/facebook/presto/block/AbstractTestBlock.java b/presto-main/src/test/java/com/facebook/presto/block/AbstractTestBlock.java index 16e2b23a40ee..aab3769c09bf 100644 --- a/presto-main/src/test/java/com/facebook/presto/block/AbstractTestBlock.java +++ b/presto-main/src/test/java/com/facebook/presto/block/AbstractTestBlock.java @@ -20,7 +20,7 @@ import com.facebook.presto.common.block.BlockEncodingSerde; import com.facebook.presto.common.block.DictionaryId; import com.facebook.presto.common.type.TypeManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.type.TypeRegistry; import com.google.common.collect.ImmutableList; @@ -67,7 +67,7 @@ public abstract class AbstractTestBlock static { // associate TYPE_MANAGER with a function manager - new FunctionManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); } protected void assertBlock(Block block, Supplier newBlockBuilder, T[] expectedValues) diff --git a/presto-main/src/test/java/com/facebook/presto/block/ColumnarTestUtils.java b/presto-main/src/test/java/com/facebook/presto/block/ColumnarTestUtils.java index 330eb4b88e8a..c7df70028922 100644 --- a/presto-main/src/test/java/com/facebook/presto/block/ColumnarTestUtils.java +++ b/presto-main/src/test/java/com/facebook/presto/block/ColumnarTestUtils.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.block.DictionaryBlock; import com.facebook.presto.common.block.RunLengthEncodedBlock; import com.facebook.presto.common.type.TypeManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.type.TypeRegistry; import io.airlift.slice.DynamicSliceOutput; @@ -38,7 +38,7 @@ public final class ColumnarTestUtils static { // associate TYPE_MANAGER with a function manager - new FunctionManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); } private ColumnarTestUtils() {} diff --git a/presto-main/src/test/java/com/facebook/presto/block/TestRowBasedSerialization.java b/presto-main/src/test/java/com/facebook/presto/block/TestRowBasedSerialization.java index 98c8387998bc..d04693ede3f0 100644 --- a/presto-main/src/test/java/com/facebook/presto/block/TestRowBasedSerialization.java +++ b/presto-main/src/test/java/com/facebook/presto/block/TestRowBasedSerialization.java @@ -20,7 +20,7 @@ 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.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.type.TypeRegistry; @@ -74,9 +74,9 @@ public class TestRowBasedSerialization public void setUp() { Metadata metadata = MetadataManager.createTestMetadataManager(); - FunctionManager functionManager = metadata.getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); TypeRegistry typeRegistry = new TypeRegistry(); - typeRegistry.setFunctionManager(functionManager); + typeRegistry.setFunctionManager(functionAndTypeManager); this.typeManager = typeRegistry; } diff --git a/presto-main/src/test/java/com/facebook/presto/cost/TestCostCalculator.java b/presto-main/src/test/java/com/facebook/presto/cost/TestCostCalculator.java index a182d7d24833..b92f6db92f61 100644 --- a/presto-main/src/test/java/com/facebook/presto/cost/TestCostCalculator.java +++ b/presto-main/src/test/java/com/facebook/presto/cost/TestCostCalculator.java @@ -798,7 +798,7 @@ private PlanNode project(String id, PlanNode source, VariableReferenceExpression private AggregationNode aggregation(String id, PlanNode source) { - AggregationNode.Aggregation aggregation = count(metadata.getFunctionManager()); + AggregationNode.Aggregation aggregation = count(metadata.getFunctionAndTypeManager()); return new AggregationNode( new PlanNodeId(id), diff --git a/presto-main/src/test/java/com/facebook/presto/cost/TestStatsNormalizer.java b/presto-main/src/test/java/com/facebook/presto/cost/TestStatsNormalizer.java index ddc5f893d910..57352af7e285 100644 --- a/presto-main/src/test/java/com/facebook/presto/cost/TestStatsNormalizer.java +++ b/presto-main/src/test/java/com/facebook/presto/cost/TestStatsNormalizer.java @@ -16,7 +16,7 @@ import com.facebook.presto.block.BlockEncodingManager; import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.relation.VariableReferenceExpression; import com.facebook.presto.sql.analyzer.FeaturesConfig; @@ -42,7 +42,7 @@ public class TestStatsNormalizer { private final TypeManager typeManager = new TypeRegistry(); - private final FunctionManager functionManager = new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + private final FunctionAndTypeManager functionAndTypeManager = new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); private final ConnectorSession session = new TestingConnectorSession(emptyList()); private final StatsNormalizer normalizer = new StatsNormalizer(); @@ -160,6 +160,6 @@ private PlanNodeStatsAssertion assertNormalized(PlanNodeStatsEstimate estimate) private double asStatsValue(Object value, Type type) { - return toStatsRepresentation(functionManager, session, type, value).orElse(NaN); + return toStatsRepresentation(functionAndTypeManager, session, type, value).orElse(NaN); } } diff --git a/presto-main/src/test/java/com/facebook/presto/cost/TestValuesNodeStats.java b/presto-main/src/test/java/com/facebook/presto/cost/TestValuesNodeStats.java index 94d57b7d3169..5901a52d3d7b 100644 --- a/presto-main/src/test/java/com/facebook/presto/cost/TestValuesNodeStats.java +++ b/presto-main/src/test/java/com/facebook/presto/cost/TestValuesNodeStats.java @@ -36,7 +36,7 @@ public class TestValuesNodeStats @Test public void testStatsForValuesNode() { - FunctionResolution resolution = new FunctionResolution(tester().getMetadata().getFunctionManager()); + FunctionResolution resolution = new FunctionResolution(tester().getMetadata().getFunctionAndTypeManager()); tester().assertStatsFor(pb -> pb .values( ImmutableList.of(pb.variable("a", BIGINT), pb.variable("b", DOUBLE)), @@ -89,7 +89,7 @@ public void testStatsForValuesNode() @Test public void testStatsForValuesNodeWithJustNulls() { - FunctionResolution resolution = new FunctionResolution(tester().getMetadata().getFunctionManager()); + FunctionResolution resolution = new FunctionResolution(tester().getMetadata().getFunctionAndTypeManager()); PlanNodeStatsEstimate bigintNullAStats = PlanNodeStatsEstimate.builder() .setOutputRowCount(1) .addVariableStatistics(new VariableReferenceExpression("a", BIGINT), VariableStatsEstimate.zero()) diff --git a/presto-main/src/test/java/com/facebook/presto/metadata/AbstractMockMetadata.java b/presto-main/src/test/java/com/facebook/presto/metadata/AbstractMockMetadata.java index bf7126792344..868a8f966c4c 100644 --- a/presto-main/src/test/java/com/facebook/presto/metadata/AbstractMockMetadata.java +++ b/presto-main/src/test/java/com/facebook/presto/metadata/AbstractMockMetadata.java @@ -503,7 +503,7 @@ public MetadataUpdates getMetadataUpdateResults(Session session, QueryManager qu } @Override - public FunctionManager getFunctionManager() + public FunctionAndTypeManager getFunctionAndTypeManager() { throw new UnsupportedOperationException(); } diff --git a/presto-main/src/test/java/com/facebook/presto/metadata/TestFunctionManager.java b/presto-main/src/test/java/com/facebook/presto/metadata/TestFunctionAndTypeManager.java similarity index 85% rename from presto-main/src/test/java/com/facebook/presto/metadata/TestFunctionManager.java rename to presto-main/src/test/java/com/facebook/presto/metadata/TestFunctionAndTypeManager.java index f256cf9f2505..4133aad1eb38 100644 --- a/presto-main/src/test/java/com/facebook/presto/metadata/TestFunctionManager.java +++ b/presto-main/src/test/java/com/facebook/presto/metadata/TestFunctionAndTypeManager.java @@ -49,7 +49,7 @@ import static com.facebook.presto.common.type.HyperLogLogType.HYPER_LOG_LOG; import static com.facebook.presto.common.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.metadata.FunctionManager.qualifyFunctionName; +import static com.facebook.presto.metadata.FunctionAndTypeManager.qualifyFunctionName; import static com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.ArgumentProperty.valueTypeArgumentProperty; import static com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.NullConvention.RETURN_NULL_ON_NULL; import static com.facebook.presto.spi.function.FunctionKind.SCALAR; @@ -71,14 +71,14 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; -public class TestFunctionManager +public class TestFunctionAndTypeManager { @Test public void testIdentityCast() { TypeRegistry typeManager = new TypeRegistry(); - FunctionManager functionManager = createFunctionManager(typeManager); - FunctionHandle exactOperator = functionManager.lookupCast(CastType.CAST, HYPER_LOG_LOG.getTypeSignature(), HYPER_LOG_LOG.getTypeSignature()); + FunctionAndTypeManager functionAndTypeManager = createFunctionManager(typeManager); + FunctionHandle exactOperator = functionAndTypeManager.lookupCast(CastType.CAST, HYPER_LOG_LOG.getTypeSignature(), HYPER_LOG_LOG.getTypeSignature()); assertEquals(exactOperator, new BuiltInFunctionHandle(new Signature(CAST.getFunctionName(), SCALAR, HYPER_LOG_LOG.getTypeSignature(), HYPER_LOG_LOG.getTypeSignature()))); } @@ -86,9 +86,9 @@ public void testIdentityCast() public void testExactMatchBeforeCoercion() { TypeRegistry typeManager = new TypeRegistry(); - FunctionManager functionManager = createFunctionManager(typeManager); + FunctionAndTypeManager functionAndTypeManager = createFunctionManager(typeManager); boolean foundOperator = false; - for (SqlFunction function : functionManager.listOperators()) { + for (SqlFunction function : functionAndTypeManager.listOperators()) { OperatorType operatorType = tryGetOperatorType(function.getSignature().getName()).get(); if (operatorType == CAST || operatorType == SATURATED_FLOOR_CAST) { continue; @@ -99,7 +99,7 @@ public void testExactMatchBeforeCoercion() if (function.getSignature().getArgumentTypes().stream().anyMatch(TypeSignature::isCalculated)) { continue; } - BuiltInFunctionHandle exactOperator = (BuiltInFunctionHandle) functionManager.resolveOperator(operatorType, fromTypeSignatures(function.getSignature().getArgumentTypes())); + BuiltInFunctionHandle exactOperator = (BuiltInFunctionHandle) functionAndTypeManager.resolveOperator(operatorType, fromTypeSignatures(function.getSignature().getArgumentTypes())); assertEquals(exactOperator.getSignature(), function.getSignature()); foundOperator = true; } @@ -115,9 +115,9 @@ public void testMagicLiteralFunction() assertEquals(signature.getReturnType().getBase(), StandardTypes.TIMESTAMP_WITH_TIME_ZONE); TypeRegistry typeManager = new TypeRegistry(); - FunctionManager functionManager = createFunctionManager(typeManager); - BuiltInFunctionHandle functionHandle = (BuiltInFunctionHandle) functionManager.resolveFunction(TEST_SESSION.getTransactionId(), signature.getName(), fromTypeSignatures(signature.getArgumentTypes())); - assertEquals(functionManager.getFunctionMetadata(functionHandle).getArgumentTypes(), ImmutableList.of(parseTypeSignature(StandardTypes.BIGINT))); + FunctionAndTypeManager functionAndTypeManager = createFunctionManager(typeManager); + BuiltInFunctionHandle functionHandle = (BuiltInFunctionHandle) functionAndTypeManager.resolveFunction(TEST_SESSION.getTransactionId(), signature.getName(), fromTypeSignatures(signature.getArgumentTypes())); + assertEquals(functionAndTypeManager.getFunctionMetadata(functionHandle).getArgumentTypes(), ImmutableList.of(parseTypeSignature(StandardTypes.BIGINT))); assertEquals(signature.getReturnType().getBase(), StandardTypes.TIMESTAMP_WITH_TIME_ZONE); } @@ -132,9 +132,9 @@ public void testDuplicateFunctions() .collect(toImmutableList()); TypeRegistry typeManager = new TypeRegistry(); - FunctionManager functionManager = createFunctionManager(typeManager); - functionManager.registerBuiltInFunctions(functions); - functionManager.registerBuiltInFunctions(functions); + FunctionAndTypeManager functionAndTypeManager = createFunctionManager(typeManager); + functionAndTypeManager.registerBuiltInFunctions(functions); + functionAndTypeManager.registerBuiltInFunctions(functions); } @Test(expectedExceptions = IllegalStateException.class, expectedExceptionsMessageRegExp = "'presto.default.sum' is both an aggregation and a scalar function") @@ -145,16 +145,16 @@ public void testConflictingScalarAggregation() .getFunctions(); TypeRegistry typeManager = new TypeRegistry(); - FunctionManager functionManager = createFunctionManager(typeManager); - functionManager.registerBuiltInFunctions(functions); + FunctionAndTypeManager functionAndTypeManager = createFunctionManager(typeManager); + functionAndTypeManager.registerBuiltInFunctions(functions); } @Test public void testListingVisibilityBetaFunctionsDisabled() { TypeRegistry typeManager = new TypeRegistry(); - FunctionManager functionManager = createFunctionManager(typeManager); - List functions = functionManager.listFunctions(TEST_SESSION); + FunctionAndTypeManager functionAndTypeManager = createFunctionManager(typeManager); + List functions = functionAndTypeManager.listFunctions(TEST_SESSION); List names = transform(functions, input -> input.getSignature().getNameSuffix()); assertTrue(names.contains("length"), "Expected function names " + names + " to contain 'length'"); @@ -176,8 +176,8 @@ public void testListingVisibilityBetaFunctionsEnabled() .setSystemProperty(EXPERIMENTAL_FUNCTIONS_ENABLED, "true") .build(); TypeRegistry typeManager = new TypeRegistry(); - FunctionManager functionManager = createFunctionManager(typeManager); - List functions = functionManager.listFunctions(session); + FunctionAndTypeManager functionAndTypeManager = createFunctionManager(typeManager); + List functions = functionAndTypeManager.listFunctions(session); List names = transform(functions, input -> input.getSignature().getNameSuffix()); assertTrue(names.contains("length"), "Expected function names " + names + " to contain 'length'"); @@ -194,14 +194,14 @@ public void testListingVisibilityBetaFunctionsEnabled() public void testOperatorTypes() { TypeRegistry typeManager = new TypeRegistry(); - FunctionManager functionManager = new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); - FunctionResolution functionResolution = new FunctionResolution(functionManager); - - assertTrue(functionManager.getFunctionMetadata(functionResolution.arithmeticFunction(ADD, BIGINT, BIGINT)).getOperatorType().map(OperatorType::isArithmeticOperator).orElse(false)); - assertFalse(functionManager.getFunctionMetadata(functionResolution.arithmeticFunction(ADD, BIGINT, BIGINT)).getOperatorType().map(OperatorType::isComparisonOperator).orElse(true)); - assertTrue(functionManager.getFunctionMetadata(functionResolution.comparisonFunction(GREATER_THAN, BIGINT, BIGINT)).getOperatorType().map(OperatorType::isComparisonOperator).orElse(false)); - assertFalse(functionManager.getFunctionMetadata(functionResolution.comparisonFunction(GREATER_THAN, BIGINT, BIGINT)).getOperatorType().map(OperatorType::isArithmeticOperator).orElse(true)); - assertFalse(functionManager.getFunctionMetadata(functionResolution.notFunction()).getOperatorType().isPresent()); + FunctionAndTypeManager functionAndTypeManager = new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + FunctionResolution functionResolution = new FunctionResolution(functionAndTypeManager); + + assertTrue(functionAndTypeManager.getFunctionMetadata(functionResolution.arithmeticFunction(ADD, BIGINT, BIGINT)).getOperatorType().map(OperatorType::isArithmeticOperator).orElse(false)); + assertFalse(functionAndTypeManager.getFunctionMetadata(functionResolution.arithmeticFunction(ADD, BIGINT, BIGINT)).getOperatorType().map(OperatorType::isComparisonOperator).orElse(true)); + assertTrue(functionAndTypeManager.getFunctionMetadata(functionResolution.comparisonFunction(GREATER_THAN, BIGINT, BIGINT)).getOperatorType().map(OperatorType::isComparisonOperator).orElse(false)); + assertFalse(functionAndTypeManager.getFunctionMetadata(functionResolution.comparisonFunction(GREATER_THAN, BIGINT, BIGINT)).getOperatorType().map(OperatorType::isArithmeticOperator).orElse(true)); + assertFalse(functionAndTypeManager.getFunctionMetadata(functionResolution.notFunction()).getOperatorType().isPresent()); } @Test @@ -345,11 +345,11 @@ public void testResolveFunctionForUnknown() .failsWithMessage("Could not choose a best candidate operator. Explicit type casts must be added."); } - private FunctionManager createFunctionManager(TypeRegistry typeManager) + private FunctionAndTypeManager createFunctionManager(TypeRegistry typeManager) { BlockEncodingManager blockEncodingManager = new BlockEncodingManager(); FeaturesConfig featuresConfig = new FeaturesConfig(); - return new FunctionManager(typeManager, blockEncodingManager, featuresConfig); + return new FunctionAndTypeManager(typeManager, blockEncodingManager, featuresConfig); } private SignatureBuilder functionSignature(String... argumentTypes) @@ -430,9 +430,9 @@ public ResolveFunctionAssertion failsWithMessage(String... messages) private FunctionHandle resolveFunctionHandle() { FeaturesConfig featuresConfig = new FeaturesConfig(); - FunctionManager functionManager = new FunctionManager(typeRegistry, blockEncoding, featuresConfig); - functionManager.registerBuiltInFunctions(createFunctionsFromSignatures()); - return functionManager.resolveFunction(TEST_SESSION.getTransactionId(), qualifyFunctionName(QualifiedName.of(TEST_FUNCTION_NAME)), fromTypeSignatures(parameterTypes)); + FunctionAndTypeManager functionAndTypeManager = new FunctionAndTypeManager(typeRegistry, blockEncoding, featuresConfig); + functionAndTypeManager.registerBuiltInFunctions(createFunctionsFromSignatures()); + return functionAndTypeManager.resolveFunction(TEST_SESSION.getTransactionId(), qualifyFunctionName(QualifiedName.of(TEST_FUNCTION_NAME)), fromTypeSignatures(parameterTypes)); } private List createFunctionsFromSignatures() @@ -447,7 +447,7 @@ public BuiltInScalarFunctionImplementation specialize( BoundVariables boundVariables, int arity, TypeManager typeManager, - FunctionManager functionManager) + FunctionAndTypeManager functionAndTypeManager) { return new BuiltInScalarFunctionImplementation( false, diff --git a/presto-main/src/test/java/com/facebook/presto/metadata/TestPolymorphicScalarFunction.java b/presto-main/src/test/java/com/facebook/presto/metadata/TestPolymorphicScalarFunction.java index 150540617b7a..576d063c87a0 100644 --- a/presto-main/src/test/java/com/facebook/presto/metadata/TestPolymorphicScalarFunction.java +++ b/presto-main/src/test/java/com/facebook/presto/metadata/TestPolymorphicScalarFunction.java @@ -54,7 +54,7 @@ public class TestPolymorphicScalarFunction { private static final TypeRegistry TYPE_REGISTRY = new TypeRegistry(); - private static final FunctionManager FUNCTION_MANAGER = new FunctionManager(TYPE_REGISTRY, new BlockEncodingManager(), new FeaturesConfig()); + private static final FunctionAndTypeManager FUNCTION_MANAGER = new FunctionAndTypeManager(TYPE_REGISTRY, new BlockEncodingManager(), new FeaturesConfig()); private static final Signature SIGNATURE = SignatureBuilder.builder() .name("foo") .kind(SCALAR) diff --git a/presto-main/src/test/java/com/facebook/presto/metadata/TestSignatureBinder.java b/presto-main/src/test/java/com/facebook/presto/metadata/TestSignatureBinder.java index 0df4dcd8d10c..b51ec3987f94 100644 --- a/presto-main/src/test/java/com/facebook/presto/metadata/TestSignatureBinder.java +++ b/presto-main/src/test/java/com/facebook/presto/metadata/TestSignatureBinder.java @@ -64,7 +64,7 @@ public class TestSignatureBinder TestSignatureBinder() { // associate typeRegistry with a function manager - new FunctionManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); } @Test diff --git a/presto-main/src/test/java/com/facebook/presto/operator/BenchmarkHashAndStreamingAggregationOperators.java b/presto-main/src/test/java/com/facebook/presto/operator/BenchmarkHashAndStreamingAggregationOperators.java index 82d7c763927f..12a4d575b8f8 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/BenchmarkHashAndStreamingAggregationOperators.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/BenchmarkHashAndStreamingAggregationOperators.java @@ -16,7 +16,7 @@ import com.facebook.presto.RowPagesBuilder; import com.facebook.presto.common.Page; import com.facebook.presto.common.block.BlockBuilder; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.HashAggregationOperator.HashAggregationOperatorFactory; import com.facebook.presto.operator.StreamingAggregationOperator.StreamingAggregationOperatorFactory; @@ -80,12 +80,12 @@ public class BenchmarkHashAndStreamingAggregationOperators { private static final MetadataManager metadata = MetadataManager.createTestMetadataManager(); - private static final FunctionManager functionManager = metadata.getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = metadata.getFunctionAndTypeManager(); - private static final InternalAggregationFunction LONG_SUM = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("sum", fromTypes(BIGINT))); - private static final InternalAggregationFunction COUNT = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("count", ImmutableList.of())); + private static final InternalAggregationFunction LONG_SUM = FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("sum", fromTypes(BIGINT))); + private static final InternalAggregationFunction COUNT = FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("count", ImmutableList.of())); @State(Thread) public static class Context diff --git a/presto-main/src/test/java/com/facebook/presto/operator/GenericLongFunction.java b/presto-main/src/test/java/com/facebook/presto/operator/GenericLongFunction.java index 71517c1c2dcd..8be13a3fe4ce 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/GenericLongFunction.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/GenericLongFunction.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.function.QualifiedFunctionName; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation; import com.facebook.presto.spi.function.Signature; @@ -71,7 +71,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { MethodHandle methodHandle = METHOD_HANDLE.bindTo(longUnaryOperator); return new BuiltInScalarFunctionImplementation(false, ImmutableList.of(valueTypeArgumentProperty(RETURN_NULL_ON_NULL)), methodHandle); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestAggregationOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestAggregationOperator.java index cbdd5fac8fb3..94d6d6fb04a1 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestAggregationOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestAggregationOperator.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.Page; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.AggregationOperator.AggregationOperatorFactory; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; @@ -55,7 +55,7 @@ @Test(singleThreaded = true) public class TestAggregationOperator { - private static final FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); private static final InternalAggregationFunction LONG_AVERAGE = getAggregation("avg", BIGINT); private static final InternalAggregationFunction DOUBLE_SUM = getAggregation("sum", DOUBLE); @@ -163,6 +163,6 @@ private void testMemoryTracking(boolean useSystemMemory) private static InternalAggregationFunction getAggregation(String name, Type... arguments) { - return functionManager.getAggregateFunctionImplementation(functionManager.lookupFunction(name, fromTypes(arguments))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(FUNCTION_AND_TYPE_MANAGER.lookupFunction(name, fromTypes(arguments))); } } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestAnnotationEngineForAggregates.java b/presto-main/src/test/java/com/facebook/presto/operator/TestAnnotationEngineForAggregates.java index 790419a1a48a..7e8367d1e7bd 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestAnnotationEngineForAggregates.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestAnnotationEngineForAggregates.java @@ -24,7 +24,7 @@ import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.common.type.VarcharType; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.aggregation.AggregationImplementation; import com.facebook.presto.operator.aggregation.AggregationMetadata; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; @@ -823,8 +823,8 @@ public void testInjectOperatorAggregateParse() assertTrue(implementation.getInputParameterMetadataTypes().equals(expectedMetadataTypes)); TypeManager typeRegistry = new TypeRegistry(); - FunctionManager functionManager = new FunctionManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); - InternalAggregationFunction specialized = aggregation.specialize(BoundVariables.builder().build(), 1, typeRegistry, functionManager); + FunctionAndTypeManager functionAndTypeManager = new FunctionAndTypeManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); + InternalAggregationFunction specialized = aggregation.specialize(BoundVariables.builder().build(), 1, typeRegistry, functionAndTypeManager); assertEquals(specialized.getFinalType(), DoubleType.DOUBLE); assertTrue(specialized.isDecomposable()); assertEquals(specialized.name(), "inject_operator_aggregate"); @@ -907,8 +907,8 @@ public void testInjectTypeAggregateParse() assertTrue(implementation.getInputParameterMetadataTypes().equals(expectedMetadataTypes)); TypeManager typeRegistry = new TypeRegistry(); - FunctionManager functionManager = new FunctionManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); - InternalAggregationFunction specialized = aggregation.specialize(BoundVariables.builder().setTypeVariable("T", DoubleType.DOUBLE).build(), 1, typeRegistry, functionManager); + FunctionAndTypeManager functionAndTypeManager = new FunctionAndTypeManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); + InternalAggregationFunction specialized = aggregation.specialize(BoundVariables.builder().setTypeVariable("T", DoubleType.DOUBLE).build(), 1, typeRegistry, functionAndTypeManager); assertEquals(specialized.getFinalType(), DoubleType.DOUBLE); assertTrue(specialized.isDecomposable()); assertEquals(specialized.name(), "inject_type_aggregate"); @@ -988,8 +988,8 @@ public void testInjectLiteralAggregateParse() assertTrue(implementation.getInputParameterMetadataTypes().equals(expectedMetadataTypes)); TypeManager typeRegistry = new TypeRegistry(); - FunctionManager functionManager = new FunctionManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); - InternalAggregationFunction specialized = aggregation.specialize(BoundVariables.builder().setLongVariable("x", 17L).build(), 1, typeRegistry, functionManager); + FunctionAndTypeManager functionAndTypeManager = new FunctionAndTypeManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); + InternalAggregationFunction specialized = aggregation.specialize(BoundVariables.builder().setLongVariable("x", 17L).build(), 1, typeRegistry, functionAndTypeManager); assertEquals(specialized.getFinalType(), VarcharType.createVarcharType(17)); assertTrue(specialized.isDecomposable()); assertEquals(specialized.name(), "inject_literal_aggregate"); @@ -1060,13 +1060,13 @@ public void testLongConstraintAggregateFunctionParse() assertTrue(implementation.getInputParameterMetadataTypes().equals(expectedMetadataTypes)); TypeManager typeRegistry = new TypeRegistry(); - FunctionManager functionManager = new FunctionManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); + FunctionAndTypeManager functionAndTypeManager = new FunctionAndTypeManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); InternalAggregationFunction specialized = aggregation.specialize( BoundVariables.builder() .setLongVariable("x", 17L) .setLongVariable("y", 13L) .setLongVariable("z", 30L) - .build(), 2, typeRegistry, functionManager); + .build(), 2, typeRegistry, functionAndTypeManager); assertEquals(specialized.getFinalType(), VarcharType.createVarcharType(30)); assertTrue(specialized.isDecomposable()); assertEquals(specialized.name(), "parametric_aggregate_long_constraint"); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestFilterAndProjectOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestFilterAndProjectOperator.java index eaee522ac93d..94ac9812c1bf 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestFilterAndProjectOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestFilterAndProjectOperator.java @@ -14,7 +14,7 @@ package com.facebook.presto.operator; import com.facebook.presto.common.Page; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.project.PageProcessor; import com.facebook.presto.spi.plan.PlanNodeId; @@ -88,10 +88,10 @@ public void test() .build(); MetadataManager metadata = createTestMetadataManager(); - FunctionManager functionManager = metadata.getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); RowExpression filter = call( BETWEEN.name(), - functionManager.resolveOperator(BETWEEN, fromTypes(BIGINT, BIGINT, BIGINT)), + functionAndTypeManager.resolveOperator(BETWEEN, fromTypes(BIGINT, BIGINT, BIGINT)), BOOLEAN, field(1, BIGINT), constant(10L, BIGINT), @@ -100,7 +100,7 @@ public void test() RowExpression field0 = field(0, VARCHAR); RowExpression add5 = call( ADD.name(), - functionManager.resolveOperator(ADD, fromTypes(BIGINT, BIGINT)), + functionAndTypeManager.resolveOperator(ADD, fromTypes(BIGINT, BIGINT)), BIGINT, field(1, BIGINT), constant(5L, BIGINT)); @@ -145,7 +145,7 @@ public void testMergeOutput() RowExpression filter = call( EQUAL.name(), - metadata.getFunctionManager().resolveOperator(EQUAL, fromTypes(BIGINT, BIGINT)), + metadata.getFunctionAndTypeManager().resolveOperator(EQUAL, fromTypes(BIGINT, BIGINT)), BOOLEAN, field(1, BIGINT), constant(10L, BIGINT)); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestHashAggregationOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestHashAggregationOperator.java index ba3b99af7773..faf6e90a808c 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestHashAggregationOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestHashAggregationOperator.java @@ -23,7 +23,7 @@ import com.facebook.presto.common.block.RunLengthEncodedBlock; import com.facebook.presto.common.type.Type; import com.facebook.presto.memory.context.AggregatedMemoryContext; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.HashAggregationOperator.HashAggregationOperatorFactory; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; @@ -97,12 +97,12 @@ @Test(singleThreaded = true) public class TestHashAggregationOperator { - private static final FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); private static final InternalAggregationFunction LONG_AVERAGE = getAggregation("avg", BIGINT); private static final InternalAggregationFunction LONG_SUM = getAggregation("sum", BIGINT); - private static final InternalAggregationFunction COUNT = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("count", ImmutableList.of())); + private static final InternalAggregationFunction COUNT = FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("count", ImmutableList.of())); private static final int MAX_BLOCK_SIZE_IN_BYTES = 64 * 1024; @@ -781,7 +781,7 @@ private int getHashCapacity(Operator operator) private static InternalAggregationFunction getAggregation(String name, Type... arguments) { - return functionManager.getAggregateFunctionImplementation(functionManager.lookupFunction(name, fromTypes(arguments))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(FUNCTION_AND_TYPE_MANAGER.lookupFunction(name, fromTypes(arguments))); } private static class FailingSpillerFactory diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestPositionLinks.java b/presto-main/src/test/java/com/facebook/presto/operator/TestPositionLinks.java index db424e078e71..7226e635e97f 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestPositionLinks.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestPositionLinks.java @@ -315,7 +315,7 @@ private static PagesHashStrategy pagesHashStrategy() ImmutableList.of(), OptionalInt.empty(), Optional.of(0), - MetadataManager.createTestMetadataManager().getFunctionManager(), + MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(), new FeaturesConfig().isGroupByUsesEqualTo()); } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestRealAverageAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/TestRealAverageAggregation.java index 57c33cdd6775..a782eb88a4f4 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestRealAverageAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestRealAverageAggregation.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.block.Block; import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.StandardTypes; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.aggregation.AbstractTestAggregationFunction; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; @@ -41,9 +41,9 @@ public class TestRealAverageAggregation @BeforeClass public void setUp() { - FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); - avgFunction = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("avg", fromTypes(REAL))); + FunctionAndTypeManager functionAndTypeManager = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); + avgFunction = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("avg", fromTypes(REAL))); } @Test diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestScanFilterAndProjectOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestScanFilterAndProjectOperator.java index d87c119bcd0c..25c6b62b4356 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestScanFilterAndProjectOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestScanFilterAndProjectOperator.java @@ -19,7 +19,7 @@ import com.facebook.presto.common.block.Block; import com.facebook.presto.common.block.LazyBlock; import com.facebook.presto.common.block.LazyBlockLoader; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.metadata.Split; @@ -153,7 +153,7 @@ public void testPageSourceMergeOutput() RowExpression filter = call( EQUAL.name(), - createTestMetadataManager().getFunctionManager().resolveOperator(EQUAL, fromTypes(BIGINT, BIGINT)), + createTestMetadataManager().getFunctionAndTypeManager().resolveOperator(EQUAL, fromTypes(BIGINT, BIGINT)), BOOLEAN, field(0, BIGINT), constant(10L, BIGINT)); @@ -332,14 +332,14 @@ public void testPageYield() })); } Metadata metadata = functionAssertions.getMetadata(); - FunctionManager functionManager = metadata.getFunctionManager(); - functionManager.registerBuiltInFunctions(functions.build()); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); + functionAndTypeManager.registerBuiltInFunctions(functions.build()); // match each column with a projection ExpressionCompiler expressionCompiler = new ExpressionCompiler(metadata, new PageFunctionCompiler(metadata, 0)); ImmutableList.Builder projections = ImmutableList.builder(); for (int i = 0; i < totalColumns; i++) { - projections.add(call("generic_long_page_col", functionManager.lookupFunction("generic_long_page_col" + i, fromTypes(BIGINT)), BIGINT, field(0, BIGINT))); + projections.add(call("generic_long_page_col", functionAndTypeManager.lookupFunction("generic_long_page_col" + i, fromTypes(BIGINT)), BIGINT, field(0, BIGINT))); } Supplier cursorProcessor = expressionCompiler.compileCursorProcessor(driverContext.getSession().getSqlFunctionProperties(), Optional.empty(), projections.build(), "key"); Supplier pageProcessor = expressionCompiler.compilePageProcessor(driverContext.getSession().getSqlFunctionProperties(), Optional.empty(), projections.build(), false, MAX_BATCH_SIZE); @@ -397,8 +397,8 @@ public void testRecordCursorYield() // set up generic long function with a callback to force yield Metadata metadata = functionAssertions.getMetadata(); - FunctionManager functionManager = metadata.getFunctionManager(); - functionManager.registerBuiltInFunctions(ImmutableList.of(new GenericLongFunction("record_cursor", value -> { + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); + functionAndTypeManager.registerBuiltInFunctions(ImmutableList.of(new GenericLongFunction("record_cursor", value -> { driverContext.getYieldSignal().forceYieldForTesting(); return value; }))); @@ -406,7 +406,7 @@ public void testRecordCursorYield() List projections = ImmutableList.of(call( "generic_long_record_cursor", - functionManager.lookupFunction("generic_long_record_cursor", fromTypes(BIGINT)), + functionAndTypeManager.lookupFunction("generic_long_record_cursor", fromTypes(BIGINT)), BIGINT, field(0, BIGINT))); Supplier cursorProcessor = expressionCompiler.compileCursorProcessor(driverContext.getSession().getSqlFunctionProperties(), Optional.empty(), projections, "key"); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestStreamingAggregationOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestStreamingAggregationOperator.java index f7c55caff36e..49a045c138fc 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestStreamingAggregationOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestStreamingAggregationOperator.java @@ -15,7 +15,7 @@ import com.facebook.presto.RowPagesBuilder; import com.facebook.presto.common.Page; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.StreamingAggregationOperator.StreamingAggregationOperatorFactory; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; @@ -50,12 +50,12 @@ @Test(singleThreaded = true) public class TestStreamingAggregationOperator { - private static final FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); - private static final InternalAggregationFunction LONG_SUM = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("sum", fromTypes(BIGINT))); - private static final InternalAggregationFunction COUNT = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("count", ImmutableList.of())); + private static final InternalAggregationFunction LONG_SUM = FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("sum", fromTypes(BIGINT))); + private static final InternalAggregationFunction COUNT = FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("count", ImmutableList.of())); private ExecutorService executor; private ScheduledExecutorService scheduledExecutor; diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestTableFinishOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestTableFinishOperator.java index 7f14deb53a16..0bf9ba592c1c 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestTableFinishOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestTableFinishOperator.java @@ -20,7 +20,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.execution.Lifespan; import com.facebook.presto.execution.TaskId; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.TableFinishOperator.PageSinkCommitter; import com.facebook.presto.operator.TableFinishOperator.TableFinishOperatorFactory; import com.facebook.presto.operator.TableFinishOperator.TableFinisher; @@ -74,9 +74,9 @@ public class TestTableFinishOperator { - private static final FunctionManager functionManager = createTestMetadataManager().getFunctionManager(); - private static final InternalAggregationFunction LONG_MAX = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("max", fromTypes(BIGINT))); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = createTestMetadataManager().getFunctionAndTypeManager(); + private static final InternalAggregationFunction LONG_MAX = FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("max", fromTypes(BIGINT))); private static final JsonCodec TABLE_COMMIT_CONTEXT_CODEC = jsonCodec(TableCommitContext.class); private ScheduledExecutorService scheduledExecutor; diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestTableWriterOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestTableWriterOperator.java index b53b43458ef3..4fb9e806311e 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestTableWriterOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestTableWriterOperator.java @@ -25,7 +25,7 @@ import com.facebook.presto.execution.scheduler.ExecutionWriterTarget.CreateHandle; import com.facebook.presto.memory.context.MemoryTrackingContext; import com.facebook.presto.metadata.ConnectorMetadataUpdaterManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.OutputTableHandle; import com.facebook.presto.operator.AggregationOperator.AggregationOperatorFactory; import com.facebook.presto.operator.DevNullOperator.DevNullOperatorFactory; @@ -208,9 +208,9 @@ public void testStatisticsAggregation() TaskContext taskContext = createTaskContext(executor, scheduledExecutor, session); DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); TaskMetadataContext taskMetadataContext = taskContext.getTaskMetadataContext(); - FunctionManager functionManager = createTestMetadataManager().getFunctionManager(); - InternalAggregationFunction longMaxFunction = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("max", fromTypes(BIGINT))); + FunctionAndTypeManager functionAndTypeManager = createTestMetadataManager().getFunctionAndTypeManager(); + InternalAggregationFunction longMaxFunction = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("max", fromTypes(BIGINT))); TableWriterOperator operator = (TableWriterOperator) createTableWriterOperator( pageSinkManager, new AggregationOperatorFactory( diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestAggregationFunction.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestAggregationFunction.java index 375d9a6b8c40..1fe396550e30 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestAggregationFunction.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestAggregationFunction.java @@ -20,7 +20,7 @@ import com.facebook.presto.common.block.RunLengthEncodedBlock; import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeSignature; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.Plugin; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.sql.analyzer.FeaturesConfig; @@ -34,8 +34,8 @@ import java.util.List; +import static com.facebook.presto.metadata.FunctionAndTypeManager.qualifyFunctionName; import static com.facebook.presto.metadata.FunctionExtractor.extractFunctions; -import static com.facebook.presto.metadata.FunctionManager.qualifyFunctionName; import static com.facebook.presto.operator.aggregation.AggregationTestUtils.assertAggregation; import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypeSignatures; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; @@ -44,7 +44,7 @@ public abstract class AbstractTestAggregationFunction { protected TypeRegistry typeRegistry; - protected FunctionManager functionManager; + protected FunctionAndTypeManager functionAndTypeManager; protected Session session; protected AbstractTestAggregationFunction() @@ -61,13 +61,13 @@ protected AbstractTestAggregationFunction(Session session) public final void initTestAggregationFunction() { typeRegistry = new TypeRegistry(); - functionManager = new FunctionManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); + functionAndTypeManager = new FunctionAndTypeManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); } @AfterClass(alwaysRun = true) public final void destroyTestAggregationFunction() { - functionManager = null; + functionAndTypeManager = null; typeRegistry = null; } @@ -75,7 +75,7 @@ public final void destroyTestAggregationFunction() protected void registerFunctions(Plugin plugin) { - functionManager.registerBuiltInFunctions(extractFunctions(plugin.getFunctions())); + functionAndTypeManager.registerBuiltInFunctions(extractFunctions(plugin.getFunctions())); } protected void registerTypes(Plugin plugin) @@ -88,8 +88,8 @@ protected void registerTypes(Plugin plugin) protected final InternalAggregationFunction getFunction() { List parameterTypes = fromTypeSignatures(Lists.transform(getFunctionParameterTypes(), TypeSignature::parseTypeSignature)); - FunctionHandle functionHandle = functionManager.resolveFunction(session.getTransactionId(), qualifyFunctionName(QualifiedName.of(getFunctionName())), parameterTypes); - return functionManager.getAggregateFunctionImplementation(functionHandle); + FunctionHandle functionHandle = functionAndTypeManager.resolveFunction(session.getTransactionId(), qualifyFunctionName(QualifiedName.of(getFunctionName())), parameterTypes); + return functionAndTypeManager.getAggregateFunctionImplementation(functionHandle); } protected abstract String getFunctionName(); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestApproximateCountDistinct.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestApproximateCountDistinct.java index 7645e6a5c38c..c2a04724dcbf 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestApproximateCountDistinct.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestApproximateCountDistinct.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.block.Block; import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -47,7 +47,7 @@ public abstract class AbstractTestApproximateCountDistinct public abstract Object randomValue(); - protected static final FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); + protected static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); protected int getUniqueValuesCount() { diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/BenchmarkArrayAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/BenchmarkArrayAggregation.java index 77c756f408e1..dd32b549cd68 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/BenchmarkArrayAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/BenchmarkArrayAggregation.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.block.Block; import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.google.common.collect.ImmutableList; import io.airlift.slice.Slices; @@ -83,7 +83,7 @@ public static class BenchmarkData @Setup(Invocation) public void setup() { - FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); Block block; Type elementType; switch (type) { @@ -103,8 +103,8 @@ public void setup() throw new UnsupportedOperationException(); } - InternalAggregationFunction function = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction(name, fromTypes(elementType))); + InternalAggregationFunction function = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction(name, fromTypes(elementType))); accumulator = function.bind(ImmutableList.of(0), Optional.empty()).createAccumulator(); block = createChannel(ARRAY_SIZE, elementType); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/BenchmarkGroupedTypedHistogram.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/BenchmarkGroupedTypedHistogram.java index b9110bc83424..5be8675df251 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/BenchmarkGroupedTypedHistogram.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/BenchmarkGroupedTypedHistogram.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.Page; import com.facebook.presto.common.block.Block; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.GroupByIdBlock; import com.facebook.presto.operator.aggregation.groupByAggregations.GroupByAggregationTestUtils; @@ -154,10 +154,10 @@ public GroupedAccumulator testSharedGroupWithLargeBlocksRunner(Data data) private static InternalAggregationFunction getInternalAggregationFunctionVarChar(HistogramGroupImplementation groupMode) { - FunctionManager functionManager = getMetadata(groupMode).getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = getMetadata(groupMode).getFunctionAndTypeManager(); - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction(NAME, fromTypes(VARCHAR))); + return functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction(NAME, fromTypes(VARCHAR))); } private static MetadataManager getMetadata(HistogramGroupImplementation groupMode) diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctBoolean.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctBoolean.java index bf67ec9365d3..babda73a72e7 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctBoolean.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctBoolean.java @@ -33,8 +33,8 @@ public class TestApproximateCountDistinctBoolean @Override public InternalAggregationFunction getAggregationFunction() { - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("approx_distinct", fromTypes(BOOLEAN, DOUBLE))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("approx_distinct", fromTypes(BOOLEAN, DOUBLE))); } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctDouble.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctDouble.java index 0d49a01b3fb8..5aca77bb6f42 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctDouble.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctDouble.java @@ -26,8 +26,8 @@ public class TestApproximateCountDistinctDouble @Override public InternalAggregationFunction getAggregationFunction() { - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("approx_distinct", fromTypes(DOUBLE, DOUBLE))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("approx_distinct", fromTypes(DOUBLE, DOUBLE))); } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctInteger.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctInteger.java index 069bc505cf2a..abedb435f840 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctInteger.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctInteger.java @@ -27,8 +27,8 @@ public class TestApproximateCountDistinctInteger @Override public InternalAggregationFunction getAggregationFunction() { - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("approx_distinct", fromTypes(INTEGER, DOUBLE))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("approx_distinct", fromTypes(INTEGER, DOUBLE))); } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctIpAddress.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctIpAddress.java index 793036a304aa..e2fafd5f7590 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctIpAddress.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctIpAddress.java @@ -28,8 +28,8 @@ public class TestApproximateCountDistinctIpAddress @Override public InternalAggregationFunction getAggregationFunction() { - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("approx_distinct", fromTypes(IPADDRESS, DOUBLE))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("approx_distinct", fromTypes(IPADDRESS, DOUBLE))); } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctLong.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctLong.java index d174b858ea1d..3e07c09d4bc8 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctLong.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctLong.java @@ -27,8 +27,8 @@ public class TestApproximateCountDistinctLong @Override public InternalAggregationFunction getAggregationFunction() { - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("approx_distinct", fromTypes(BIGINT, DOUBLE))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("approx_distinct", fromTypes(BIGINT, DOUBLE))); } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctLongDecimal.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctLongDecimal.java index 4abe34d084ca..9171da6d9273 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctLongDecimal.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctLongDecimal.java @@ -31,8 +31,8 @@ public class TestApproximateCountDistinctLongDecimal @Override public InternalAggregationFunction getAggregationFunction() { - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("approx_distinct", fromTypes(LONG_DECIMAL, DOUBLE))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("approx_distinct", fromTypes(LONG_DECIMAL, DOUBLE))); } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctSmallint.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctSmallint.java index 823b4398fbe7..33b7969c10ab 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctSmallint.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctSmallint.java @@ -27,8 +27,8 @@ public class TestApproximateCountDistinctSmallint @Override public InternalAggregationFunction getAggregationFunction() { - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("approx_distinct", fromTypes(SMALLINT, DOUBLE))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("approx_distinct", fromTypes(SMALLINT, DOUBLE))); } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctTinyint.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctTinyint.java index 1d6581e76fb1..7718f857437b 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctTinyint.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctTinyint.java @@ -27,8 +27,8 @@ public class TestApproximateCountDistinctTinyint @Override public InternalAggregationFunction getAggregationFunction() { - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("approx_distinct", fromTypes(TINYINT, DOUBLE))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("approx_distinct", fromTypes(TINYINT, DOUBLE))); } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctVarBinary.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctVarBinary.java index 016c4f1d4676..e51365538c75 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctVarBinary.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximateCountDistinctVarBinary.java @@ -28,8 +28,8 @@ public class TestApproximateCountDistinctVarBinary @Override public InternalAggregationFunction getAggregationFunction() { - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("approx_distinct", fromTypes(VARCHAR, DOUBLE))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("approx_distinct", fromTypes(VARCHAR, DOUBLE))); } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximatePercentileAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximatePercentileAggregation.java index c482c15f0993..3029a480ca03 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximatePercentileAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestApproximatePercentileAggregation.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.block.RunLengthEncodedBlock; import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.google.common.collect.ImmutableList; import org.testng.annotations.Test; @@ -37,7 +37,7 @@ public class TestApproximatePercentileAggregation { - private static final FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); private static final InternalAggregationFunction DOUBLE_APPROXIMATE_PERCENTILE_AGGREGATION = getAggregation(DOUBLE, DOUBLE); private static final InternalAggregationFunction DOUBLE_APPROXIMATE_PERCENTILE_AGGREGATION_WITH_ACCURACY = getAggregation(DOUBLE, DOUBLE, DOUBLE); @@ -496,7 +496,7 @@ public void testDoublePartialStep() private static InternalAggregationFunction getAggregation(Type... arguments) { - return functionManager.getAggregateFunctionImplementation(functionManager.lookupFunction("approx_percentile", fromTypes(arguments))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(FUNCTION_AND_TYPE_MANAGER.lookupFunction("approx_percentile", fromTypes(arguments))); } private static RunLengthEncodedBlock createRLEBlock(double percentile, int positionCount) diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestArbitraryAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestArbitraryAggregation.java index 6097090250be..019bab8226ff 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestArbitraryAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestArbitraryAggregation.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.google.common.collect.ImmutableList; import org.testng.annotations.Test; @@ -42,7 +42,7 @@ public class TestArbitraryAggregation { private static final MetadataManager metadata = MetadataManager.createTestMetadataManager(); - private static final FunctionManager functionManager = metadata.getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = metadata.getFunctionAndTypeManager(); @Test public void testAllRegistered() @@ -166,6 +166,6 @@ public void testValidInt() private static InternalAggregationFunction getAggregation(Type... arguments) { - return functionManager.getAggregateFunctionImplementation(functionManager.lookupFunction("arbitrary", fromTypes(arguments))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(FUNCTION_AND_TYPE_MANAGER.lookupFunction("arbitrary", fromTypes(arguments))); } } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestArrayAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestArrayAggregation.java index e9e0f550bf85..7b6f07ee600b 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestArrayAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestArrayAggregation.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.SqlDate; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.aggregation.groupByAggregations.AggregationTestInput; import com.facebook.presto.operator.aggregation.groupByAggregations.AggregationTestInputBuilder; @@ -49,7 +49,7 @@ public class TestArrayAggregation { - private static final FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); @Test public void testEmpty() @@ -223,6 +223,6 @@ private GroupedAccumulator createGroupedAccumulator(InternalAggregationFunction private InternalAggregationFunction getAggregation(Type... arguments) { - return functionManager.getAggregateFunctionImplementation(functionManager.lookupFunction("array_agg", fromTypes(arguments))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(FUNCTION_AND_TYPE_MANAGER.lookupFunction("array_agg", fromTypes(arguments))); } } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestChecksumAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestChecksumAggregation.java index 292a61709a71..6313419b8991 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestChecksumAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestChecksumAggregation.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.DecimalType; import com.facebook.presto.common.type.SqlVarbinary; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import org.testng.annotations.Test; @@ -42,7 +42,7 @@ public class TestChecksumAggregation { - private static final FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); @Test public void testEmpty() @@ -126,6 +126,6 @@ private static SqlVarbinary expectedChecksum(Type type, Block block) private InternalAggregationFunction getAggregation(Type argument) { - return functionManager.getAggregateFunctionImplementation(functionManager.lookupFunction("checksum", fromTypes(argument))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(FUNCTION_AND_TYPE_MANAGER.lookupFunction("checksum", fromTypes(argument))); } } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestCountNullAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestCountNullAggregation.java index 25c0b91b4be8..d02ee0ff9580 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestCountNullAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestCountNullAggregation.java @@ -39,7 +39,7 @@ public class TestCountNullAggregation @BeforeClass public void setup() { - functionManager.registerBuiltInFunctions(new FunctionListBuilder().aggregates(CountNull.class).getFunctions()); + functionAndTypeManager.registerBuiltInFunctions(new FunctionListBuilder().aggregates(CountNull.class).getFunctions()); } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestDoubleHistogramAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestDoubleHistogramAggregation.java index f3cc2284f5b8..632649a2e2fc 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestDoubleHistogramAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestDoubleHistogramAggregation.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.PageBuilder; import com.facebook.presto.common.block.Block; import com.facebook.presto.common.type.MapType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.spi.PrestoException; import com.google.common.collect.ImmutableList; @@ -43,9 +43,9 @@ public class TestDoubleHistogramAggregation public TestDoubleHistogramAggregation() { - FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); - InternalAggregationFunction function = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("numeric_histogram", fromTypes(BIGINT, DOUBLE, DOUBLE))); + FunctionAndTypeManager functionAndTypeManager = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); + InternalAggregationFunction function = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("numeric_histogram", fromTypes(BIGINT, DOUBLE, DOUBLE))); factory = function.bind(ImmutableList.of(0, 1, 2), Optional.empty()); input = makeInput(10); } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestEntropyAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestEntropyAggregation.java index 21b295798920..5f8cac390853 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestEntropyAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestEntropyAggregation.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.block.Block; import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.StandardTypes; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.spi.PrestoException; import com.google.common.collect.ImmutableList; @@ -45,9 +45,9 @@ public class TestEntropyAggregation @BeforeClass public void setUp() { - FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); - entropyFunction = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction(TestEntropyAggregation.FUNCTION_NAME, fromTypes(BIGINT))); + FunctionAndTypeManager functionAndTypeManager = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); + entropyFunction = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction(TestEntropyAggregation.FUNCTION_NAME, fromTypes(BIGINT))); } @Test diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestHistogram.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestHistogram.java index c418cb0a6cf4..7a548b8ac192 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestHistogram.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestHistogram.java @@ -22,7 +22,7 @@ import com.facebook.presto.common.type.SqlTimestampWithTimeZone; import com.facebook.presto.common.type.TimeZoneKey; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.aggregation.groupByAggregations.AggregationTestInput; import com.facebook.presto.operator.aggregation.groupByAggregations.AggregationTestInputBuilder; @@ -388,18 +388,18 @@ private InternalAggregationFunction getInternalDefaultVarCharAggregationn() private InternalAggregationFunction getAggregation(Type... arguments) { - FunctionManager functionManager = getFunctionManager(NEW); - return functionManager.getAggregateFunctionImplementation(functionManager.lookupFunction(NAME, fromTypes(arguments))); + FunctionAndTypeManager functionAndTypeManager = getFunctionManager(NEW); + return functionAndTypeManager.getAggregateFunctionImplementation(functionAndTypeManager.lookupFunction(NAME, fromTypes(arguments))); } - public FunctionManager getFunctionManager() + public FunctionAndTypeManager getFunctionManager() { return getFunctionManager(NEW); } - public FunctionManager getFunctionManager(HistogramGroupImplementation groupMode) + public FunctionAndTypeManager getFunctionManager(HistogramGroupImplementation groupMode) { return MetadataManager.createTestMetadataManager(new FeaturesConfig() - .setHistogramGroupImplementation(groupMode)).getFunctionManager(); + .setHistogramGroupImplementation(groupMode)).getFunctionAndTypeManager(); } } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMapAggAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMapAggAggregation.java index ea327ae12275..7f2a41bb2068 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMapAggAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMapAggAggregation.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.MapType; import com.facebook.presto.common.type.RowType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -45,7 +45,7 @@ public class TestMapAggAggregation { - private static final FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); @Test public void testDuplicateKeysValues() @@ -188,6 +188,6 @@ public void testArrayDoubleMap() private InternalAggregationFunction getAggregation(Type... arguments) { - return functionManager.getAggregateFunctionImplementation(functionManager.lookupFunction(NAME, fromTypes(arguments))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(FUNCTION_AND_TYPE_MANAGER.lookupFunction(NAME, fromTypes(arguments))); } } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMapUnionAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMapUnionAggregation.java index 8cb6428b0619..915a77b46978 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMapUnionAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMapUnionAggregation.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.MapType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.spi.function.FunctionHandle; import com.google.common.collect.ImmutableList; @@ -39,14 +39,14 @@ public class TestMapUnionAggregation { - private static final FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); @Test public void testSimpleWithDuplicates() { MapType mapType = mapType(DOUBLE, VARCHAR); - FunctionHandle functionHandle = functionManager.lookupFunction(NAME, fromTypes(mapType)); - InternalAggregationFunction aggFunc = functionManager.getAggregateFunctionImplementation(functionHandle); + FunctionHandle functionHandle = FUNCTION_AND_TYPE_MANAGER.lookupFunction(NAME, fromTypes(mapType)); + InternalAggregationFunction aggFunc = FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(functionHandle); assertAggregation( aggFunc, ImmutableMap.of(23.0, "aaa", 33.0, "bbb", 43.0, "ccc", 53.0, "ddd", 13.0, "eee"), @@ -56,8 +56,8 @@ public void testSimpleWithDuplicates() mapBlockOf(DOUBLE, VARCHAR, ImmutableMap.of(43.0, "ccc", 53.0, "ddd", 13.0, "eee")))); mapType = mapType(DOUBLE, BIGINT); - functionHandle = functionManager.lookupFunction(NAME, fromTypes(mapType)); - aggFunc = functionManager.getAggregateFunctionImplementation(functionHandle); + functionHandle = FUNCTION_AND_TYPE_MANAGER.lookupFunction(NAME, fromTypes(mapType)); + aggFunc = FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(functionHandle); assertAggregation( aggFunc, ImmutableMap.of(1.0, 99L, 2.0, 99L, 3.0, 99L, 4.0, 44L), @@ -67,8 +67,8 @@ public void testSimpleWithDuplicates() mapBlockOf(DOUBLE, BIGINT, ImmutableMap.of(1.0, 44L, 2.0, 44L, 4.0, 44L)))); mapType = mapType(BOOLEAN, BIGINT); - functionHandle = functionManager.lookupFunction(NAME, fromTypes(mapType)); - aggFunc = functionManager.getAggregateFunctionImplementation(functionHandle); + functionHandle = FUNCTION_AND_TYPE_MANAGER.lookupFunction(NAME, fromTypes(mapType)); + aggFunc = FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(functionHandle); assertAggregation( aggFunc, ImmutableMap.of(false, 12L, true, 13L), @@ -82,8 +82,8 @@ public void testSimpleWithDuplicates() public void testSimpleWithNulls() { MapType mapType = mapType(DOUBLE, VARCHAR); - FunctionHandle functionHandle = functionManager.lookupFunction(NAME, fromTypes(mapType)); - InternalAggregationFunction aggFunc = functionManager.getAggregateFunctionImplementation(functionHandle); + FunctionHandle functionHandle = FUNCTION_AND_TYPE_MANAGER.lookupFunction(NAME, fromTypes(mapType)); + InternalAggregationFunction aggFunc = FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(functionHandle); Map expected = mapOf(23.0, "aaa", 33.0, null, 43.0, "ccc", 53.0, "ddd"); @@ -101,8 +101,8 @@ public void testSimpleWithNulls() public void testStructural() { MapType mapType = mapType(DOUBLE, new ArrayType(VARCHAR)); - FunctionHandle functionHandle = functionManager.lookupFunction(NAME, fromTypes(mapType)); - InternalAggregationFunction aggFunc = functionManager.getAggregateFunctionImplementation(functionHandle); + FunctionHandle functionHandle = FUNCTION_AND_TYPE_MANAGER.lookupFunction(NAME, fromTypes(mapType)); + InternalAggregationFunction aggFunc = FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(functionHandle); assertAggregation( aggFunc, ImmutableMap.of( @@ -134,8 +134,8 @@ public void testStructural() ImmutableList.of("w", "z"))))); mapType = mapType(DOUBLE, mapType(VARCHAR, VARCHAR)); - functionHandle = functionManager.lookupFunction(NAME, fromTypes(mapType)); - aggFunc = functionManager.getAggregateFunctionImplementation(functionHandle); + functionHandle = FUNCTION_AND_TYPE_MANAGER.lookupFunction(NAME, fromTypes(mapType)); + aggFunc = FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(functionHandle); assertAggregation( aggFunc, ImmutableMap.of( @@ -160,8 +160,8 @@ public void testStructural() ImmutableMap.of("e", "f"))))); mapType = mapType(new ArrayType(VARCHAR), DOUBLE); - functionHandle = functionManager.lookupFunction(NAME, fromTypes(mapType)); - aggFunc = functionManager.getAggregateFunctionImplementation(functionHandle); + functionHandle = FUNCTION_AND_TYPE_MANAGER.lookupFunction(NAME, fromTypes(mapType)); + aggFunc = FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(functionHandle); assertAggregation( aggFunc, ImmutableMap.of( diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMultimapAggAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMultimapAggAggregation.java index abcd060d49bd..1c3143bcf603 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMultimapAggAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMultimapAggAggregation.java @@ -20,7 +20,7 @@ import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.RowType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.aggregation.groupByAggregations.AggregationTestInput; import com.facebook.presto.operator.aggregation.groupByAggregations.AggregationTestInputBuilder; import com.facebook.presto.operator.aggregation.groupByAggregations.AggregationTestOutput; @@ -51,7 +51,7 @@ public class TestMultimapAggAggregation { - private static final FunctionManager functionManager = createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = createTestMetadataManager().getFunctionAndTypeManager(); @Test public void testSingleValueMap() @@ -181,7 +181,7 @@ private static void testMultimapAgg(Type keyType, List expectedKeys, T private static InternalAggregationFunction getInternalAggregationFunction(Type keyType, Type valueType) { - return functionManager.getAggregateFunctionImplementation(functionManager.lookupFunction(NAME, fromTypes(keyType, valueType))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(FUNCTION_AND_TYPE_MANAGER.lookupFunction(NAME, fromTypes(keyType, valueType))); } private static void testMultimapAgg(InternalAggregationFunction aggFunc, Type keyType, List expectedKeys, Type valueType, List expectedValues) diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestPrecisionRecallAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestPrecisionRecallAggregation.java index 614e3d846c35..9c636bbe0114 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestPrecisionRecallAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestPrecisionRecallAggregation.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.block.Block; import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.StandardTypes; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.spi.PrestoException; import com.google.common.collect.ImmutableList; @@ -51,9 +51,9 @@ public abstract class TestPrecisionRecallAggregation @BeforeClass public void setUp() { - FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); - precisionRecallFunction = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction( + FunctionAndTypeManager functionAndTypeManager = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); + precisionRecallFunction = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction( this.functionName, fromTypes(BIGINT, BOOLEAN, DOUBLE, DOUBLE))); } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestQuantileDigestAggregationFunction.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestQuantileDigestAggregationFunction.java index fd4363e64728..57b11bc167ef 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestQuantileDigestAggregationFunction.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestQuantileDigestAggregationFunction.java @@ -19,7 +19,7 @@ import com.facebook.presto.common.type.SqlVarbinary; import com.facebook.presto.common.type.StandardTypes; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.google.common.primitives.Doubles; import com.google.common.primitives.Floats; import org.testng.annotations.Test; @@ -127,9 +127,9 @@ public void testBigintsWithWeight() @Override protected InternalAggregationFunction getAggregationFunction(Type... type) { - FunctionManager functionManager = METADATA.getFunctionManager(); - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("qdigest_agg", fromTypes(type))); + FunctionAndTypeManager functionAndTypeManager = METADATA.getFunctionAndTypeManager(); + return functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("qdigest_agg", fromTypes(type))); } private void testAggregationBigint(Block inputBlock, Block weightsBlock, double maxError, long... inputs) diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestRealHistogramAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestRealHistogramAggregation.java index 8244184fc93a..2a80fb06508d 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestRealHistogramAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestRealHistogramAggregation.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.PageBuilder; import com.facebook.presto.common.block.Block; import com.facebook.presto.common.type.MapType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.type.TypeRegistry; @@ -47,9 +47,9 @@ public class TestRealHistogramAggregation public TestRealHistogramAggregation() { TypeRegistry typeRegistry = new TypeRegistry(); - FunctionManager functionManager = new FunctionManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); - InternalAggregationFunction function = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("numeric_histogram", fromTypes(BIGINT, REAL, DOUBLE))); + FunctionAndTypeManager functionAndTypeManager = new FunctionAndTypeManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); + InternalAggregationFunction function = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("numeric_histogram", fromTypes(BIGINT, REAL, DOUBLE))); factory = function.bind(ImmutableList.of(0, 1, 2), Optional.empty()); input = makeInput(10); } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestTDigestAggregationFunction.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestTDigestAggregationFunction.java index 2883c426593a..ba731c1f3b36 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestTDigestAggregationFunction.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestTDigestAggregationFunction.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.type.SqlVarbinary; import com.facebook.presto.common.type.StandardTypes; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.tdigest.TDigest; import java.util.ArrayList; @@ -51,9 +51,9 @@ protected double getParameter() @Override protected InternalAggregationFunction getAggregationFunction(Type... type) { - FunctionManager functionManager = METADATA.getFunctionManager(); - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("tdigest_agg", fromTypes(type))); + FunctionAndTypeManager functionAndTypeManager = METADATA.getFunctionAndTypeManager(); + return functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("tdigest_agg", fromTypes(type))); } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/differentialentropy/TestIllegalMethodAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/differentialentropy/TestIllegalMethodAggregation.java index 2e45d1b9cc51..427fc83552fd 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/differentialentropy/TestIllegalMethodAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/differentialentropy/TestIllegalMethodAggregation.java @@ -13,7 +13,7 @@ */ package com.facebook.presto.operator.aggregation.differentialentropy; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; import com.facebook.presto.spi.PrestoException; @@ -35,9 +35,9 @@ public class TestIllegalMethodAggregation expectedExceptionsMessageRegExp = "In differential_entropy UDF, invalid method: no_such_method") public void testIllegalMethod() { - FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); - InternalAggregationFunction function = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction( + FunctionAndTypeManager functionAndTypeManager = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); + InternalAggregationFunction function = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction( "differential_entropy", fromTypes(BIGINT, DOUBLE, DOUBLE, VARCHAR, DOUBLE, DOUBLE))); aggregation( @@ -53,9 +53,9 @@ public void testIllegalMethod() @Test public void testNullMethod() { - FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); - InternalAggregationFunction function = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction( + FunctionAndTypeManager functionAndTypeManager = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); + InternalAggregationFunction function = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction( "differential_entropy", fromTypes(BIGINT, DOUBLE, DOUBLE, VARCHAR, DOUBLE, DOUBLE))); createStringsBlock((String) null); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/minmaxby/TestMinMaxByAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/minmaxby/TestMinMaxByAggregation.java index 6efbcfdc8f29..cb6ff0d635c7 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/minmaxby/TestMinMaxByAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/minmaxby/TestMinMaxByAggregation.java @@ -21,7 +21,7 @@ import com.facebook.presto.common.type.RowType; import com.facebook.presto.common.type.SqlDecimal; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; import com.facebook.presto.operator.aggregation.state.StateCompiler; @@ -60,7 +60,7 @@ public class TestMinMaxByAggregation { private static final MetadataManager metadata = MetadataManager.createTestMetadataManager(); - private static final FunctionManager functionManager = metadata.getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = metadata.getFunctionAndTypeManager(); @Test public void testAllRegistered() @@ -610,11 +610,11 @@ public void testLongAndBlockPositionValueStateSerialization() private InternalAggregationFunction getMinByAggregation(Type... arguments) { - return functionManager.getAggregateFunctionImplementation(functionManager.lookupFunction("min_by", fromTypes(arguments))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(FUNCTION_AND_TYPE_MANAGER.lookupFunction("min_by", fromTypes(arguments))); } private InternalAggregationFunction getMaxByAggregation(Type... arguments) { - return functionManager.getAggregateFunctionImplementation(functionManager.lookupFunction("max_by", fromTypes(arguments))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(FUNCTION_AND_TYPE_MANAGER.lookupFunction("max_by", fromTypes(arguments))); } } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/minmaxby/TestMinMaxByNAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/minmaxby/TestMinMaxByNAggregation.java index e02adbc90c0e..7486036f8358 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/minmaxby/TestMinMaxByNAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/minmaxby/TestMinMaxByNAggregation.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.Page; import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; import com.facebook.presto.spi.PrestoException; @@ -40,7 +40,7 @@ public class TestMinMaxByNAggregation { - private static final FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); @Test public void testMaxDoubleDouble() @@ -295,13 +295,13 @@ public void testOutOfBound() private InternalAggregationFunction getMaxByAggregation(Type... arguments) { - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("max_by", fromTypes(arguments))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("max_by", fromTypes(arguments))); } private InternalAggregationFunction getMinByAggregation(Type... arguments) { - return functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("min_by", fromTypes(arguments))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation( + FUNCTION_AND_TYPE_MANAGER.lookupFunction("min_by", fromTypes(arguments))); } } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/index/TestFieldSetFilteringRecordSet.java b/presto-main/src/test/java/com/facebook/presto/operator/index/TestFieldSetFilteringRecordSet.java index cdd7618090cb..edec0031ecb7 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/index/TestFieldSetFilteringRecordSet.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/index/TestFieldSetFilteringRecordSet.java @@ -16,7 +16,7 @@ import com.facebook.presto.block.BlockEncodingManager; import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.TypeManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.InMemoryRecordSet; import com.facebook.presto.spi.RecordCursor; import com.facebook.presto.sql.analyzer.FeaturesConfig; @@ -38,11 +38,11 @@ public class TestFieldSetFilteringRecordSet public void test() { TypeManager typeManager = new TypeRegistry(); - FunctionManager functionManager = new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + FunctionAndTypeManager functionAndTypeManager = new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); ArrayType arrayOfBigintType = new ArrayType(BIGINT); FieldSetFilteringRecordSet fieldSetFilteringRecordSet = new FieldSetFilteringRecordSet( - functionManager, + functionAndTypeManager, new InMemoryRecordSet( ImmutableList.of(BIGINT, BIGINT, TIMESTAMP_WITH_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE, arrayOfBigintType, arrayOfBigintType), ImmutableList.of( diff --git a/presto-main/src/test/java/com/facebook/presto/operator/project/TestPageProcessor.java b/presto-main/src/test/java/com/facebook/presto/operator/project/TestPageProcessor.java index d939c39254b4..1d5d58d34409 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/project/TestPageProcessor.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/project/TestPageProcessor.java @@ -389,7 +389,7 @@ public void testExpressionProfiler() MetadataManager metadata = createTestMetadataManager(); CallExpression add10Expression = call( ADD.name(), - metadata.getFunctionManager().resolveOperator(ADD, fromTypes(BIGINT, BIGINT)), + metadata.getFunctionAndTypeManager().resolveOperator(ADD, fromTypes(BIGINT, BIGINT)), BIGINT, field(0, BIGINT), constant(10L, BIGINT)); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/AbstractTestFunctions.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/AbstractTestFunctions.java index 039a3e5e3d46..e542003890f2 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/AbstractTestFunctions.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/AbstractTestFunctions.java @@ -198,7 +198,7 @@ protected void tryEvaluateWithAll(String projection, Type expectedType) protected void registerScalarFunction(SqlScalarFunction sqlScalarFunction) { Metadata metadata = functionAssertions.getMetadata(); - metadata.getFunctionManager().registerBuiltInFunctions(ImmutableList.of(sqlScalarFunction)); + metadata.getFunctionAndTypeManager().registerBuiltInFunctions(ImmutableList.of(sqlScalarFunction)); } protected void registerScalar(Class clazz) @@ -207,7 +207,7 @@ protected void registerScalar(Class clazz) List functions = new FunctionListBuilder() .scalars(clazz) .getFunctions(); - metadata.getFunctionManager().registerBuiltInFunctions(functions); + metadata.getFunctionAndTypeManager().registerBuiltInFunctions(functions); } protected void registerParametricScalar(Class clazz) @@ -216,7 +216,7 @@ protected void registerParametricScalar(Class clazz) List functions = new FunctionListBuilder() .scalar(clazz) .getFunctions(); - metadata.getFunctionManager().registerBuiltInFunctions(functions); + metadata.getFunctionAndTypeManager().registerBuiltInFunctions(functions); } protected void registerFunctions(Plugin plugin) diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayDistinct.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayDistinct.java index 74a41d5756a8..695bca1128f0 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayDistinct.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayDistinct.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.DriverYieldSignal; import com.facebook.presto.operator.aggregation.TypedSet; @@ -106,7 +106,7 @@ public static class BenchmarkData public void setup() { MetadataManager metadata = MetadataManager.createTestMetadataManager(); - FunctionManager functionManager = metadata.getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); metadata.registerBuiltInFunctions(extractFunctions(BenchmarkArrayDistinct.class)); ExpressionCompiler compiler = new ExpressionCompiler(metadata, new PageFunctionCompiler(metadata, 0)); ImmutableList.Builder projectionsBuilder = ImmutableList.builder(); @@ -114,7 +114,7 @@ public void setup() for (int i = 0; i < TYPES.size(); i++) { Type elementType = TYPES.get(i); ArrayType arrayType = new ArrayType(elementType); - FunctionHandle functionHandle = functionManager.lookupFunction(name, fromTypes(arrayType)); + FunctionHandle functionHandle = functionAndTypeManager.lookupFunction(name, fromTypes(arrayType)); projectionsBuilder.add(new CallExpression(name, functionHandle, arrayType, ImmutableList.of(field(i, arrayType)))); blocks[i] = createChannel(POSITIONS, ARRAY_SIZE, arrayType); } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayFilter.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayFilter.java index 35c6e8fb8dab..b0886f896da8 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayFilter.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayFilter.java @@ -21,8 +21,8 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.FunctionListBuilder; -import com.facebook.presto.metadata.FunctionManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.operator.DriverYieldSignal; @@ -127,7 +127,7 @@ public static class BenchmarkData public void setup() { MetadataManager metadata = MetadataManager.createTestMetadataManager(); - FunctionManager functionManager = metadata.getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); metadata.registerBuiltInFunctions(new FunctionListBuilder().function(EXACT_ARRAY_FILTER_FUNCTION).getFunctions()); ExpressionCompiler compiler = new ExpressionCompiler(metadata, new PageFunctionCompiler(metadata, 0)); ImmutableList.Builder projectionsBuilder = ImmutableList.builder(); @@ -135,8 +135,8 @@ public void setup() for (int i = 0; i < TYPES.size(); i++) { Type elementType = TYPES.get(i); ArrayType arrayType = new ArrayType(elementType); - FunctionHandle functionHandle = functionManager.lookupFunction(name, fromTypeSignatures(arrayType.getTypeSignature(), parseTypeSignature("function(bigint,boolean)"))); - FunctionHandle greaterThan = functionManager.resolveOperator(GREATER_THAN, fromTypes(BIGINT, BIGINT)); + FunctionHandle functionHandle = functionAndTypeManager.lookupFunction(name, fromTypeSignatures(arrayType.getTypeSignature(), parseTypeSignature("function(bigint,boolean)"))); + FunctionHandle greaterThan = functionAndTypeManager.resolveOperator(GREATER_THAN, fromTypes(BIGINT, BIGINT)); projectionsBuilder.add(new CallExpression(name, functionHandle, arrayType, ImmutableList.of( field(0, arrayType), new LambdaDefinitionExpression( @@ -233,7 +233,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type type = boundVariables.getTypeVariable("T"); return new BuiltInScalarFunctionImplementation( diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayHashCodeOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayHashCodeOperator.java index 3b551d714a9f..9bdd06a29b7e 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayHashCodeOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayHashCodeOperator.java @@ -19,8 +19,8 @@ import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.StandardTypes; import com.facebook.presto.common.type.Type; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.FunctionListBuilder; -import com.facebook.presto.metadata.FunctionManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.DriverYieldSignal; import com.facebook.presto.operator.project.PageProcessor; @@ -116,7 +116,7 @@ public static class BenchmarkData public void setup() { MetadataManager metadata = MetadataManager.createTestMetadataManager(); - FunctionManager functionManager = metadata.getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); metadata.registerBuiltInFunctions(new FunctionListBuilder().scalar(BenchmarkOldArrayHash.class).getFunctions()); metadata.registerBuiltInFunctions(new FunctionListBuilder().scalar(BenchmarkAnotherArrayHash.class).getFunctions()); ExpressionCompiler compiler = new ExpressionCompiler(metadata, new PageFunctionCompiler(metadata, 0)); @@ -140,7 +140,7 @@ public void setup() throw new UnsupportedOperationException(); } ArrayType arrayType = new ArrayType(elementType); - FunctionHandle functionHandle = functionManager.lookupFunction(name, fromTypes(arrayType)); + FunctionHandle functionHandle = functionAndTypeManager.lookupFunction(name, fromTypes(arrayType)); projectionsBuilder.add(new CallExpression(name, functionHandle, BIGINT, ImmutableList.of(field(0, arrayType)))); blocks[0] = createChannel(POSITIONS, ARRAY_SIZE, arrayType); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayIntersect.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayIntersect.java index 7744e1fe33f3..95c5d45db828 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayIntersect.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayIntersect.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.DriverYieldSignal; import com.facebook.presto.operator.project.PageProcessor; @@ -122,8 +122,8 @@ public void setup() ArrayType arrayType = new ArrayType(elementType); MetadataManager metadata = createTestMetadataManager(); - FunctionManager functionManager = metadata.getFunctionManager(); - FunctionHandle functionHandle = functionManager.lookupFunction(name, fromTypes(arrayType, arrayType)); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); + FunctionHandle functionHandle = functionAndTypeManager.lookupFunction(name, fromTypes(arrayType, arrayType)); ImmutableList projections = ImmutableList.of( new CallExpression(name, functionHandle, arrayType, ImmutableList.of(field(0, arrayType), field(1, arrayType)))); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayJoin.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayJoin.java index aac8650a7d8f..8a2334f69877 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayJoin.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayJoin.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.block.Block; import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.ArrayType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.DriverYieldSignal; import com.facebook.presto.operator.project.PageProcessor; @@ -91,8 +91,8 @@ public static class BenchmarkData public void setup() { MetadataManager metadata = createTestMetadataManager(); - FunctionManager functionManager = metadata.getFunctionManager(); - FunctionHandle functionHandle = functionManager.lookupFunction("array_join", fromTypes(new ArrayType(BIGINT), VARCHAR)); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); + FunctionHandle functionHandle = functionAndTypeManager.lookupFunction("array_join", fromTypes(new ArrayType(BIGINT), VARCHAR)); List projections = ImmutableList.of( new CallExpression("array_join", functionHandle, VARCHAR, ImmutableList.of( diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArraySort.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArraySort.java index fc746549b9b4..40889781c860 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArraySort.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArraySort.java @@ -112,7 +112,7 @@ public void setup() for (int i = 0; i < TYPES.size(); i++) { Type elementType = TYPES.get(i); ArrayType arrayType = new ArrayType(elementType); - FunctionHandle functionHandle = metadata.getFunctionManager().lookupFunction(name, fromTypes(arrayType)); + FunctionHandle functionHandle = metadata.getFunctionAndTypeManager().lookupFunction(name, fromTypes(arrayType)); projectionsBuilder.add(new CallExpression(name, functionHandle, arrayType, ImmutableList.of(field(i, arrayType)))); blocks[i] = createChannel(POSITIONS, ARRAY_SIZE, arrayType); } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArraySubscript.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArraySubscript.java index 92e212f4babe..fa7c77407cd3 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArraySubscript.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArraySubscript.java @@ -135,7 +135,7 @@ public void setup() ImmutableList.Builder projectionsBuilder = ImmutableList.builder(); - FunctionHandle functionHandle = metadata.getFunctionManager().resolveOperator(SUBSCRIPT, fromTypes(arrayType, BIGINT)); + FunctionHandle functionHandle = metadata.getFunctionAndTypeManager().resolveOperator(SUBSCRIPT, fromTypes(arrayType, BIGINT)); for (int i = 0; i < arraySize; i++) { projectionsBuilder.add(new CallExpression( SUBSCRIPT.name(), diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayTransform.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayTransform.java index 508e5629541b..2d346a9c5278 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayTransform.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayTransform.java @@ -19,7 +19,7 @@ import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.DriverYieldSignal; import com.facebook.presto.operator.project.PageProcessor; @@ -107,7 +107,7 @@ public static class BenchmarkData public void setup() { MetadataManager metadata = MetadataManager.createTestMetadataManager(); - FunctionManager functionManager = metadata.getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); ExpressionCompiler compiler = new ExpressionCompiler(metadata, new PageFunctionCompiler(metadata, 0)); ImmutableList.Builder projectionsBuilder = ImmutableList.builder(); Block[] blocks = new Block[TYPES.size()]; @@ -115,8 +115,8 @@ public void setup() for (int i = 0; i < TYPES.size(); i++) { Type elementType = TYPES.get(i); ArrayType arrayType = new ArrayType(elementType); - FunctionHandle functionHandle = functionManager.lookupFunction("transform", fromTypeSignatures(arrayType.getTypeSignature(), parseTypeSignature("function(bigint,boolean)"))); - FunctionHandle greaterThan = functionManager.resolveOperator(GREATER_THAN, fromTypes(BIGINT, BIGINT)); + FunctionHandle functionHandle = functionAndTypeManager.lookupFunction("transform", fromTypeSignatures(arrayType.getTypeSignature(), parseTypeSignature("function(bigint,boolean)"))); + FunctionHandle greaterThan = functionAndTypeManager.resolveOperator(GREATER_THAN, fromTypes(BIGINT, BIGINT)); projectionsBuilder.add(new CallExpression("transform", functionHandle, returnType, ImmutableList.of( new InputReferenceExpression(0, arrayType), new LambdaDefinitionExpression( diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkEqualsOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkEqualsOperator.java index 95d47b233b8d..1a3c445e7c8f 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkEqualsOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkEqualsOperator.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.PageBuilder; import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.DriverYieldSignal; import com.facebook.presto.operator.project.PageProcessor; @@ -84,26 +84,26 @@ public class BenchmarkEqualsOperator public void setup() { MetadataManager metadata = MetadataManager.createTestMetadataManager(); - FunctionManager functionManager = metadata.getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); ExpressionCompiler expressionCompiler = new ExpressionCompiler( metadata, new PageFunctionCompiler(metadata, 0)); - RowExpression projection = generateComplexComparisonProjection(functionManager, FIELDS_COUNT, COMPARISONS_COUNT); + RowExpression projection = generateComplexComparisonProjection(functionAndTypeManager, FIELDS_COUNT, COMPARISONS_COUNT); compiledProcessor = expressionCompiler.compilePageProcessor(SESSION.getSqlFunctionProperties(), Optional.empty(), ImmutableList.of(projection)).get(); } - private static RowExpression generateComplexComparisonProjection(FunctionManager functionManager, int fieldsCount, int comparisonsCount) + private static RowExpression generateComplexComparisonProjection(FunctionAndTypeManager functionAndTypeManager, int fieldsCount, int comparisonsCount) { checkArgument(fieldsCount > 0, "fieldsCount must be greater than zero"); checkArgument(comparisonsCount > 0, "comparisonsCount must be greater than zero"); if (comparisonsCount == 1) { - return createComparison(functionManager, 0, 0); + return createComparison(functionAndTypeManager, 0, 0); } return createConjunction( - createComparison(functionManager, 0, comparisonsCount % fieldsCount), - generateComplexComparisonProjection(functionManager, fieldsCount, comparisonsCount - 1)); + createComparison(functionAndTypeManager, 0, comparisonsCount % fieldsCount), + generateComplexComparisonProjection(functionAndTypeManager, fieldsCount, comparisonsCount - 1)); } private static RowExpression createConjunction(RowExpression left, RowExpression right) @@ -111,11 +111,11 @@ private static RowExpression createConjunction(RowExpression left, RowExpression return specialForm(OR, BOOLEAN, left, right); } - private static RowExpression createComparison(FunctionManager functionManager, int leftField, int rightField) + private static RowExpression createComparison(FunctionAndTypeManager functionAndTypeManager, int leftField, int rightField) { return call( EQUAL.name(), - functionManager.resolveOperator(EQUAL, fromTypes(BIGINT, BIGINT)), + functionAndTypeManager.resolveOperator(EQUAL, fromTypes(BIGINT, BIGINT)), BOOLEAN, field(leftField, BIGINT), field(rightField, BIGINT)); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkJsonToArrayCast.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkJsonToArrayCast.java index 83bcfd3313f5..df8063fece08 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkJsonToArrayCast.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkJsonToArrayCast.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.DriverYieldSignal; import com.facebook.presto.operator.project.PageProcessor; @@ -113,8 +113,8 @@ public void setup() } MetadataManager metadata = createTestMetadataManager(); - FunctionManager functionManager = metadata.getFunctionManager(); - FunctionHandle functionHandle = functionManager.lookupCast(CAST, JSON.getTypeSignature(), (new ArrayType(elementType)).getTypeSignature()); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); + FunctionHandle functionHandle = functionAndTypeManager.lookupCast(CAST, JSON.getTypeSignature(), (new ArrayType(elementType)).getTypeSignature()); List projections = ImmutableList.of( new CallExpression(CAST.name(), functionHandle, new ArrayType(elementType), ImmutableList.of(field(0, JSON)))); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkJsonToMapCast.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkJsonToMapCast.java index 392f1f82e428..9c16ff02abe7 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkJsonToMapCast.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkJsonToMapCast.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.block.Block; import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.DriverYieldSignal; import com.facebook.presto.operator.project.PageProcessor; @@ -98,8 +98,8 @@ public static class BenchmarkData public void setup() { MetadataManager metadata = createTestMetadataManager(); - FunctionManager functionManager = metadata.getFunctionManager(); - FunctionHandle functionHandle = functionManager.lookupCast(CAST, JSON.getTypeSignature(), mapType(VARCHAR, BIGINT).getTypeSignature()); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); + FunctionHandle functionHandle = functionAndTypeManager.lookupCast(CAST, JSON.getTypeSignature(), mapType(VARCHAR, BIGINT).getTypeSignature()); Type valueType; switch (valueTypeName) { diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapConcat.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapConcat.java index e72dddbef649..7888b45fcc5b 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapConcat.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapConcat.java @@ -138,7 +138,7 @@ public void setup() ImmutableList.Builder projectionsBuilder = ImmutableList.builder(); - FunctionHandle functionHandle = metadata.getFunctionManager().lookupFunction(name, fromTypes(mapType, mapType)); + FunctionHandle functionHandle = metadata.getFunctionAndTypeManager().lookupFunction(name, fromTypes(mapType, mapType)); projectionsBuilder.add(new CallExpression( name, functionHandle, diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapSubscript.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapSubscript.java index ff3ab2d18aa9..a06a530b8710 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapSubscript.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapSubscript.java @@ -147,7 +147,7 @@ public void setup() ImmutableList.Builder projectionsBuilder = ImmutableList.builder(); - FunctionHandle functionHandle = metadata.getFunctionManager().resolveOperator(SUBSCRIPT, fromTypes(mapType, mapType.getKeyType())); + FunctionHandle functionHandle = metadata.getFunctionAndTypeManager().resolveOperator(SUBSCRIPT, fromTypes(mapType, mapType.getKeyType())); for (int i = 0; i < mapSize; i++) { projectionsBuilder.add(new CallExpression( SUBSCRIPT.name(), diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapToMapCast.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapToMapCast.java index 9170b20f6866..f78b773a900d 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapToMapCast.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapToMapCast.java @@ -93,7 +93,7 @@ public static class BenchmarkData public void setup() { MetadataManager metadata = createTestMetadataManager(); - FunctionHandle functionHandle = metadata.getFunctionManager().lookupCast(CAST, mapType(DOUBLE, BIGINT).getTypeSignature(), mapType(BIGINT, DOUBLE).getTypeSignature()); + FunctionHandle functionHandle = metadata.getFunctionAndTypeManager().lookupCast(CAST, mapType(DOUBLE, BIGINT).getTypeSignature(), mapType(BIGINT, DOUBLE).getTypeSignature()); List projections = ImmutableList.of( new CallExpression(CAST.name(), functionHandle, mapType(BIGINT, DOUBLE), ImmutableList.of(field(0, mapType(DOUBLE, BIGINT))))); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkRowToRowCast.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkRowToRowCast.java index e944377b2239..ea975cb0198b 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkRowToRowCast.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkRowToRowCast.java @@ -93,7 +93,7 @@ public static class BenchmarkData public void setup() { MetadataManager metadata = createTestMetadataManager(); - FunctionHandle functionHandle = metadata.getFunctionManager().lookupCast(CAST, RowType.anonymous(fromFieldTypes).getTypeSignature(), RowType.anonymous(toFieldTypes).getTypeSignature()); + FunctionHandle functionHandle = metadata.getFunctionAndTypeManager().lookupCast(CAST, RowType.anonymous(fromFieldTypes).getTypeSignature(), RowType.anonymous(toFieldTypes).getTypeSignature()); List projections = ImmutableList.of( new CallExpression(CAST.name(), functionHandle, RowType.anonymous(fromFieldTypes), ImmutableList.of(field(0, RowType.anonymous(toFieldTypes))))); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkTransformKey.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkTransformKey.java index a69dbce53a9d..c0fe1176b79c 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkTransformKey.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkTransformKey.java @@ -19,7 +19,7 @@ import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.MapType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.DriverYieldSignal; import com.facebook.presto.operator.project.PageProcessor; @@ -107,7 +107,7 @@ public static class BenchmarkData public void setup() { MetadataManager metadata = MetadataManager.createTestMetadataManager(); - FunctionManager functionManager = metadata.getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); ExpressionCompiler compiler = new ExpressionCompiler(metadata, new PageFunctionCompiler(metadata, 0)); ImmutableList.Builder projectionsBuilder = ImmutableList.builder(); Type elementType; @@ -125,12 +125,12 @@ public void setup() throw new UnsupportedOperationException(); } MapType mapType = mapType(elementType, elementType); - FunctionHandle functionHandle = functionManager.lookupFunction( + FunctionHandle functionHandle = functionAndTypeManager.lookupFunction( name, fromTypeSignatures( mapType.getTypeSignature(), parseTypeSignature(format("function(%s, %s, %s)", type, type, type)))); - FunctionHandle add = functionManager.resolveOperator(ADD, fromTypes(elementType, elementType)); + FunctionHandle add = functionAndTypeManager.resolveOperator(ADD, fromTypes(elementType, elementType)); projectionsBuilder.add(call(name, functionHandle, mapType, ImmutableList.of( field(0, mapType), new LambdaDefinitionExpression( diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkTransformValue.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkTransformValue.java index ef788bb02950..a88314bd3023 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkTransformValue.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkTransformValue.java @@ -19,7 +19,7 @@ import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.MapType; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.DriverYieldSignal; import com.facebook.presto.operator.project.PageProcessor; @@ -110,7 +110,7 @@ public static class BenchmarkData public void setup() { MetadataManager metadata = MetadataManager.createTestMetadataManager(); - FunctionManager functionManager = metadata.getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); ExpressionCompiler compiler = new ExpressionCompiler(metadata, new PageFunctionCompiler(metadata, 0)); ImmutableList.Builder projectionsBuilder = ImmutableList.builder(); Type elementType; @@ -133,12 +133,12 @@ public void setup() } MapType mapType = mapType(elementType, elementType); MapType returnType = mapType(elementType, BOOLEAN); - FunctionHandle functionHandle = functionManager.lookupFunction( + FunctionHandle functionHandle = functionAndTypeManager.lookupFunction( name, fromTypeSignatures( mapType.getTypeSignature(), parseTypeSignature(format("function(%s, %s, boolean)", type, type)))); - FunctionHandle greaterThan = metadata.getFunctionManager().resolveOperator( + FunctionHandle greaterThan = metadata.getFunctionAndTypeManager().resolveOperator( GREATER_THAN, fromTypes(elementType, elementType)); projectionsBuilder.add(call(name, functionHandle, returnType, ImmutableList.of( diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/FunctionAssertions.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/FunctionAssertions.java index 16466608bdfd..de306b97a081 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/FunctionAssertions.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/FunctionAssertions.java @@ -988,7 +988,7 @@ private static SourceOperatorFactory compileScanFilterProject(SqlFunctionPropert private RowExpression toRowExpression(Expression projection, Map, Type> expressionTypes, Map layout) { - return translate(projection, expressionTypes, layout, metadata.getFunctionManager(), metadata.getTypeManager(), session); + return translate(projection, expressionTypes, layout, metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), session); } private static Page getAtMostOnePage(Operator operator, Page sourcePage) diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestPageProcessorCompiler.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestPageProcessorCompiler.java index eb99c97c858a..b44a5d764ff3 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestPageProcessorCompiler.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestPageProcessorCompiler.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.block.DictionaryBlock; import com.facebook.presto.common.block.RunLengthEncodedBlock; import com.facebook.presto.common.type.ArrayType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.DriverYieldSignal; import com.facebook.presto.operator.project.PageProcessor; @@ -80,10 +80,10 @@ public void tearDown() @Test public void testNoCaching() { - FunctionManager functionManager = createTestMetadataManager().getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = createTestMetadataManager().getFunctionAndTypeManager(); ImmutableList.Builder projectionsBuilder = ImmutableList.builder(); ArrayType arrayType = new ArrayType(VARCHAR); - FunctionHandle functionHandle = functionManager.lookupFunction("concat", fromTypes(arrayType, arrayType)); + FunctionHandle functionHandle = functionAndTypeManager.lookupFunction("concat", fromTypes(arrayType, arrayType)); projectionsBuilder.add(new CallExpression("concat", functionHandle, arrayType, ImmutableList.of(field(0, arrayType), field(1, arrayType)))); ImmutableList projections = projectionsBuilder.build(); @@ -121,10 +121,10 @@ public void testSanityRLE() @Test public void testSanityFilterOnDictionary() { - FunctionManager functionManager = createTestMetadataManager().getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = createTestMetadataManager().getFunctionAndTypeManager(); CallExpression lengthVarchar = new CallExpression( - "length", functionManager.lookupFunction("length", fromTypes(VARCHAR)), BIGINT, ImmutableList.of(field(0, VARCHAR))); - FunctionHandle lessThan = functionManager.resolveOperator(LESS_THAN, fromTypes(BIGINT, BIGINT)); + "length", functionAndTypeManager.lookupFunction("length", fromTypes(VARCHAR)), BIGINT, ImmutableList.of(field(0, VARCHAR))); + FunctionHandle lessThan = functionAndTypeManager.resolveOperator(LESS_THAN, fromTypes(BIGINT, BIGINT)); CallExpression filter = new CallExpression(LESS_THAN.name(), lessThan, BOOLEAN, ImmutableList.of(lengthVarchar, constant(10L, BIGINT))); PageProcessor processor = compiler.compilePageProcessor(TEST_SESSION.getSqlFunctionProperties(), Optional.of(filter), ImmutableList.of(field(0, VARCHAR)), false, MAX_BATCH_SIZE).get(); @@ -162,8 +162,8 @@ public void testSanityFilterOnDictionary() @Test public void testSanityFilterOnRLE() { - FunctionManager functionManager = createTestMetadataManager().getFunctionManager(); - FunctionHandle lessThan = functionManager.resolveOperator(LESS_THAN, fromTypes(BIGINT, BIGINT)); + FunctionAndTypeManager functionAndTypeManager = createTestMetadataManager().getFunctionAndTypeManager(); + FunctionHandle lessThan = functionAndTypeManager.resolveOperator(LESS_THAN, fromTypes(BIGINT, BIGINT)); CallExpression filter = new CallExpression(LESS_THAN.name(), lessThan, BOOLEAN, ImmutableList.of(field(0, BIGINT), constant(10L, BIGINT))); PageProcessor processor = compiler.compilePageProcessor(TEST_SESSION.getSqlFunctionProperties(), Optional.of(filter), ImmutableList.of(field(0, BIGINT)), false, MAX_BATCH_SIZE).get(); @@ -208,16 +208,16 @@ public void testSanityColumnarDictionary() @Test public void testNonDeterministicProject() { - FunctionManager functionManager = createTestMetadataManager().getFunctionManager(); - FunctionHandle lessThan = functionManager.resolveOperator(LESS_THAN, fromTypes(BIGINT, BIGINT)); + FunctionAndTypeManager functionAndTypeManager = createTestMetadataManager().getFunctionAndTypeManager(); + FunctionHandle lessThan = functionAndTypeManager.resolveOperator(LESS_THAN, fromTypes(BIGINT, BIGINT)); CallExpression random = new CallExpression( - "random", functionManager.lookupFunction("random", fromTypes(BIGINT)), BIGINT, singletonList(constant(10L, BIGINT))); + "random", functionAndTypeManager.lookupFunction("random", fromTypes(BIGINT)), BIGINT, singletonList(constant(10L, BIGINT))); InputReferenceExpression col0 = field(0, BIGINT); CallExpression lessThanRandomExpression = new CallExpression(LESS_THAN.name(), lessThan, BOOLEAN, ImmutableList.of(col0, random)); PageProcessor processor = compiler.compilePageProcessor(TEST_SESSION.getSqlFunctionProperties(), Optional.empty(), ImmutableList.of(lessThanRandomExpression), false, MAX_BATCH_SIZE).get(); - assertFalse(new RowExpressionDeterminismEvaluator(metadataManager.getFunctionManager()).isDeterministic(lessThanRandomExpression)); + assertFalse(new RowExpressionDeterminismEvaluator(metadataManager.getFunctionAndTypeManager()).isDeterministic(lessThanRandomExpression)); Page page = new Page(createLongDictionaryBlock(1, 100)); Page outputPage = getOnlyElement( diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestProvidedBlockBuilderReturnPlaceConvention.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestProvidedBlockBuilderReturnPlaceConvention.java index 016c9a52faca..f0ad036b13f0 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestProvidedBlockBuilderReturnPlaceConvention.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestProvidedBlockBuilderReturnPlaceConvention.java @@ -20,7 +20,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.ReturnPlaceConvention; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.ScalarImplementationChoice; @@ -167,7 +167,7 @@ protected FunctionWithProvidedBlockReturnPlaceConvention1() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type type = boundVariables.getTypeVariable("T"); MethodHandle methodHandleStack = MethodHandles.identity(type.getJavaType()); @@ -287,7 +287,7 @@ protected FunctionWithProvidedBlockReturnPlaceConvention2() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { Type type = boundVariables.getTypeVariable("T"); MethodHandle methodHandleStack = MethodHandles.identity(wrap(type.getJavaType())); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/TestExpressionInterpreter.java b/presto-main/src/test/java/com/facebook/presto/sql/TestExpressionInterpreter.java index 44db4e90a99d..ea6756aa89f9 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/TestExpressionInterpreter.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/TestExpressionInterpreter.java @@ -145,7 +145,7 @@ public class TestExpressionInterpreter @BeforeClass public void setup() { - METADATA.getFunctionManager().registerBuiltInFunctions(ImmutableList.of(APPLY_FUNCTION)); + METADATA.getFunctionAndTypeManager().registerBuiltInFunctions(ImmutableList.of(APPLY_FUNCTION)); } @Test @@ -1719,7 +1719,7 @@ else if (left instanceof SpecialFormExpression) { private static boolean isRemovableCast(Object value) { if (value instanceof CallExpression && - new FunctionResolution(METADATA.getFunctionManager()).isCastFunction(((CallExpression) value).getFunctionHandle())) { + new FunctionResolution(METADATA.getFunctionAndTypeManager()).isCastFunction(((CallExpression) value).getFunctionHandle())) { Type targetType = ((CallExpression) value).getType(); Type sourceType = ((CallExpression) value).getArguments().get(0).getType(); return METADATA.getTypeManager().canCoerce(sourceType, targetType); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/TestRowExpressionSerde.java b/presto-main/src/test/java/com/facebook/presto/sql/TestRowExpressionSerde.java index d672fd5f69a1..1227a1503474 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/TestRowExpressionSerde.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/TestRowExpressionSerde.java @@ -230,12 +230,12 @@ private RowExpression getRoundTrip(String sql, boolean optimize) private FunctionHandle operator(OperatorType operatorType, Type... types) { - return metadata.getFunctionManager().resolveOperator(operatorType, fromTypes(types)); + return metadata.getFunctionAndTypeManager().resolveOperator(operatorType, fromTypes(types)); } private FunctionHandle function(String name, Type... types) { - return metadata.getFunctionManager().lookupFunction(name, fromTypes(types)); + return metadata.getFunctionAndTypeManager().lookupFunction(name, fromTypes(types)); } private JsonCodec getJsonCodec() @@ -267,7 +267,7 @@ private JsonCodec getJsonCodec() private RowExpression translate(Expression expression, boolean optimize) { - RowExpression rowExpression = SqlToRowExpressionTranslator.translate(expression, getExpressionTypes(expression), ImmutableMap.of(), metadata.getFunctionManager(), metadata.getTypeManager(), TEST_SESSION); + RowExpression rowExpression = SqlToRowExpressionTranslator.translate(expression, getExpressionTypes(expression), ImmutableMap.of(), metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), TEST_SESSION); if (optimize) { RowExpressionOptimizer optimizer = new RowExpressionOptimizer(metadata); return optimizer.optimize(rowExpression, OPTIMIZED, TEST_SESSION.toConnectorSession()); @@ -278,7 +278,7 @@ private RowExpression translate(Expression expression, boolean optimize) private Map, Type> getExpressionTypes(Expression expression) { ExpressionAnalyzer expressionAnalyzer = ExpressionAnalyzer.createWithoutSubqueries( - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), TEST_SESSION, TypeProvider.empty(), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/TestingRowExpressionTranslator.java b/presto-main/src/test/java/com/facebook/presto/sql/TestingRowExpressionTranslator.java index b2b732b20fab..2794043591d2 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/TestingRowExpressionTranslator.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/TestingRowExpressionTranslator.java @@ -74,14 +74,14 @@ public RowExpression translate(Expression expression, TypeProvider typeProvider) expression, getExpressionTypes(expression, typeProvider), ImmutableMap.of(), - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), TEST_SESSION); } public RowExpression translateAndOptimize(Expression expression, Map, Type> types) { - RowExpression rowExpression = SqlToRowExpressionTranslator.translate(expression, types, ImmutableMap.of(), metadata.getFunctionManager(), metadata.getTypeManager(), TEST_SESSION); + RowExpression rowExpression = SqlToRowExpressionTranslator.translate(expression, types, ImmutableMap.of(), metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), TEST_SESSION); RowExpressionOptimizer optimizer = new RowExpressionOptimizer(metadata); return optimizer.optimize(rowExpression, OPTIMIZED, TEST_SESSION.toConnectorSession()); } @@ -99,7 +99,7 @@ Expression simplifyExpression(Expression expression) private Map, Type> getExpressionTypes(Expression expression, TypeProvider typeProvider) { ExpressionAnalyzer expressionAnalyzer = ExpressionAnalyzer.createWithoutSubqueries( - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), TEST_SESSION, typeProvider, 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 b172f4cddfa0..0c5358aabfca 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 @@ -1612,7 +1612,7 @@ public void setup() new AnalyzePropertyManager(), transactionManager); - metadata.getFunctionManager().registerBuiltInFunctions(ImmutableList.of(APPLY_FUNCTION)); + metadata.getFunctionAndTypeManager().registerBuiltInFunctions(ImmutableList.of(APPLY_FUNCTION)); Catalog tpchTestCatalog = createTestingCatalog(TPCH_CATALOG, TPCH_CONNECTOR_ID); catalogManager.registerCatalog(tpchTestCatalog); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/BenchmarkPageProcessor.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/BenchmarkPageProcessor.java index e0f1f6e9733f..9a38de283a50 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/BenchmarkPageProcessor.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/BenchmarkPageProcessor.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.PageBuilder; import com.facebook.presto.common.block.Block; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.DriverYieldSignal; import com.facebook.presto.operator.project.InputPageProjection; @@ -160,7 +160,7 @@ public static class BenchmarkData private static final Slice MAX_SHIP_DATE = utf8Slice("1995-01-01"); private MetadataManager metadataManager = createTestMetadataManager(); - private FunctionManager functionManager = metadataManager.getFunctionManager(); + private FunctionAndTypeManager functionManager = metadataManager.getFunctionAndTypeManager(); private PageProcessor compiledProcessor; private Tpch1FilterAndProject handcodedProcessor; private PageProcessor identityProjectionProcessor; @@ -205,7 +205,7 @@ private static Page createInputPage() return pageBuilder.build(); } - private final RowExpression createFilterExpression(FunctionManager functionManager) + private final RowExpression createFilterExpression(FunctionAndTypeManager functionAndTypeManager) { if (filterFails.equals("never")) { return new ConstantExpression(true, BOOLEAN); @@ -265,13 +265,13 @@ else if (filterFails.equals("always")) { } } - private final RowExpression createProjectExpression(FunctionManager functionManager) + private final RowExpression createProjectExpression(FunctionAndTypeManager functionAndTypeManager) { switch (projectionDataType) { case "BIGINT": return call( MULTIPLY.name(), - functionManager.resolveOperator(MULTIPLY, fromTypes(BIGINT, BIGINT)), + functionAndTypeManager.resolveOperator(MULTIPLY, fromTypes(BIGINT, BIGINT)), BIGINT, field(EXTENDED_PRICE_IN_CENTS, BIGINT), field(DISCOUNT_PERCENT, BIGINT)); @@ -279,7 +279,7 @@ private final RowExpression createProjectExpression(FunctionManager functionMana case "DOUBLE": return call( MULTIPLY.name(), - functionManager.resolveOperator(MULTIPLY, fromTypes(DOUBLE, DOUBLE)), + functionAndTypeManager.resolveOperator(MULTIPLY, fromTypes(DOUBLE, DOUBLE)), DOUBLE, field(EXTENDED_PRICE, DOUBLE), field(DISCOUNT, DOUBLE)); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/CommonSubExpressionBenchmark.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/CommonSubExpressionBenchmark.java index c57be7a44ada..bd677507b60c 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/CommonSubExpressionBenchmark.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/CommonSubExpressionBenchmark.java @@ -184,7 +184,7 @@ private RowExpression rowExpression(String value) Expression expression = createExpression(value, METADATA, TypeProvider.copyOf(symbolTypes)); Map, Type> expressionTypes = getExpressionTypes(TEST_SESSION, METADATA, SQL_PARSER, TypeProvider.copyOf(symbolTypes), expression, emptyList(), WarningCollector.NOOP); - RowExpression rowExpression = SqlToRowExpressionTranslator.translate(expression, expressionTypes, sourceLayout, METADATA.getFunctionManager(), METADATA.getTypeManager(), TEST_SESSION); + RowExpression rowExpression = SqlToRowExpressionTranslator.translate(expression, expressionTypes, sourceLayout, METADATA.getFunctionAndTypeManager(), METADATA.getTypeManager(), TEST_SESSION); RowExpressionOptimizer optimizer = new RowExpressionOptimizer(METADATA); return optimizer.optimize(rowExpression, OPTIMIZED, TEST_SESSION.toConnectorSession()); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/PageProcessorBenchmark.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/PageProcessorBenchmark.java index 290e8a15a92e..adf99daa385e 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/PageProcessorBenchmark.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/PageProcessorBenchmark.java @@ -182,7 +182,7 @@ private RowExpression rowExpression(String value) Expression expression = createExpression(value, METADATA, TypeProvider.copyOf(symbolTypes)); Map, Type> expressionTypes = getExpressionTypes(TEST_SESSION, METADATA, SQL_PARSER, TypeProvider.copyOf(symbolTypes), expression, emptyList(), WarningCollector.NOOP); - RowExpression rowExpression = SqlToRowExpressionTranslator.translate(expression, expressionTypes, sourceLayout, METADATA.getFunctionManager(), METADATA.getTypeManager(), TEST_SESSION); + RowExpression rowExpression = SqlToRowExpressionTranslator.translate(expression, expressionTypes, sourceLayout, METADATA.getFunctionAndTypeManager(), METADATA.getTypeManager(), TEST_SESSION); RowExpressionOptimizer optimizer = new RowExpressionOptimizer(METADATA); return optimizer.optimize(rowExpression, OPTIMIZED, TEST_SESSION.toConnectorSession()); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestCommonSubExpressionRewritter.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestCommonSubExpressionRewritter.java index e4b7f3489369..a118623d4fae 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestCommonSubExpressionRewritter.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestCommonSubExpressionRewritter.java @@ -146,7 +146,7 @@ private RowExpression rowExpression(String sql) expression, expressionTypes, ImmutableMap.of(), - METADATA.getFunctionManager(), + METADATA.getFunctionAndTypeManager(), METADATA.getTypeManager(), SESSION); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestCursorProcessorCompiler.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestCursorProcessorCompiler.java index aba6cf8cfa82..6057a8a6a50c 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestCursorProcessorCompiler.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestCursorProcessorCompiler.java @@ -19,7 +19,7 @@ import com.facebook.presto.common.block.Block; import com.facebook.presto.common.block.BlockBuilder; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.operator.DriverYieldSignal; import com.facebook.presto.operator.index.PageRecordSet; @@ -67,7 +67,7 @@ public class TestCursorProcessorCompiler { private static final Metadata METADATA = createTestMetadataManager(); - private static final FunctionManager FUNCTION_MANAGER = METADATA.getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_MANAGER = METADATA.getFunctionAndTypeManager(); private static final CallExpression ADD_X_Y = call( ADD.name(), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestInCodeGenerator.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestInCodeGenerator.java index cfa14ca83e24..0e61cfd0bb7b 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestInCodeGenerator.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestInCodeGenerator.java @@ -13,7 +13,7 @@ */ package com.facebook.presto.sql.gen; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.relation.CallExpression; import com.facebook.presto.spi.relation.RowExpression; import io.airlift.slice.Slices; @@ -42,7 +42,7 @@ public class TestInCodeGenerator @Test public void testInteger() { - FunctionManager functionManager = createTestMetadataManager().getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = createTestMetadataManager().getFunctionAndTypeManager(); List values = new ArrayList<>(); values.add(constant((long) Integer.MIN_VALUE, INTEGER)); values.add(constant((long) Integer.MAX_VALUE, INTEGER)); @@ -53,7 +53,7 @@ public void testInteger() assertEquals(checkSwitchGenerationCase(INTEGER, values), DIRECT_SWITCH); values.add(new CallExpression( CAST.name(), - functionManager.lookupCast(CAST, DOUBLE.getTypeSignature(), INTEGER.getTypeSignature()), + functionAndTypeManager.lookupCast(CAST, DOUBLE.getTypeSignature(), INTEGER.getTypeSignature()), INTEGER, Collections.singletonList(constant(12345678901234.0, DOUBLE)))); assertEquals(checkSwitchGenerationCase(INTEGER, values), DIRECT_SWITCH); @@ -70,7 +70,7 @@ public void testInteger() @Test public void testBigint() { - FunctionManager functionManager = createTestMetadataManager().getFunctionManager(); + FunctionAndTypeManager functionAndTypeManager = createTestMetadataManager().getFunctionAndTypeManager(); List values = new ArrayList<>(); values.add(constant(Integer.MAX_VALUE + 1L, BIGINT)); values.add(constant(Integer.MIN_VALUE - 1L, BIGINT)); @@ -81,7 +81,7 @@ public void testBigint() assertEquals(checkSwitchGenerationCase(BIGINT, values), HASH_SWITCH); values.add(new CallExpression( CAST.name(), - functionManager.lookupCast(CAST, DOUBLE.getTypeSignature(), BIGINT.getTypeSignature()), + functionAndTypeManager.lookupCast(CAST, DOUBLE.getTypeSignature(), BIGINT.getTypeSignature()), BIGINT, Collections.singletonList(constant(12345678901234.0, DOUBLE)))); assertEquals(checkSwitchGenerationCase(BIGINT, values), HASH_SWITCH); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestJoinCompiler.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestJoinCompiler.java index 6abc856a8b56..f93da0cd34c0 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestJoinCompiler.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestJoinCompiler.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.PageBuilder; import com.facebook.presto.common.block.Block; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.PagesHashStrategy; import com.facebook.presto.operator.SimplePagesHashStrategy; @@ -49,7 +49,7 @@ public class TestJoinCompiler { private static final JoinCompiler joinCompiler = new JoinCompiler(MetadataManager.createTestMetadataManager(), new FeaturesConfig()); - private static final FunctionManager FUNCTION_MANAGER = MetadataManager.createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_MANAGER = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); private static final boolean groupByUsesEqualTo = new FeaturesConfig().isGroupByUsesEqualTo(); @DataProvider(name = "hashEnabledValues") diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestPageFunctionCompiler.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestPageFunctionCompiler.java index 138932a9c3bc..b411a0ddee2d 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestPageFunctionCompiler.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestPageFunctionCompiler.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.Page; import com.facebook.presto.common.block.Block; import com.facebook.presto.common.block.BlockBuilder; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.operator.DriverYieldSignal; import com.facebook.presto.operator.Work; import com.facebook.presto.operator.project.PageFilter; @@ -58,7 +58,7 @@ public class TestPageFunctionCompiler { - private static final FunctionManager FUNCTION_MANAGER = createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_MANAGER = createTestMetadataManager().getFunctionAndTypeManager(); private static final CallExpression ADD_10_EXPRESSION = call( ADD.name(), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestRowExpressionPredicateCompiler.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestRowExpressionPredicateCompiler.java index 5d279f04cc24..3f47d9be66b2 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestRowExpressionPredicateCompiler.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestRowExpressionPredicateCompiler.java @@ -46,7 +46,7 @@ public class TestRowExpressionPredicateCompiler { private Metadata metadata = createTestMetadataManager(); - private FunctionResolution functionResolution = new FunctionResolution(metadata.getFunctionManager()); + private FunctionResolution functionResolution = new FunctionResolution(metadata.getFunctionAndTypeManager()); @Test public void test() diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestVarArgsToArrayAdapterGenerator.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestVarArgsToArrayAdapterGenerator.java index 33fe38480dad..9df25efd94bd 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestVarArgsToArrayAdapterGenerator.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestVarArgsToArrayAdapterGenerator.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.function.QualifiedFunctionName; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.operator.scalar.AbstractTestFunctions; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation; @@ -109,7 +109,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { VarArgsToArrayAdapterGenerator.MethodHandleAndConstructor methodHandleAndConstructor = generateVarArgsToArrayAdapter( long.class, diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestCanonicalize.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestCanonicalize.java index ac2ee3f79118..919a8c0fb45f 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestCanonicalize.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestCanonicalize.java @@ -75,7 +75,7 @@ public void testDuplicatesInWindowOrderBy() .addFunction(functionCall("row_number", Optional.empty(), ImmutableList.of())), values("A"))), ImmutableList.of( - new UnaliasSymbolReferences(getMetadata().getFunctionManager()), + new UnaliasSymbolReferences(getMetadata().getFunctionAndTypeManager()), new IterativeOptimizer( new RuleStatsRecorder(), getQueryRunner().getStatsCalculator(), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEffectivePredicateExtractor.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEffectivePredicateExtractor.java index 0cee54f3ac22..2ac76587ae1d 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEffectivePredicateExtractor.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEffectivePredicateExtractor.java @@ -112,12 +112,12 @@ public class TestEffectivePredicateExtractor private final Metadata metadata = MetadataManager.createTestMetadataManager(); private final LogicalRowExpressions logicalRowExpressions = new LogicalRowExpressions( - new RowExpressionDeterminismEvaluator(metadata.getFunctionManager()), - new FunctionResolution(metadata.getFunctionManager()), - metadata.getFunctionManager()); + new RowExpressionDeterminismEvaluator(metadata.getFunctionAndTypeManager()), + new FunctionResolution(metadata.getFunctionAndTypeManager()), + metadata.getFunctionAndTypeManager()); private final EffectivePredicateExtractor effectivePredicateExtractor = new EffectivePredicateExtractor( new RowExpressionDomainTranslator(metadata), - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), metadata.getTypeManager()); private Map scanAssignments; @@ -159,8 +159,8 @@ public void testAggregation() greaterThan(AV, bigintLiteral(2)), equals(EV, FV))), ImmutableMap.of( - CV, count(metadata.getFunctionManager()), - DV, count(metadata.getFunctionManager())), + CV, count(metadata.getFunctionAndTypeManager()), + DV, count(metadata.getFunctionAndTypeManager())), singleGroupingSet(ImmutableList.of(AV, BV, CV)), ImmutableList.of(), AggregationNode.Step.FINAL, @@ -201,7 +201,7 @@ public void testFilter() { PlanNode node = filter(baseTableScan, and( - greaterThan(AV, call(metadata.getFunctionManager(), "rand", DOUBLE, ImmutableList.of())), + greaterThan(AV, call(metadata.getFunctionAndTypeManager(), "rand", DOUBLE, ImmutableList.of())), lessThan(BV, bigintLiteral(10)))); RowExpression effectivePredicate = effectivePredicateExtractor.extract(node); @@ -764,7 +764,7 @@ private RowExpression compare(OperatorType type, RowExpression left, RowExpressi { return call( type.getFunctionName().getFunctionName(), - metadata.getFunctionManager().resolveOperator(type, fromTypes(left.getType(), right.getType())), + metadata.getFunctionAndTypeManager().resolveOperator(type, fromTypes(left.getType(), right.getType())), BOOLEAN, left, right); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEqualityInference.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEqualityInference.java index da168c5bc642..5c48cca66983 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEqualityInference.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEqualityInference.java @@ -482,7 +482,7 @@ private static CallExpression compare(OperatorType type, RowExpression left, Row { return call( type.getFunctionName().getFunctionName(), - METADATA.getFunctionManager().resolveOperator(type, fromTypes(left.getType(), right.getType())), + METADATA.getFunctionAndTypeManager().resolveOperator(type, fromTypes(left.getType(), right.getType())), BOOLEAN, left, right); @@ -504,7 +504,7 @@ private static CallExpression arithmeticOperation(OperatorType type, RowExpressi { return call( type.getFunctionName().getFunctionName(), - METADATA.getFunctionManager().resolveOperator(type, fromTypes(left.getType(), right.getType())), + METADATA.getFunctionAndTypeManager().resolveOperator(type, fromTypes(left.getType(), right.getType())), left.getType(), left, right); @@ -514,7 +514,7 @@ private static boolean isOperation(RowExpression expression, OperatorType type) { if (expression instanceof CallExpression) { CallExpression call = (CallExpression) expression; - Optional expressionOperatorType = METADATA.getFunctionManager().getFunctionMetadata(call.getFunctionHandle()).getOperatorType(); + Optional expressionOperatorType = METADATA.getFunctionAndTypeManager().getFunctionMetadata(call.getFunctionHandle()).getOperatorType(); if (expressionOperatorType.isPresent()) { return expressionOperatorType.get() == type; } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestNullabilityAnalyzer.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestNullabilityAnalyzer.java index cedce889dcb9..a10a84fc9449 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestNullabilityAnalyzer.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestNullabilityAnalyzer.java @@ -46,7 +46,7 @@ public class TestNullabilityAnalyzer .put("c", RowType.from(ImmutableList.of(field("field_1", BIGINT)))) .build()); - private static final NullabilityAnalyzer analyzer = new NullabilityAnalyzer(METADATA.getFunctionManager(), METADATA.getTypeManager()); + private static final NullabilityAnalyzer analyzer = new NullabilityAnalyzer(METADATA.getFunctionAndTypeManager(), METADATA.getTypeManager()); @Test void test() diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestRowExpressionDomainTranslator.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestRowExpressionDomainTranslator.java index 111e7d12fe06..14d5729ed2a1 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestRowExpressionDomainTranslator.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestRowExpressionDomainTranslator.java @@ -902,7 +902,7 @@ public void testFromNullLiteralPredicate() @Test public void testExpressionConstantFolding() { - FunctionHandle hex = metadata.getFunctionManager().lookupFunction("from_hex", fromTypes(VARCHAR)); + FunctionHandle hex = metadata.getFunctionAndTypeManager().lookupFunction("from_hex", fromTypes(VARCHAR)); RowExpression originalExpression = greaterThan(C_VARBINARY, call("from_hex", hex, VARBINARY, stringLiteral("123456"))); ExtractionResult result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_CONSTANT); @@ -1225,13 +1225,13 @@ private static RowExpression isNull(RowExpression expression) private RowExpression cast(RowExpression expression, Type toType) { - FunctionHandle cast = metadata.getFunctionManager().lookupCast(CastType.CAST, expression.getType().getTypeSignature(), toType.getTypeSignature()); + FunctionHandle cast = metadata.getFunctionAndTypeManager().lookupCast(CastType.CAST, expression.getType().getTypeSignature(), toType.getTypeSignature()); return call(CastType.CAST.name(), cast, toType, expression); } private RowExpression not(RowExpression expression) { - return call("not", new FunctionResolution(metadata.getFunctionManager()).notFunction(), expression.getType(), expression); + return call("not", new FunctionResolution(metadata.getFunctionAndTypeManager()).notFunction(), expression.getType(), expression); } private RowExpression in(RowExpression value, List inList) @@ -1254,7 +1254,7 @@ private RowExpression binaryOperator(OperatorType operatorType, RowExpression le { return call( operatorType.name(), - metadata.getFunctionManager().resolveOperator(operatorType, fromTypes(left.getType(), right.getType())), + metadata.getFunctionAndTypeManager().resolveOperator(operatorType, fromTypes(left.getType(), right.getType())), BOOLEAN, left, right); @@ -1264,7 +1264,7 @@ private RowExpression between(RowExpression value, RowExpression min, RowExpress { return call( OperatorType.BETWEEN.name(), - metadata.getFunctionManager().resolveOperator(OperatorType.BETWEEN, fromTypes(value.getType(), min.getType(), max.getType())), + metadata.getFunctionAndTypeManager().resolveOperator(OperatorType.BETWEEN, fromTypes(value.getType(), min.getType(), max.getType())), BOOLEAN, value, min, @@ -1318,10 +1318,10 @@ private RowExpression unprocessableExpression2(VariableReferenceExpression expre private RowExpression randPredicate(VariableReferenceExpression expression) { - RowExpression random = call("random", metadata.getFunctionManager().lookupFunction("random", fromTypes()), DOUBLE); + RowExpression random = call("random", metadata.getFunctionAndTypeManager().lookupFunction("random", fromTypes()), DOUBLE); return greaterThan( expression, - call(CastType.CAST.name(), metadata.getFunctionManager().lookupCast(CastType.CAST, DOUBLE.getTypeSignature(), expression.getType().getTypeSignature()), expression.getType(), random)); + call(CastType.CAST.name(), metadata.getFunctionAndTypeManager().lookupCast(CastType.CAST, DOUBLE.getTypeSignature(), expression.getType().getTypeSignature()), expression.getType(), random)); } private void assertUnsupportedPredicate(RowExpression expression) diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestRowExpressionFormatter.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestRowExpressionFormatter.java index 06322f93aeda..808a618e25ab 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestRowExpressionFormatter.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestRowExpressionFormatter.java @@ -22,7 +22,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.CastType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.relation.CallExpression; import com.facebook.presto.spi.relation.RowExpression; import com.facebook.presto.spi.relation.SpecialFormExpression; @@ -83,8 +83,8 @@ public class TestRowExpressionFormatter { private static final TypeManager typeManager = new TypeRegistry(); - private static final FunctionManager functionManager = new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); - private static final RowExpressionFormatter FORMATTER = new RowExpressionFormatter(functionManager); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + private static final RowExpressionFormatter FORMATTER = new RowExpressionFormatter(FUNCTION_AND_TYPE_MANAGER); private static final VariableReferenceExpression C_BIGINT = new VariableReferenceExpression("c_bigint", BIGINT); private static final VariableReferenceExpression C_BIGINT_ARRAY = new VariableReferenceExpression("c_bigint_array", new ArrayType(BIGINT)); @@ -200,7 +200,7 @@ public void testCalls() RowExpression expression = createCallExpression(ADD); callExpression = call( NEGATION.name(), - functionManager.resolveOperator(NEGATION, fromTypes(expression.getType())), + FUNCTION_AND_TYPE_MANAGER.resolveOperator(NEGATION, fromTypes(expression.getType())), expression.getType(), expression); assertEquals(format(callExpression), "-((c_bigint) + (BIGINT 5))"); @@ -209,7 +209,7 @@ public void testCalls() ArrayType arrayType = (ArrayType) C_BIGINT_ARRAY.getType(); Type elementType = arrayType.getElementType(); RowExpression subscriptExpression = call(SUBSCRIPT.name(), - functionManager.resolveOperator(SUBSCRIPT, fromTypes(arrayType, elementType)), + FUNCTION_AND_TYPE_MANAGER.resolveOperator(SUBSCRIPT, fromTypes(arrayType, elementType)), elementType, ImmutableList.of(C_BIGINT_ARRAY, constant(0L, INTEGER))); callExpression = subscriptExpression; @@ -218,7 +218,7 @@ public void testCalls() // cast callExpression = call( CAST.name(), - functionManager.lookupCast(CastType.CAST, TINYINT.getTypeSignature(), BIGINT.getTypeSignature()), + FUNCTION_AND_TYPE_MANAGER.lookupCast(CastType.CAST, TINYINT.getTypeSignature(), BIGINT.getTypeSignature()), BIGINT, constant(1L, TINYINT)); assertEquals(format(callExpression), "CAST(TINYINT 1 AS bigint)"); @@ -226,7 +226,7 @@ public void testCalls() // between callExpression = call( BETWEEN.name(), - functionManager.resolveOperator(BETWEEN, fromTypes(BIGINT, BIGINT, BIGINT)), + FUNCTION_AND_TYPE_MANAGER.resolveOperator(BETWEEN, fromTypes(BIGINT, BIGINT, BIGINT)), BOOLEAN, subscriptExpression, constant(1L, BIGINT), @@ -236,7 +236,7 @@ public void testCalls() // other callExpression = call( HASH_CODE.name(), - functionManager.resolveOperator(HASH_CODE, fromTypes(BIGINT)), + FUNCTION_AND_TYPE_MANAGER.resolveOperator(HASH_CODE, fromTypes(BIGINT)), BIGINT, constant(1L, BIGINT)); assertEquals(format(callExpression), "HASH_CODE(BIGINT 1)"); @@ -266,7 +266,7 @@ public void testComplex() RowExpression expression = createCallExpression(ADD); complexExpression = call( SUBTRACT.name(), - functionManager.resolveOperator(SUBTRACT, fromTypes(BIGINT, BIGINT)), + FUNCTION_AND_TYPE_MANAGER.resolveOperator(SUBTRACT, fromTypes(BIGINT, BIGINT)), BIGINT, C_BIGINT, expression); @@ -275,7 +275,7 @@ public void testComplex() RowExpression expression1 = createCallExpression(ADD); RowExpression expression2 = call( MULTIPLY.name(), - functionManager.resolveOperator(MULTIPLY, fromTypes(BIGINT, BIGINT)), + FUNCTION_AND_TYPE_MANAGER.resolveOperator(MULTIPLY, fromTypes(BIGINT, BIGINT)), BIGINT, expression1, C_BIGINT); @@ -286,17 +286,17 @@ public void testComplex() ArrayType arrayType = (ArrayType) C_BIGINT_ARRAY.getType(); Type elementType = arrayType.getElementType(); expression1 = call(SUBSCRIPT.name(), - functionManager.resolveOperator(SUBSCRIPT, fromTypes(arrayType, elementType)), + FUNCTION_AND_TYPE_MANAGER.resolveOperator(SUBSCRIPT, fromTypes(arrayType, elementType)), elementType, ImmutableList.of(C_BIGINT_ARRAY, constant(5L, INTEGER))); expression2 = call( NEGATION.name(), - functionManager.resolveOperator(NEGATION, fromTypes(expression1.getType())), + FUNCTION_AND_TYPE_MANAGER.resolveOperator(NEGATION, fromTypes(expression1.getType())), expression1.getType(), expression1); expression3 = call( ADD.name(), - functionManager.resolveOperator(ADD, fromTypes(expression2.getType(), BIGINT)), + FUNCTION_AND_TYPE_MANAGER.resolveOperator(ADD, fromTypes(expression2.getType(), BIGINT)), BIGINT, expression2, constant(5L, BIGINT)); @@ -312,7 +312,7 @@ private static CallExpression createCallExpression(OperatorType type) { return call( type.name(), - functionManager.resolveOperator(type, fromTypes(BIGINT, BIGINT)), + FUNCTION_AND_TYPE_MANAGER.resolveOperator(type, fromTypes(BIGINT, BIGINT)), BIGINT, C_BIGINT, constant(5L, BIGINT)); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestRowExpressionRewriter.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestRowExpressionRewriter.java index 92ddd3ddd3aa..cbdfb4059791 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestRowExpressionRewriter.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestRowExpressionRewriter.java @@ -16,7 +16,7 @@ import com.facebook.presto.block.BlockEncodingManager; import com.facebook.presto.expressions.RowExpressionRewriter; import com.facebook.presto.expressions.RowExpressionTreeRewriter; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionMetadata; import com.facebook.presto.spi.function.StandardFunctionResolution; import com.facebook.presto.spi.relation.CallExpression; @@ -42,13 +42,13 @@ public class TestRowExpressionRewriter { - private FunctionManager functionManager; + private FunctionAndTypeManager functionAndTypeManager; @BeforeClass public void setup() { TypeRegistry typeManager = new TypeRegistry(); - this.functionManager = new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + this.functionAndTypeManager = new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); } @Test @@ -57,7 +57,7 @@ public void testSimple() // successful rewrite RowExpression predicate = call( GREATER_THAN.name(), - functionManager.resolveOperator(GREATER_THAN, fromTypes(BIGINT, BIGINT)), + functionAndTypeManager.resolveOperator(GREATER_THAN, fromTypes(BIGINT, BIGINT)), BOOLEAN, constant(1L, BIGINT), constant(2L, BIGINT)); @@ -72,7 +72,7 @@ public void testSimple() // no rewrite RowExpression nonPredicate = call( ADD.name(), - functionManager.resolveOperator(ADD, fromTypes(BIGINT, BIGINT)), + functionAndTypeManager.resolveOperator(ADD, fromTypes(BIGINT, BIGINT)), BIGINT, constant(1L, BIGINT), constant(2L, BIGINT)); @@ -86,7 +86,7 @@ public void testInliner() { RowExpression predicate = call( GREATER_THAN.name(), - functionManager.resolveOperator(GREATER_THAN, fromTypes(BIGINT, BIGINT)), + functionAndTypeManager.resolveOperator(GREATER_THAN, fromTypes(BIGINT, BIGINT)), BOOLEAN, constant(1L, BIGINT), constant(2L, BIGINT)); @@ -94,7 +94,7 @@ public void testInliner() // no rewrite RowExpression nonPredicate = call( ADD.name(), - functionManager.resolveOperator(ADD, fromTypes(BIGINT, BIGINT)), + functionAndTypeManager.resolveOperator(ADD, fromTypes(BIGINT, BIGINT)), BIGINT, constant(1L, BIGINT), constant(2L, BIGINT)); @@ -115,20 +115,20 @@ public static RowExpression rewrite(RowExpression expression) private static class Visitor extends RowExpressionRewriter { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final StandardFunctionResolution functionResolution; Visitor() { TypeRegistry typeManager = new TypeRegistry(); - this.functionManager = new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); - this.functionResolution = new FunctionResolution(functionManager); + this.functionAndTypeManager = new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + this.functionResolution = new FunctionResolution(functionAndTypeManager); } @Override public RowExpression rewriteCall(CallExpression node, Void context, RowExpressionTreeRewriter treeRewriter) { - FunctionMetadata metadata = functionManager.getFunctionMetadata(node.getFunctionHandle()); + FunctionMetadata metadata = functionAndTypeManager.getFunctionMetadata(node.getFunctionHandle()); if (metadata.getOperatorType().isPresent() && metadata.getOperatorType().get().isComparisonOperator()) { return call("not", functionResolution.notFunction(), BOOLEAN, node); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestSortExpressionExtractor.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestSortExpressionExtractor.java index 6e5bfcb69579..d143d11e4642 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestSortExpressionExtractor.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestSortExpressionExtractor.java @@ -96,7 +96,7 @@ private void assertNoSortExpression(Expression expression) Optional actual = SortExpressionExtractor.extractSortExpression( BUILD_VARIABLES, TRANSLATOR.translate(expression, TYPES), - METADATA.getFunctionManager()); + METADATA.getFunctionAndTypeManager()); assertEquals(actual, Optional.empty()); } @@ -132,7 +132,7 @@ private static void assertGetSortExpression(Expression expression, String expect Optional actual = SortExpressionExtractor.extractSortExpression( BUILD_VARIABLES, TRANSLATOR.translate(expression, TYPES), - METADATA.getFunctionManager()); + METADATA.getFunctionAndTypeManager()); assertEquals(actual, expected); } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestTypeValidator.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestTypeValidator.java index 92a3e0036c3a..a59fd402d8c2 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestTypeValidator.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestTypeValidator.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.predicate.TupleDomain; import com.facebook.presto.common.type.StandardTypes; import com.facebook.presto.common.type.VarcharType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorId; import com.facebook.presto.spi.TableHandle; @@ -76,7 +76,7 @@ public class TestTypeValidator Optional.empty()); private static final SqlParser SQL_PARSER = new SqlParser(); private static final TypeValidator TYPE_VALIDATOR = new TypeValidator(); - private static final FunctionManager FUNCTION_MANAGER = createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_MANAGER = createTestMetadataManager().getFunctionAndTypeManager(); private static final FunctionHandle SUM = FUNCTION_MANAGER.lookupFunction("sum", fromTypes(DOUBLE)); private PlanVariableAllocator variableAllocator; diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AggregationFunctionMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AggregationFunctionMatcher.java index 5b1bbc0ee676..ff2215d567a9 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AggregationFunctionMatcher.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AggregationFunctionMatcher.java @@ -14,7 +14,7 @@ package com.facebook.presto.sql.planner.assertions; import com.facebook.presto.Session; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.AggregationNode.Aggregation; @@ -60,7 +60,7 @@ public Optional getAssignedVariable(PlanNode node, FunctionCall expectedCall = callMaker.getExpectedValue(symbolAliases); for (Map.Entry assignment : aggregationNode.getAggregations().entrySet()) { - if (verifyAggregation(metadata.getFunctionManager(), assignment.getValue(), expectedCall)) { + if (verifyAggregation(metadata.getFunctionAndTypeManager(), assignment.getValue(), expectedCall)) { checkState(!result.isPresent(), "Ambiguous function calls in %s", aggregationNode); result = Optional.of(assignment.getKey()); } @@ -69,9 +69,9 @@ public Optional getAssignedVariable(PlanNode node, return result; } - private static boolean verifyAggregation(FunctionManager functionManager, Aggregation aggregation, FunctionCall expectedCall) + private static boolean verifyAggregation(FunctionAndTypeManager functionAndTypeManager, Aggregation aggregation, FunctionCall expectedCall) { - return functionManager.getFunctionMetadata(aggregation.getFunctionHandle()).getName().getFunctionName().equalsIgnoreCase(expectedCall.getName().getSuffix()) && + return functionAndTypeManager.getFunctionMetadata(aggregation.getFunctionHandle()).getName().getFunctionName().equalsIgnoreCase(expectedCall.getName().getSuffix()) && aggregation.getArguments().size() == expectedCall.getArguments().size() && Streams.zip( aggregation.getArguments().stream(), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/BasePlanTest.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/BasePlanTest.java index 0459082cc0ca..de72928d3e68 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/BasePlanTest.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/BasePlanTest.java @@ -162,7 +162,7 @@ protected void assertDistributedPlan(String sql, Session session, PlanMatchPatte protected void assertMinimallyOptimizedPlan(@Language("SQL") String sql, PlanMatchPattern pattern) { List optimizers = ImmutableList.of( - new UnaliasSymbolReferences(queryRunner.getMetadata().getFunctionManager()), + new UnaliasSymbolReferences(queryRunner.getMetadata().getFunctionAndTypeManager()), new PruneUnreferencedOutputs(), new IterativeOptimizer( new RuleStatsRecorder(), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/DynamicFilterMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/DynamicFilterMatcher.java index c06387a0a1bc..ba77c6fc4c3c 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/DynamicFilterMatcher.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/DynamicFilterMatcher.java @@ -78,9 +78,9 @@ private MatchResult match(FilterNode filterNode, Session session, Metadata metad this.symbolAliases = symbolAliases; LogicalRowExpressions logicalRowExpressions = new LogicalRowExpressions( - new RowExpressionDeterminismEvaluator(metadata.getFunctionManager()), - new FunctionResolution(metadata.getFunctionManager()), - metadata.getFunctionManager()); + new RowExpressionDeterminismEvaluator(metadata.getFunctionAndTypeManager()), + new FunctionResolution(metadata.getFunctionAndTypeManager()), + metadata.getFunctionAndTypeManager()); boolean staticFilterMatches = expectedStaticFilter.map(filter -> { RowExpressionVerifier verifier = new RowExpressionVerifier(symbolAliases, metadata, session); RowExpression staticFilter = logicalRowExpressions.combineConjuncts(extractDynamicFilters(filterNode.getPredicate()).getStaticConjuncts()); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanAssert.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanAssert.java index 6cebe5051a58..6db995cb3936 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanAssert.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanAssert.java @@ -46,7 +46,7 @@ public static void assertPlan(Session session, Metadata metadata, StatsProvider // TODO (Issue #13231) add back printing unresolved plan once we have no need to translate OriginalExpression to RowExpression if (!matches.isMatch()) { PlanNode resolvedPlan = resolveGroupReferences(actual.getRoot(), lookup); - String resolvedFormattedPlan = textLogicalPlan(planSanitizer.apply(resolvedPlan), actual.getTypes(), metadata.getFunctionManager(), StatsAndCosts.empty(), session, 0); + String resolvedFormattedPlan = textLogicalPlan(planSanitizer.apply(resolvedPlan), actual.getTypes(), metadata.getFunctionAndTypeManager(), StatsAndCosts.empty(), session, 0); throw new AssertionError(format( "Plan does not match, expected [\n\n%s\n] but found [\n\n%s\n]", pattern, diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/RowExpressionVerifier.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/RowExpressionVerifier.java index 2ce851d2448d..fecd596d2539 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/RowExpressionVerifier.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/RowExpressionVerifier.java @@ -103,7 +103,7 @@ final class RowExpressionVerifier this.symbolAliases = requireNonNull(symbolAliases, "symbolLayout is null"); this.metadata = requireNonNull(metadata, "metadata is null"); this.session = requireNonNull(session, "session is null"); - this.functionResolution = new FunctionResolution(metadata.getFunctionManager()); + this.functionResolution = new FunctionResolution(metadata.getFunctionAndTypeManager()); } @Override @@ -208,7 +208,7 @@ protected Boolean visitInPredicate(InPredicate expected, RowExpression actual) protected Boolean visitComparisonExpression(ComparisonExpression expected, RowExpression actual) { if (actual instanceof CallExpression) { - FunctionMetadata functionMetadata = metadata.getFunctionManager().getFunctionMetadata(((CallExpression) actual).getFunctionHandle()); + FunctionMetadata functionMetadata = metadata.getFunctionAndTypeManager().getFunctionMetadata(((CallExpression) actual).getFunctionHandle()); if (!functionMetadata.getOperatorType().isPresent() || !functionMetadata.getOperatorType().get().isComparisonOperator()) { return false; } @@ -261,7 +261,7 @@ private static OperatorType getOperatorType(ComparisonExpression.Operator operat protected Boolean visitArithmeticBinary(ArithmeticBinaryExpression expected, RowExpression actual) { if (actual instanceof CallExpression) { - FunctionMetadata functionMetadata = metadata.getFunctionManager().getFunctionMetadata(((CallExpression) actual).getFunctionHandle()); + FunctionMetadata functionMetadata = metadata.getFunctionAndTypeManager().getFunctionMetadata(((CallExpression) actual).getFunctionHandle()); if (!functionMetadata.getOperatorType().isPresent() || !functionMetadata.getOperatorType().get().isArithmeticOperator()) { return false; } @@ -512,7 +512,7 @@ protected Boolean visitFunctionCall(FunctionCall expected, RowExpression actual) } CallExpression actualFunction = (CallExpression) actual; - if (!expected.getName().getSuffix().equals(metadata.getFunctionManager().getFunctionMetadata(actualFunction.getFunctionHandle()).getName().getFunctionName())) { + if (!expected.getName().getSuffix().equals(metadata.getFunctionAndTypeManager().getFunctionMetadata(actualFunction.getFunctionHandle()).getName().getFunctionName())) { return false; } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/WindowFunctionMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/WindowFunctionMatcher.java index a60c914057de..d8de8c29c84c 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/WindowFunctionMatcher.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/WindowFunctionMatcher.java @@ -74,7 +74,7 @@ public Optional getAssignedVariable(PlanNode node, List matchedOutputs = windowNode.getWindowFunctions().entrySet().stream() .filter(assignment -> { - if (!expectedCall.getName().equals(QualifiedName.of(metadata.getFunctionManager().getFunctionMetadata(assignment.getValue().getFunctionCall().getFunctionHandle()).getName().getFunctionName()))) { + if (!expectedCall.getName().equals(QualifiedName.of(metadata.getFunctionAndTypeManager().getFunctionMetadata(assignment.getValue().getFunctionCall().getFunctionHandle()).getName().getFunctionName()))) { return false; } if (!functionHandle.map(assignment.getValue().getFunctionHandle()::equals).orElse(true)) { diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestInlineSqlFunctions.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestInlineSqlFunctions.java index 00ffbb341ea0..640a2e0dc873 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestInlineSqlFunctions.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestInlineSqlFunctions.java @@ -24,7 +24,7 @@ import com.facebook.presto.functionNamespace.execution.SqlFunctionExecutors; import com.facebook.presto.functionNamespace.execution.thrift.ThriftSqlFunctionExecutor; import com.facebook.presto.functionNamespace.testing.InMemoryFunctionNamespaceManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.function.FunctionImplementationType; @@ -119,8 +119,8 @@ public class TestInlineSqlFunctions public void setup() { RuleTester tester = new RuleTester(); - FunctionManager functionManager = tester.getMetadata().getFunctionManager(); - functionManager.addFunctionNamespace( + FunctionAndTypeManager functionAndTypeManager = tester.getMetadata().getFunctionAndTypeManager(); + functionAndTypeManager.addFunctionNamespace( "unittest", new InMemoryFunctionNamespaceManager( "unittest", @@ -130,10 +130,10 @@ public void setup() JAVA, THRIFT), new ThriftSqlFunctionExecutor(null)), new SqlInvokedFunctionNamespaceManagerConfig().setSupportedFunctionLanguages("sql,java"))); - functionManager.createFunction(SQL_FUNCTION_SQUARE, true); - functionManager.createFunction(THRIFT_FUNCTION_FOO, true); - functionManager.createFunction(SQL_FUNCTION_ADD_1_TO_INT_ARRAY, true); - functionManager.createFunction(SQL_FUNCTION_ADD_1_TO_BIGINT_ARRAY, true); + functionAndTypeManager.createFunction(SQL_FUNCTION_SQUARE, true); + functionAndTypeManager.createFunction(THRIFT_FUNCTION_FOO, true); + functionAndTypeManager.createFunction(SQL_FUNCTION_ADD_1_TO_INT_ARRAY, true); + functionAndTypeManager.createFunction(SQL_FUNCTION_ADD_1_TO_BIGINT_ARRAY, true); this.tester = tester; } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestJoinEnumerator.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestJoinEnumerator.java index 25ed69cff20e..b794a4ac237c 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestJoinEnumerator.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestJoinEnumerator.java @@ -67,7 +67,7 @@ public void setUp() queryRunner = new LocalQueryRunner(testSessionBuilder().build()); metadata = queryRunner.getMetadata(); determinismEvaluator = new RowExpressionDeterminismEvaluator(metadata); - functionResolution = new FunctionResolution(metadata.getFunctionManager()); + functionResolution = new FunctionResolution(metadata.getFunctionAndTypeManager()); } @AfterClass(alwaysRun = true) diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestJoinNodeFlattener.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestJoinNodeFlattener.java index b83c9645e9eb..0140679c7c8e 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestJoinNodeFlattener.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestJoinNodeFlattener.java @@ -63,7 +63,7 @@ public void setUp() { queryRunner = new LocalQueryRunner(testSessionBuilder().build()); determinismEvaluator = new RowExpressionDeterminismEvaluator(queryRunner.getMetadata()); - functionResolution = new FunctionResolution(queryRunner.getMetadata().getFunctionManager()); + functionResolution = new FunctionResolution(queryRunner.getMetadata().getFunctionAndTypeManager()); } @AfterClass(alwaysRun = true) diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestMergeAdjacentWindows.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestMergeAdjacentWindows.java index ecf34c21701f..579030675b7a 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestMergeAdjacentWindows.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestMergeAdjacentWindows.java @@ -58,9 +58,9 @@ public class TestMergeAdjacentWindows Optional.empty(), Optional.empty()); - private static final FunctionHandle SUM_FUNCTION_HANDLE = createTestMetadataManager().getFunctionManager().lookupFunction("sum", fromTypes(DOUBLE)); - private static final FunctionHandle AVG_FUNCTION_HANDLE = createTestMetadataManager().getFunctionManager().lookupFunction("avg", fromTypes(DOUBLE)); - private static final FunctionHandle LAG_FUNCTION_HANDLE = createTestMetadataManager().getFunctionManager().lookupFunction("lag", fromTypes(DOUBLE)); + private static final FunctionHandle SUM_FUNCTION_HANDLE = createTestMetadataManager().getFunctionAndTypeManager().lookupFunction("sum", fromTypes(DOUBLE)); + private static final FunctionHandle AVG_FUNCTION_HANDLE = createTestMetadataManager().getFunctionAndTypeManager().lookupFunction("avg", fromTypes(DOUBLE)); + private static final FunctionHandle LAG_FUNCTION_HANDLE = createTestMetadataManager().getFunctionAndTypeManager().lookupFunction("lag", fromTypes(DOUBLE)); private static final String columnAAlias = "ALIAS_A"; private static final ExpectedValueProvider specificationA = specification(ImmutableList.of(columnAAlias), ImmutableList.of(), ImmutableMap.of()); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPlanRemoteProjections.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPlanRemoteProjections.java index 451d3c4682af..69eca6683dcc 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPlanRemoteProjections.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPlanRemoteProjections.java @@ -20,7 +20,7 @@ import com.facebook.presto.functionNamespace.execution.SqlFunctionExecutors; import com.facebook.presto.functionNamespace.execution.thrift.ThriftSqlFunctionExecutor; import com.facebook.presto.functionNamespace.testing.InMemoryFunctionNamespaceManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionImplementationType; import com.facebook.presto.spi.function.Parameter; import com.facebook.presto.spi.function.RoutineCharacteristics; @@ -99,8 +99,8 @@ public class TestPlanRemoteProjections @BeforeClass public void setup() { - FunctionManager functionManager = getFunctionManager(); - functionManager.addFunctionNamespace( + FunctionAndTypeManager functionAndTypeManager = getFunctionManager(); + functionAndTypeManager.addFunctionNamespace( "unittest", new InMemoryFunctionNamespaceManager( "unittest", @@ -110,10 +110,10 @@ public void setup() JAVA, THRIFT), new ThriftSqlFunctionExecutor(null)), new SqlInvokedFunctionNamespaceManagerConfig().setSupportedFunctionLanguages("sql,java"))); - functionManager.createFunction(FUNCTION_REMOTE_FOO_0, true); - functionManager.createFunction(FUNCTION_REMOTE_FOO_1, true); - functionManager.createFunction(FUNCTION_REMOTE_FOO_2, true); - functionManager.createFunction(FUNCTION_REMOTE_FOO_3, true); + functionAndTypeManager.createFunction(FUNCTION_REMOTE_FOO_0, true); + functionAndTypeManager.createFunction(FUNCTION_REMOTE_FOO_1, true); + functionAndTypeManager.createFunction(FUNCTION_REMOTE_FOO_2, true); + functionAndTypeManager.createFunction(FUNCTION_REMOTE_FOO_3, true); } @Test diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPruneOrderByInAggregation.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPruneOrderByInAggregation.java index e490c99990a4..400679343e7b 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPruneOrderByInAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPruneOrderByInAggregation.java @@ -13,7 +13,7 @@ */ package com.facebook.presto.sql.planner.iterative.rule; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.relation.VariableReferenceExpression; @@ -38,7 +38,7 @@ public class TestPruneOrderByInAggregation extends BaseRuleTest { - private static final FunctionManager FUNCTION_MANAGER = MetadataManager.createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_MANAGER = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); @Test public void testBasics() diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPruneWindowColumns.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPruneWindowColumns.java index cd4e4e8a20f0..60ab90d96b8e 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPruneWindowColumns.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPruneWindowColumns.java @@ -59,7 +59,7 @@ public class TestPruneWindowColumns extends BaseRuleTest { private static final String FUNCTION_NAME = "min"; - private static final FunctionHandle FUNCTION_HANDLE = createTestMetadataManager().getFunctionManager().lookupFunction(FUNCTION_NAME, fromTypes(BIGINT)); + private static final FunctionHandle FUNCTION_HANDLE = createTestMetadataManager().getFunctionAndTypeManager().lookupFunction(FUNCTION_NAME, fromTypes(BIGINT)); private static final List inputSymbolNameList = ImmutableList.of("orderKey", "partitionKey", "hash", "startValue1", "startValue2", "endValue1", "endValue2", "input1", "input2", "unused"); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPushProjectionThroughUnion.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPushProjectionThroughUnion.java index c8467a4bcf45..2cd7155505a9 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPushProjectionThroughUnion.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestPushProjectionThroughUnion.java @@ -49,7 +49,7 @@ public void testDoesNotFire() @Test public void test() { - FunctionResolution functionResolution = new FunctionResolution(tester().getMetadata().getFunctionManager()); + FunctionResolution functionResolution = new FunctionResolution(tester().getMetadata().getFunctionAndTypeManager()); tester().assertThat(new PushProjectionThroughUnion()) .on(p -> { VariableReferenceExpression a = p.variable("a"); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestReorderJoins.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestReorderJoins.java index 3454f4c69dfa..6f27db5eb1bc 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestReorderJoins.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestReorderJoins.java @@ -44,7 +44,7 @@ import static com.facebook.presto.common.function.OperatorType.LESS_THAN; import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.BooleanType.BOOLEAN; -import static com.facebook.presto.metadata.FunctionManager.qualifyFunctionName; +import static com.facebook.presto.metadata.FunctionAndTypeManager.qualifyFunctionName; import static com.facebook.presto.sql.analyzer.FeaturesConfig.JoinDistributionType.AUTOMATIC; import static com.facebook.presto.sql.analyzer.FeaturesConfig.JoinDistributionType.BROADCAST; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.equiJoinClause; @@ -74,7 +74,7 @@ public void setUp() JOIN_DISTRIBUTION_TYPE, JoinDistributionType.AUTOMATIC.name(), JOIN_REORDERING_STRATEGY, JoinReorderingStrategy.AUTOMATIC.name()), Optional.of(4)); - this.functionResolution = new FunctionResolution(tester.getMetadata().getFunctionManager()); + this.functionResolution = new FunctionResolution(tester.getMetadata().getFunctionAndTypeManager()); } @AfterClass(alwaysRun = true) @@ -336,7 +336,7 @@ public void testDoesNotFireForNonDeterministicFilter() ImmutableList.of(p.variable("A1"), p.variable("B1")), Optional.of(comparisonRowExpression(LESS_THAN, variable("A1", BIGINT), call( RANDOM.toString(), - tester.getMetadata().getFunctionManager().resolveFunction(Optional.empty(), qualifyFunctionName(RANDOM), ImmutableList.of()), + tester.getMetadata().getFunctionAndTypeManager().resolveFunction(Optional.empty(), qualifyFunctionName(RANDOM), ImmutableList.of()), BIGINT, ImmutableList.of()))))) .doesNotFire(); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRewriteFilterWithExternalFunctionToProject.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRewriteFilterWithExternalFunctionToProject.java index 6de595b25f69..b02a9f5d2997 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRewriteFilterWithExternalFunctionToProject.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestRewriteFilterWithExternalFunctionToProject.java @@ -20,7 +20,7 @@ import com.facebook.presto.functionNamespace.execution.SqlFunctionExecutors; import com.facebook.presto.functionNamespace.execution.thrift.ThriftSqlFunctionExecutor; import com.facebook.presto.functionNamespace.testing.InMemoryFunctionNamespaceManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionImplementationType; import com.facebook.presto.spi.function.Parameter; import com.facebook.presto.spi.function.RoutineCharacteristics; @@ -64,8 +64,8 @@ public class TestRewriteFilterWithExternalFunctionToProject @BeforeClass public void setup() { - FunctionManager functionManager = getFunctionManager(); - functionManager.addFunctionNamespace( + FunctionAndTypeManager functionAndTypeManager = getFunctionManager(); + functionAndTypeManager.addFunctionNamespace( "unittest", new InMemoryFunctionNamespaceManager( "unittest", @@ -75,8 +75,8 @@ public void setup() JAVA, THRIFT), new ThriftSqlFunctionExecutor(null)), new SqlInvokedFunctionNamespaceManagerConfig().setSupportedFunctionLanguages("sql,java"))); - functionManager.createFunction(FUNCTION_TANGENT, true); - functionManager.createFunction(FUNCTION_REMOTE_FOO, true); + functionAndTypeManager.createFunction(FUNCTION_TANGENT, true); + functionAndTypeManager.createFunction(FUNCTION_REMOTE_FOO, true); } @Test diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestSwapAdjacentWindowsBySpecifications.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestSwapAdjacentWindowsBySpecifications.java index f5e38dec594a..84bc4109db11 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestSwapAdjacentWindowsBySpecifications.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestSwapAdjacentWindowsBySpecifications.java @@ -59,7 +59,7 @@ public TestSwapAdjacentWindowsBySpecifications() Optional.empty(), Optional.empty()); - functionHandle = createTestMetadataManager().getFunctionManager().lookupFunction("avg", fromTypes(BIGINT)); + functionHandle = createTestMetadataManager().getFunctionAndTypeManager().lookupFunction("avg", fromTypes(BIGINT)); } @Test diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestTransformCorrelatedScalarAggregationToJoin.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestTransformCorrelatedScalarAggregationToJoin.java index e1b19b491899..318e36f88509 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestTransformCorrelatedScalarAggregationToJoin.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestTransformCorrelatedScalarAggregationToJoin.java @@ -36,7 +36,7 @@ public class TestTransformCorrelatedScalarAggregationToJoin @Test public void doesNotFireOnPlanWithoutApplyNode() { - tester().assertThat(new TransformCorrelatedScalarAggregationToJoin(tester().getMetadata().getFunctionManager())) + tester().assertThat(new TransformCorrelatedScalarAggregationToJoin(tester().getMetadata().getFunctionAndTypeManager())) .on(p -> p.values(p.variable("a"))) .doesNotFire(); } @@ -44,7 +44,7 @@ public void doesNotFireOnPlanWithoutApplyNode() @Test public void doesNotFireOnCorrelatedWithoutAggregation() { - tester().assertThat(new TransformCorrelatedScalarAggregationToJoin(tester().getMetadata().getFunctionManager())) + tester().assertThat(new TransformCorrelatedScalarAggregationToJoin(tester().getMetadata().getFunctionAndTypeManager())) .on(p -> p.lateral( ImmutableList.of(p.variable("corr")), p.values(p.variable("corr")), @@ -55,7 +55,7 @@ public void doesNotFireOnCorrelatedWithoutAggregation() @Test public void doesNotFireOnUncorrelated() { - tester().assertThat(new TransformCorrelatedScalarAggregationToJoin(tester().getMetadata().getFunctionManager())) + tester().assertThat(new TransformCorrelatedScalarAggregationToJoin(tester().getMetadata().getFunctionAndTypeManager())) .on(p -> p.lateral( ImmutableList.of(), p.values(p.variable("a")), @@ -66,7 +66,7 @@ public void doesNotFireOnUncorrelated() @Test public void doesNotFireOnCorrelatedWithNonScalarAggregation() { - tester().assertThat(new TransformCorrelatedScalarAggregationToJoin(tester().getMetadata().getFunctionManager())) + tester().assertThat(new TransformCorrelatedScalarAggregationToJoin(tester().getMetadata().getFunctionAndTypeManager())) .on(p -> p.lateral( ImmutableList.of(p.variable("corr")), p.values(p.variable("corr")), @@ -80,7 +80,7 @@ public void doesNotFireOnCorrelatedWithNonScalarAggregation() @Test public void rewritesOnSubqueryWithoutProjection() { - tester().assertThat(new TransformCorrelatedScalarAggregationToJoin(tester().getMetadata().getFunctionManager())) + tester().assertThat(new TransformCorrelatedScalarAggregationToJoin(tester().getMetadata().getFunctionAndTypeManager())) .on(p -> p.lateral( ImmutableList.of(p.variable("corr")), p.values(p.variable("corr")), @@ -102,7 +102,7 @@ public void rewritesOnSubqueryWithoutProjection() @Test public void rewritesOnSubqueryWithProjection() { - tester().assertThat(new TransformCorrelatedScalarAggregationToJoin(tester().getMetadata().getFunctionManager())) + tester().assertThat(new TransformCorrelatedScalarAggregationToJoin(tester().getMetadata().getFunctionAndTypeManager())) .on(p -> p.lateral( ImmutableList.of(p.variable("corr")), p.values(p.variable("corr")), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestTransformExistsApplyToLateralJoin.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestTransformExistsApplyToLateralJoin.java index b0cb16f6e096..b3707c2c7896 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestTransformExistsApplyToLateralJoin.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestTransformExistsApplyToLateralJoin.java @@ -38,11 +38,11 @@ public class TestTransformExistsApplyToLateralJoin @Test public void testDoesNotFire() { - tester().assertThat(new TransformExistsApplyToLateralNode(tester().getMetadata().getFunctionManager())) + tester().assertThat(new TransformExistsApplyToLateralNode(tester().getMetadata().getFunctionAndTypeManager())) .on(p -> p.values(p.variable("a"))) .doesNotFire(); - tester().assertThat(new TransformExistsApplyToLateralNode(tester().getMetadata().getFunctionManager())) + tester().assertThat(new TransformExistsApplyToLateralNode(tester().getMetadata().getFunctionAndTypeManager())) .on(p -> p.lateral( ImmutableList.of(p.variable("a")), @@ -54,7 +54,7 @@ public void testDoesNotFire() @Test public void testRewrite() { - tester().assertThat(new TransformExistsApplyToLateralNode(tester().getMetadata().getFunctionManager())) + tester().assertThat(new TransformExistsApplyToLateralNode(tester().getMetadata().getFunctionAndTypeManager())) .on(p -> p.apply( assignment(p.variable("b", BOOLEAN), expression("EXISTS(SELECT TRUE)")), @@ -73,7 +73,7 @@ public void testRewrite() @Test public void testRewritesToLimit() { - tester().assertThat(new TransformExistsApplyToLateralNode(tester().getMetadata().getFunctionManager())) + tester().assertThat(new TransformExistsApplyToLateralNode(tester().getMetadata().getFunctionAndTypeManager())) .on(p -> p.apply( assignment(p.variable("b", BOOLEAN), expression("EXISTS(SELECT TRUE)")), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestTranslateExpressions.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestTranslateExpressions.java index c5fda86d93e3..a33b7c8a8a7e 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestTranslateExpressions.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestTranslateExpressions.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.function.OperatorType; import com.facebook.presto.common.type.FunctionType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.plan.AggregationNode; @@ -48,7 +48,7 @@ public class TestTranslateExpressions extends BaseRuleTest { private static final Metadata METADATA = createTestMetadataManager(); - private static final FunctionManager FUNCTION_MANAGER = METADATA.getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_MANAGER = METADATA.getFunctionAndTypeManager(); private static final FunctionResolution FUNCTION_RESOLUTION = new FunctionResolution(FUNCTION_MANAGER); private static final FunctionHandle REDUCE_AGG = FUNCTION_MANAGER.lookupFunction( "reduce_agg", diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/BaseRuleTest.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/BaseRuleTest.java index 5dccc9da1e01..95ba4df4d45b 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/BaseRuleTest.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/BaseRuleTest.java @@ -13,7 +13,7 @@ */ package com.facebook.presto.sql.planner.iterative.rule.test; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.Plugin; import com.google.common.collect.ImmutableList; @@ -57,8 +57,8 @@ protected Metadata getMetadata() return tester.getMetadata(); } - protected FunctionManager getFunctionManager() + protected FunctionAndTypeManager getFunctionManager() { - return tester.getMetadata().getFunctionManager(); + return tester.getMetadata().getFunctionAndTypeManager(); } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java index 90514cb9f523..9155a0d790e5 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java @@ -99,7 +99,7 @@ import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.VarbinaryType.VARBINARY; -import static com.facebook.presto.metadata.FunctionManager.qualifyFunctionName; +import static com.facebook.presto.metadata.FunctionAndTypeManager.qualifyFunctionName; import static com.facebook.presto.spi.plan.LimitNode.Step.FINAL; import static com.facebook.presto.sql.analyzer.ExpressionAnalyzer.getExpressionTypes; import static com.facebook.presto.sql.planner.PlannerUtils.toOrderingScheme; @@ -292,13 +292,13 @@ public AggregationNode aggregation(Consumer aggregationBuild public CallExpression binaryOperation(OperatorType operatorType, RowExpression left, RowExpression right) { - FunctionHandle functionHandle = new FunctionResolution(metadata.getFunctionManager()).arithmeticFunction(operatorType, left.getType(), right.getType()); + FunctionHandle functionHandle = new FunctionResolution(metadata.getFunctionAndTypeManager()).arithmeticFunction(operatorType, left.getType(), right.getType()); return call(operatorType.getOperator(), functionHandle, left.getType(), left, right); } public CallExpression comparison(OperatorType operatorType, RowExpression left, RowExpression right) { - FunctionHandle functionHandle = new FunctionResolution(metadata.getFunctionManager()).comparisonFunction(operatorType, left.getType(), right.getType()); + FunctionHandle functionHandle = new FunctionResolution(metadata.getFunctionAndTypeManager()).comparisonFunction(operatorType, left.getType(), right.getType()); return call(operatorType.getOperator(), functionHandle, left.getType(), left, right); } @@ -339,7 +339,7 @@ private AggregationBuilder addAggregation(VariableReferenceExpression output, Ex { checkArgument(expression instanceof FunctionCall); FunctionCall call = (FunctionCall) expression; - FunctionHandle functionHandle = metadata.getFunctionManager().resolveFunction( + FunctionHandle functionHandle = metadata.getFunctionAndTypeManager().resolveFunction( session.getTransactionId(), qualifyFunctionName(call.getName()), TypeSignatureProvider.fromTypes(inputTypes)); @@ -347,7 +347,7 @@ private AggregationBuilder addAggregation(VariableReferenceExpression output, Ex new CallExpression( call.getName().getSuffix(), functionHandle, - metadata.getType(metadata.getFunctionManager().getFunctionMetadata(functionHandle).getReturnType()), + metadata.getType(metadata.getFunctionAndTypeManager().getFunctionMetadata(functionHandle).getReturnType()), call.getArguments().stream().map(OriginalExpressionUtils::castToRowExpression).collect(toImmutableList())), call.getFilter().map(OriginalExpressionUtils::castToRowExpression), call.getOrderBy().map(orderBy -> toOrderingScheme(orderBy, types)), @@ -890,7 +890,7 @@ public RowExpression rowExpression(String sql) expression, expressionTypes, ImmutableMap.of(), - metadata.getFunctionManager(), + metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), session); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleAssert.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleAssert.java index 4c2f4ea1cf34..f303ce850630 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleAssert.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleAssert.java @@ -134,7 +134,7 @@ public void doesNotFire() fail(String.format( "Expected %s to not fire for:\n%s", rule.getClass().getName(), - inTransaction(session -> textLogicalPlan(plan, ruleApplication.types, metadata.getFunctionManager(), StatsAndCosts.empty(), session, 2)))); + inTransaction(session -> textLogicalPlan(plan, ruleApplication.types, metadata.getFunctionAndTypeManager(), StatsAndCosts.empty(), session, 2)))); } } @@ -206,7 +206,7 @@ private String formatPlan(PlanNode plan, TypeProvider types) { StatsProvider statsProvider = new CachingStatsProvider(statsCalculator, session, types); CostProvider costProvider = new CachingCostProvider(costCalculator, statsProvider, session); - return inTransaction(session -> textLogicalPlan(translateExpressions(plan, types), types, metadata.getFunctionManager(), StatsAndCosts.create(plan, statsProvider, costProvider), session, 2, false)); + return inTransaction(session -> textLogicalPlan(translateExpressions(plan, types), types, metadata.getFunctionAndTypeManager(), StatsAndCosts.create(plan, statsProvider, costProvider), session, 2, false)); } private T inTransaction(Function transactionSessionConsumer) diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestEliminateSorts.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestEliminateSorts.java index f5cf5bf6aabe..2ef65f2ae80b 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestEliminateSorts.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestEliminateSorts.java @@ -95,7 +95,7 @@ public void assertUnitPlan(@Language("SQL") String sql, PlanMatchPattern pattern getQueryRunner().getCostCalculator(), new TranslateExpressions(getMetadata(), new SqlParser()).rules()), new AddExchanges(getQueryRunner().getMetadata(), new SqlParser()), - new UnaliasSymbolReferences(getMetadata().getFunctionManager()), + new UnaliasSymbolReferences(getMetadata().getFunctionAndTypeManager()), new PruneUnreferencedOutputs(), new IterativeOptimizer( new RuleStatsRecorder(), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestMergeWindows.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestMergeWindows.java index e6c5e8d351d8..12f14491c101 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestMergeWindows.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestMergeWindows.java @@ -555,7 +555,7 @@ public void testNotMergeDifferentNullOrdering() private void assertUnitPlan(@Language("SQL") String sql, PlanMatchPattern pattern) { List optimizers = ImmutableList.of( - new UnaliasSymbolReferences(getMetadata().getFunctionManager()), + new UnaliasSymbolReferences(getMetadata().getFunctionAndTypeManager()), new IterativeOptimizer( new RuleStatsRecorder(), getQueryRunner().getStatsCalculator(), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestOptimizeMixedDistinctAggregations.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestOptimizeMixedDistinctAggregations.java index 22b5508118ce..8f8572ce958c 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestOptimizeMixedDistinctAggregations.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestOptimizeMixedDistinctAggregations.java @@ -115,7 +115,7 @@ public void testNestedType() private void assertUnitPlan(String sql, PlanMatchPattern pattern) { List optimizers = ImmutableList.of( - new UnaliasSymbolReferences(getMetadata().getFunctionManager()), + new UnaliasSymbolReferences(getMetadata().getFunctionAndTypeManager()), new IterativeOptimizer( new RuleStatsRecorder(), getQueryRunner().getStatsCalculator(), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestPruneUnreferencedOutputs.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestPruneUnreferencedOutputs.java index 9203d2f556f6..a41eae7d154f 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestPruneUnreferencedOutputs.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestPruneUnreferencedOutputs.java @@ -49,7 +49,7 @@ public class TestPruneUnreferencedOutputs @Test public void windowNodePruning() { - FunctionHandle functionHandle = createTestMetadataManager().getFunctionManager().lookupFunction("rank", ImmutableList.of()); + FunctionHandle functionHandle = createTestMetadataManager().getFunctionAndTypeManager().lookupFunction("rank", ImmutableList.of()); CallExpression call = call("rank", functionHandle, BIGINT); WindowNode.Frame frame = new WindowNode.Frame( RANGE, diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestRemoveUnsupportedDynamicFilters.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestRemoveUnsupportedDynamicFilters.java index 55542d083e4a..cb13b9dcf074 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestRemoveUnsupportedDynamicFilters.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestRemoveUnsupportedDynamicFilters.java @@ -74,9 +74,9 @@ public void setup() { metadata = getQueryRunner().getMetadata(); logicalRowExpressions = new LogicalRowExpressions( - new RowExpressionDeterminismEvaluator(metadata.getFunctionManager()), - new FunctionResolution(metadata.getFunctionManager()), - metadata.getFunctionManager()); + new RowExpressionDeterminismEvaluator(metadata.getFunctionAndTypeManager()), + new FunctionResolution(metadata.getFunctionAndTypeManager()), + metadata.getFunctionAndTypeManager()); builder = new PlanBuilder(getQueryRunner().getDefaultSession(), new PlanNodeIdAllocator(), metadata); ConnectorId connectorId = getCurrentConnectorId(); TableHandle lineitemTableHandle = new TableHandle( @@ -124,10 +124,10 @@ public void testDynamicFilterConsumedOnBuildSide() PlanNode root = builder.join( INNER, builder.filter( - createDynamicFilterExpression("DF", ordersOrderKeyVariable, metadata.getFunctionManager()), + createDynamicFilterExpression("DF", ordersOrderKeyVariable, metadata.getFunctionAndTypeManager()), ordersTableScanNode), builder.filter( - createDynamicFilterExpression("DF", ordersOrderKeyVariable, metadata.getFunctionManager()), + createDynamicFilterExpression("DF", ordersOrderKeyVariable, metadata.getFunctionAndTypeManager()), lineitemTableScanNode), ImmutableList.of(new JoinNode.EquiJoinClause(ordersOrderKeyVariable, lineitemOrderKeyVariable)), ImmutableList.of(ordersOrderKeyVariable), @@ -154,7 +154,7 @@ public void testUnmatchedDynamicFilter() builder.filter( logicalRowExpressions.combineConjuncts( builder.rowExpression("LINEITEM_OK > 0"), - createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionManager())), + createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionAndTypeManager())), lineitemTableScanNode), ImmutableList.of(new JoinNode.EquiJoinClause(ordersOrderKeyVariable, lineitemOrderKeyVariable)), ImmutableList.of(), @@ -185,10 +185,10 @@ public void testNestedDynamicFilterDisjunctionRewrite() logicalRowExpressions.combineConjuncts( logicalRowExpressions.combineDisjuncts( builder.rowExpression("LINEITEM_OK IS NULL"), - createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionManager())), + createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionAndTypeManager())), logicalRowExpressions.combineDisjuncts( builder.rowExpression("LINEITEM_OK IS NOT NULL"), - createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionManager()))), + createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionAndTypeManager()))), lineitemTableScanNode), ImmutableList.of(new JoinNode.EquiJoinClause(ordersOrderKeyVariable, lineitemOrderKeyVariable)), ImmutableList.of(ordersOrderKeyVariable), @@ -216,10 +216,10 @@ public void testNestedDynamicFilterConjunctionRewrite() logicalRowExpressions.combineDisjuncts( logicalRowExpressions.combineConjuncts( builder.rowExpression("LINEITEM_OK IS NULL"), - createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionManager())), + createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionAndTypeManager())), logicalRowExpressions.combineConjuncts( builder.rowExpression("LINEITEM_OK IS NOT NULL"), - createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionManager()))), + createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionAndTypeManager()))), lineitemTableScanNode), ImmutableList.of(new JoinNode.EquiJoinClause(ordersOrderKeyVariable, lineitemOrderKeyVariable)), ImmutableList.of(ordersOrderKeyVariable), @@ -245,7 +245,7 @@ PlanNode removeUnsupportedDynamicFilters(PlanNode root) return getQueryRunner().inTransaction(session -> { // metadata.getCatalogHandle() registers the catalog for the transaction session.getCatalog().ifPresent(catalog -> metadata.getCatalogHandle(session, catalog)); - PlanNode rewrittenPlan = new RemoveUnsupportedDynamicFilters(metadata.getFunctionManager()).optimize(root, session, TypeProvider.empty(), new PlanVariableAllocator(), new PlanNodeIdAllocator(), WarningCollector.NOOP); + PlanNode rewrittenPlan = new RemoveUnsupportedDynamicFilters(metadata.getFunctionAndTypeManager()).optimize(root, session, TypeProvider.empty(), new PlanVariableAllocator(), new PlanNodeIdAllocator(), WarningCollector.NOOP); new DynamicFiltersChecker().validate(rewrittenPlan, session, metadata, new SqlParser(), TypeProvider.empty(), WarningCollector.NOOP); return rewrittenPlan; }); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestReorderWindows.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestReorderWindows.java index 6dc84bc6e481..a47fe5ad36c6 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestReorderWindows.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestReorderWindows.java @@ -323,7 +323,7 @@ public void testReorderBDAC() private void assertUnitPlan(@Language("SQL") String sql, PlanMatchPattern pattern) { List optimizers = ImmutableList.of( - new UnaliasSymbolReferences(getMetadata().getFunctionManager()), + new UnaliasSymbolReferences(getMetadata().getFunctionAndTypeManager()), new IterativeOptimizer( new RuleStatsRecorder(), getQueryRunner().getStatsCalculator(), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestSetFlatteningOptimizer.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestSetFlatteningOptimizer.java index 1a32f93c84fd..c76d5613332b 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestSetFlatteningOptimizer.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestSetFlatteningOptimizer.java @@ -126,7 +126,7 @@ public void testDoesNotFlattenDifferentSetOperations() public void assertPlan(String sql, PlanMatchPattern pattern) { List optimizers = ImmutableList.of( - new UnaliasSymbolReferences(getMetadata().getFunctionManager()), + new UnaliasSymbolReferences(getMetadata().getFunctionAndTypeManager()), new PruneUnreferencedOutputs(), new IterativeOptimizer( new RuleStatsRecorder(), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/plan/TestWindowNode.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/plan/TestWindowNode.java index b306c69ad2a0..f2d6d90b1e36 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/plan/TestWindowNode.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/plan/TestWindowNode.java @@ -99,7 +99,7 @@ public void setUp() public void testSerializationRoundtrip() { VariableReferenceExpression windowVariable = variableAllocator.newVariable("sum", BIGINT); - FunctionHandle functionHandle = createTestMetadataManager().getFunctionManager().lookupFunction("sum", fromTypes(BIGINT)); + FunctionHandle functionHandle = createTestMetadataManager().getFunctionAndTypeManager().lookupFunction("sum", fromTypes(BIGINT)); WindowNode.Frame frame = new WindowNode.Frame( RANGE, UNBOUNDED_PRECEDING, diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestDynamicFiltersChecker.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestDynamicFiltersChecker.java index 91d93559f07d..b0e6a61f6b4f 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestDynamicFiltersChecker.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestDynamicFiltersChecker.java @@ -60,9 +60,9 @@ public void setup() { metadata = getQueryRunner().getMetadata(); logicalRowExpressions = new LogicalRowExpressions( - new RowExpressionDeterminismEvaluator(metadata.getFunctionManager()), - new FunctionResolution(metadata.getFunctionManager()), - metadata.getFunctionManager()); + new RowExpressionDeterminismEvaluator(metadata.getFunctionAndTypeManager()), + new FunctionResolution(metadata.getFunctionAndTypeManager()), + metadata.getFunctionAndTypeManager()); builder = new PlanBuilder(getQueryRunner().getDefaultSession(), new PlanNodeIdAllocator(), metadata); ConnectorId connectorId = getCurrentConnectorId(); TableHandle lineitemTableHandle = new TableHandle( @@ -105,10 +105,10 @@ public void testDynamicFilterConsumedOnBuildSide() PlanNode root = builder.join( INNER, builder.filter( - createDynamicFilterExpression("DF", ordersOrderKeyVariable, metadata.getFunctionManager()), + createDynamicFilterExpression("DF", ordersOrderKeyVariable, metadata.getFunctionAndTypeManager()), ordersTableScanNode), builder.filter( - createDynamicFilterExpression("DF", ordersOrderKeyVariable, metadata.getFunctionManager()), + createDynamicFilterExpression("DF", ordersOrderKeyVariable, metadata.getFunctionAndTypeManager()), lineitemTableScanNode), ImmutableList.of(new JoinNode.EquiJoinClause(ordersOrderKeyVariable, lineitemOrderKeyVariable)), ImmutableList.of(ordersOrderKeyVariable), @@ -131,7 +131,7 @@ public void testUnmatchedDynamicFilter() builder.filter( logicalRowExpressions.combineConjuncts( builder.rowExpression("LINEITEM_OK > 0"), - createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionManager())), + createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionAndTypeManager())), lineitemTableScanNode), ImmutableList.of(new JoinNode.EquiJoinClause(ordersOrderKeyVariable, lineitemOrderKeyVariable)), ImmutableList.of(ordersOrderKeyVariable), @@ -155,10 +155,10 @@ public void testUnmatchedNestedDynamicFilter() logicalRowExpressions.combineConjuncts( logicalRowExpressions.combineDisjuncts( builder.rowExpression("LINEITEM_OK IS NULL"), - createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionManager())), + createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionAndTypeManager())), logicalRowExpressions.combineDisjuncts( builder.rowExpression("LINEITEM_OK IS NOT NULL"), - createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionManager()))), + createDynamicFilterExpression("DF", lineitemOrderKeyVariable, metadata.getFunctionAndTypeManager()))), lineitemTableScanNode), ImmutableList.of(new JoinNode.EquiJoinClause(ordersOrderKeyVariable, lineitemOrderKeyVariable)), ImmutableList.of(ordersOrderKeyVariable), diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestVerifyNoOriginalExpression.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestVerifyNoOriginalExpression.java index 642f7bece26e..61bcb8cbb8fe 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestVerifyNoOriginalExpression.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestVerifyNoOriginalExpression.java @@ -83,7 +83,7 @@ public void setup() valuesNode = builder.values(); comparisonCallExpression = new CallExpression( "LESS_THAN", - metadata.getFunctionManager().resolveOperator(LESS_THAN, fromTypes(BIGINT, BIGINT)), + metadata.getFunctionAndTypeManager().resolveOperator(LESS_THAN, fromTypes(BIGINT, BIGINT)), BooleanType.BOOLEAN, ImmutableList.of(VARIABLE_REFERENCE_EXPRESSION, VARIABLE_REFERENCE_EXPRESSION)); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/relational/TestDeterminismEvaluator.java b/presto-main/src/test/java/com/facebook/presto/sql/relational/TestDeterminismEvaluator.java index 1253944aaf37..fc618f6502f9 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/relational/TestDeterminismEvaluator.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/relational/TestDeterminismEvaluator.java @@ -13,7 +13,7 @@ */ package com.facebook.presto.sql.relational; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.relation.CallExpression; import com.facebook.presto.spi.relation.InputReferenceExpression; @@ -36,18 +36,18 @@ public class TestDeterminismEvaluator @Test public void testDeterminismEvaluator() { - FunctionManager functionManager = createTestMetadataManager().getFunctionManager(); - RowExpressionDeterminismEvaluator determinismEvaluator = new RowExpressionDeterminismEvaluator(functionManager); + FunctionAndTypeManager functionAndTypeManager = createTestMetadataManager().getFunctionAndTypeManager(); + RowExpressionDeterminismEvaluator determinismEvaluator = new RowExpressionDeterminismEvaluator(functionAndTypeManager); CallExpression random = new CallExpression( "random", - functionManager.lookupFunction("random", fromTypes(BIGINT)), + functionAndTypeManager.lookupFunction("random", fromTypes(BIGINT)), BIGINT, singletonList(constant(10L, BIGINT))); assertFalse(determinismEvaluator.isDeterministic(random)); InputReferenceExpression col0 = field(0, BIGINT); - FunctionHandle lessThan = functionManager.resolveOperator(LESS_THAN, fromTypes(BIGINT, BIGINT)); + FunctionHandle lessThan = functionAndTypeManager.resolveOperator(LESS_THAN, fromTypes(BIGINT, BIGINT)); CallExpression lessThanExpression = new CallExpression(LESS_THAN.name(), lessThan, BOOLEAN, ImmutableList.of(col0, constant(10L, BIGINT))); assertTrue(determinismEvaluator.isDeterministic(lessThanExpression)); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/relational/TestFunctionResolution.java b/presto-main/src/test/java/com/facebook/presto/sql/relational/TestFunctionResolution.java index 06a5784e4941..f8678636d158 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/relational/TestFunctionResolution.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/relational/TestFunctionResolution.java @@ -16,7 +16,7 @@ import com.facebook.presto.block.BlockEncodingManager; import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.TypeManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.StandardFunctionResolution; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.type.TypeRegistry; @@ -38,8 +38,8 @@ public class TestFunctionResolution public void setup() { TypeManager typeManager = new TypeRegistry(); - FunctionManager functionManager = new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); - functionResolution = new FunctionResolution(functionManager); + FunctionAndTypeManager functionAndTypeManager = new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + functionResolution = new FunctionResolution(functionAndTypeManager); } @Test diff --git a/presto-main/src/test/java/com/facebook/presto/sql/relational/TestLogicalRowExpressions.java b/presto-main/src/test/java/com/facebook/presto/sql/relational/TestLogicalRowExpressions.java index 12ea31520899..fe7eeba671c2 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/relational/TestLogicalRowExpressions.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/relational/TestLogicalRowExpressions.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.function.OperatorType; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.expressions.LogicalRowExpressions; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.relation.CallExpression; import com.facebook.presto.spi.relation.RowExpression; import com.facebook.presto.spi.relation.SpecialFormExpression; @@ -53,7 +53,7 @@ public class TestLogicalRowExpressions { - private FunctionManager functionManager; + private FunctionAndTypeManager functionAndTypeManager; private LogicalRowExpressions logicalRowExpressions; private static final RowExpression a = name("a"); private static final RowExpression b = name("b"); @@ -68,8 +68,8 @@ public class TestLogicalRowExpressions public void setup() { TypeManager typeManager = new TypeRegistry(); - functionManager = new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); - logicalRowExpressions = new LogicalRowExpressions(new RowExpressionDeterminismEvaluator(functionManager), new FunctionResolution(functionManager), functionManager); + functionAndTypeManager = new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + logicalRowExpressions = new LogicalRowExpressions(new RowExpressionDeterminismEvaluator(functionAndTypeManager), new FunctionResolution(functionAndTypeManager), functionAndTypeManager); } @Test @@ -119,8 +119,8 @@ public void testOr() @Test public void testDeterminism() { - RowExpression nondeterministic = call("random", functionManager.lookupFunction("random", fromTypes()), DOUBLE); - RowExpression deterministic = call("length", functionManager.lookupFunction("length", fromTypes(VARCHAR)), INTEGER); + RowExpression nondeterministic = call("random", functionAndTypeManager.lookupFunction("random", fromTypes()), DOUBLE); + RowExpression deterministic = call("length", functionAndTypeManager.lookupFunction("length", fromTypes(VARCHAR)), INTEGER); RowExpression expression = and(and(a, or(b, nondeterministic)), deterministic); @@ -175,7 +175,7 @@ public void testEliminateConstant() @Test public void testEliminateDuplicate() { - RowExpression nd = call("random", functionManager.lookupFunction("random", fromTypes()), DOUBLE); + RowExpression nd = call("random", functionAndTypeManager.lookupFunction("random", fromTypes()), DOUBLE); assertEquals( logicalRowExpressions.convertToConjunctiveNormalForm(or(and(TRUE_CONSTANT, a), and(b, b))), @@ -498,7 +498,7 @@ private RowExpression compare(RowExpression left, OperatorType operator, RowExpr { return call( operator.getOperator(), - new FunctionResolution(functionManager).comparisonFunction(operator, left.getType(), right.getType()), + new FunctionResolution(functionAndTypeManager).comparisonFunction(operator, left.getType(), right.getType()), BOOLEAN, left, right); @@ -521,6 +521,6 @@ private RowExpression or(RowExpression left, RowExpression right) private RowExpression not(RowExpression expression) { - return new CallExpression("not", new FunctionResolution(functionManager).notFunction(), BOOLEAN, ImmutableList.of(expression)); + return new CallExpression("not", new FunctionResolution(functionAndTypeManager).notFunction(), BOOLEAN, ImmutableList.of(expression)); } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/relational/TestRowExpressionOptimizer.java b/presto-main/src/test/java/com/facebook/presto/sql/relational/TestRowExpressionOptimizer.java index 7f0edc117377..4d3601a2e0b2 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/relational/TestRowExpressionOptimizer.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/relational/TestRowExpressionOptimizer.java @@ -18,7 +18,7 @@ import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.RowType; import com.facebook.presto.common.type.TypeManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.relation.CallExpression; @@ -59,14 +59,14 @@ public class TestRowExpressionOptimizer { - private FunctionManager functionManager; + private FunctionAndTypeManager functionAndTypeManager; private RowExpressionOptimizer optimizer; @BeforeClass public void setUp() { TypeManager typeManager = new TypeRegistry(); - functionManager = new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + functionAndTypeManager = new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); optimizer = new RowExpressionOptimizer(MetadataManager.createTestMetadataManager()); } @@ -81,7 +81,7 @@ public void testPossibleExponentialOptimizationTime() { RowExpression expression = constant(1L, BIGINT); for (int i = 0; i < 100; i++) { - FunctionHandle functionHandle = functionManager.resolveOperator(ADD, fromTypes(BIGINT, BIGINT)); + FunctionHandle functionHandle = functionAndTypeManager.resolveOperator(ADD, fromTypes(BIGINT, BIGINT)); expression = new CallExpression(ADD.name(), functionHandle, BIGINT, ImmutableList.of(expression, constant(1L, BIGINT))); } optimize(expression); @@ -94,7 +94,7 @@ public void testIfConstantOptimization() assertEquals(optimize(ifExpression(constant(false, BOOLEAN), 1L, 2L)), constant(2L, BIGINT)); assertEquals(optimize(ifExpression(constant(null, BOOLEAN), 1L, 2L)), constant(2L, BIGINT)); - FunctionHandle bigintEquals = functionManager.resolveOperator(EQUAL, fromTypes(BIGINT, BIGINT)); + FunctionHandle bigintEquals = functionAndTypeManager.resolveOperator(EQUAL, fromTypes(BIGINT, BIGINT)); RowExpression condition = new CallExpression(EQUAL.name(), bigintEquals, BOOLEAN, ImmutableList.of(constant(3L, BIGINT), constant(3L, BIGINT))); assertEquals(optimize(ifExpression(condition, 1L, 2L)), constant(1L, BIGINT)); } @@ -102,10 +102,10 @@ public void testIfConstantOptimization() @Test public void testCastWithJsonParseOptimization() { - FunctionHandle jsonParseFunctionHandle = functionManager.lookupFunction("json_parse", fromTypes(VARCHAR)); + FunctionHandle jsonParseFunctionHandle = functionAndTypeManager.lookupFunction("json_parse", fromTypes(VARCHAR)); // constant - FunctionHandle jsonCastFunctionHandle = functionManager.lookupCast(CAST, JSON.getTypeSignature(), parseTypeSignature("array(integer)")); + FunctionHandle jsonCastFunctionHandle = functionAndTypeManager.lookupCast(CAST, JSON.getTypeSignature(), parseTypeSignature("array(integer)")); RowExpression jsonCastExpression = new CallExpression(CAST.name(), jsonCastFunctionHandle, new ArrayType(INTEGER), ImmutableList.of(call("json_parse", jsonParseFunctionHandle, JSON, constant(utf8Slice("[1, 2]"), VARCHAR)))); RowExpression resultExpression = optimize(jsonCastExpression); assertInstanceOf(resultExpression, ConstantExpression.class); @@ -114,28 +114,28 @@ public void testCastWithJsonParseOptimization() assertEquals(toValues(INTEGER, (IntArrayBlock) resultValue), ImmutableList.of(1, 2)); // varchar to array - jsonCastFunctionHandle = functionManager.lookupCast(CAST, JSON.getTypeSignature(), parseTypeSignature("array(varchar)")); + jsonCastFunctionHandle = functionAndTypeManager.lookupCast(CAST, JSON.getTypeSignature(), parseTypeSignature("array(varchar)")); jsonCastExpression = call(CAST.name(), jsonCastFunctionHandle, new ArrayType(VARCHAR), ImmutableList.of(call("json_parse", jsonParseFunctionHandle, JSON, field(1, VARCHAR)))); resultExpression = optimize(jsonCastExpression); assertEquals( resultExpression, - call(JSON_TO_ARRAY_CAST.name(), functionManager.lookupCast(JSON_TO_ARRAY_CAST, VARCHAR.getTypeSignature(), parseTypeSignature("array(varchar)")), new ArrayType(VARCHAR), field(1, VARCHAR))); + call(JSON_TO_ARRAY_CAST.name(), functionAndTypeManager.lookupCast(JSON_TO_ARRAY_CAST, VARCHAR.getTypeSignature(), parseTypeSignature("array(varchar)")), new ArrayType(VARCHAR), field(1, VARCHAR))); // varchar to map - jsonCastFunctionHandle = functionManager.lookupCast(CAST, JSON.getTypeSignature(), parseTypeSignature("map(integer,varchar)")); + jsonCastFunctionHandle = functionAndTypeManager.lookupCast(CAST, JSON.getTypeSignature(), parseTypeSignature("map(integer,varchar)")); jsonCastExpression = call(CAST.name(), jsonCastFunctionHandle, mapType(INTEGER, VARCHAR), ImmutableList.of(call("json_parse", jsonParseFunctionHandle, JSON, field(1, VARCHAR)))); resultExpression = optimize(jsonCastExpression); assertEquals( resultExpression, - call(JSON_TO_MAP_CAST.name(), functionManager.lookupCast(JSON_TO_MAP_CAST, VARCHAR.getTypeSignature(), parseTypeSignature("map(integer, varchar)")), mapType(INTEGER, VARCHAR), field(1, VARCHAR))); + call(JSON_TO_MAP_CAST.name(), functionAndTypeManager.lookupCast(JSON_TO_MAP_CAST, VARCHAR.getTypeSignature(), parseTypeSignature("map(integer, varchar)")), mapType(INTEGER, VARCHAR), field(1, VARCHAR))); // varchar to row - jsonCastFunctionHandle = functionManager.lookupCast(CAST, JSON.getTypeSignature(), parseTypeSignature("row(varchar,bigint)")); + jsonCastFunctionHandle = functionAndTypeManager.lookupCast(CAST, JSON.getTypeSignature(), parseTypeSignature("row(varchar,bigint)")); jsonCastExpression = call(CAST.name(), jsonCastFunctionHandle, RowType.anonymous(ImmutableList.of(VARCHAR, BIGINT)), ImmutableList.of(call("json_parse", jsonParseFunctionHandle, JSON, field(1, VARCHAR)))); resultExpression = optimize(jsonCastExpression); assertEquals( resultExpression, - call(JSON_TO_ROW_CAST.name(), functionManager.lookupCast(JSON_TO_ROW_CAST, VARCHAR.getTypeSignature(), parseTypeSignature("row(varchar,bigint)")), RowType.anonymous(ImmutableList.of(VARCHAR, BIGINT)), field(1, VARCHAR))); + call(JSON_TO_ROW_CAST.name(), functionAndTypeManager.lookupCast(JSON_TO_ROW_CAST, VARCHAR.getTypeSignature(), parseTypeSignature("row(varchar,bigint)")), RowType.anonymous(ImmutableList.of(VARCHAR, BIGINT)), field(1, VARCHAR))); } private static RowExpression ifExpression(RowExpression condition, long trueValue, long falseValue) diff --git a/presto-main/src/test/java/com/facebook/presto/sql/relational/TestRowExpressionTranslator.java b/presto-main/src/test/java/com/facebook/presto/sql/relational/TestRowExpressionTranslator.java index 5af3710adc68..e771c9f87e87 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/relational/TestRowExpressionTranslator.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/relational/TestRowExpressionTranslator.java @@ -19,7 +19,7 @@ import com.facebook.presto.expressions.translator.RowExpressionTranslator; import com.facebook.presto.expressions.translator.RowExpressionTreeTranslator; import com.facebook.presto.expressions.translator.TranslatedExpression; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.spi.ColumnHandle; @@ -61,12 +61,12 @@ public class TestRowExpressionTranslator { private static final Metadata METADATA = MetadataManager.createTestMetadataManager(); - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final TestingRowExpressionTranslator sqlToRowExpressionTranslator; public TestRowExpressionTranslator() { - this.functionManager = METADATA.getFunctionManager(); + this.functionAndTypeManager = METADATA.getFunctionAndTypeManager(); this.sqlToRowExpressionTranslator = new TestingRowExpressionTranslator(METADATA); } @@ -79,7 +79,7 @@ public void testEndToEndFunctionTranslation() TranslatedExpression translatedExpression = translateWith( callExpression, - new TestFunctionTranslator(functionManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), + new TestFunctionTranslator(functionAndTypeManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), emptyMap()); assertTrue(translatedExpression.getTranslated().isPresent()); assertEquals(translatedExpression.getTranslated().get(), "LNof(1 BITWISE_AND col1)"); @@ -95,7 +95,7 @@ public void testEndToEndSpecialFormTranslation() TranslatedExpression translatedExpression = translateWith( specialForm, - new TestFunctionTranslator(functionManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), + new TestFunctionTranslator(functionAndTypeManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), emptyMap()); assertTrue(translatedExpression.getTranslated().isPresent()); assertEquals(translatedExpression.getTranslated().get(), "col1 TEST_AND col2"); @@ -111,7 +111,7 @@ public void testMissingFunctionTranslator() TranslatedExpression translatedExpression = translateWith( specialForm, - new TestFunctionTranslator(functionManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), + new TestFunctionTranslator(functionAndTypeManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), emptyMap()); assertFalse(translatedExpression.getTranslated().isPresent()); } @@ -126,7 +126,7 @@ public void testIncorrectFunctionSignatureInDefinition() TranslatedExpression translatedExpression = translateWith( specialForm, - new TestFunctionTranslator(functionManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), + new TestFunctionTranslator(functionAndTypeManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), emptyMap()); assertFalse(translatedExpression.getTranslated().isPresent()); } @@ -139,7 +139,7 @@ public void testHiddenFunctionNot() TranslatedExpression translatedExpression = translateWith( specialForm, - new TestFunctionTranslator(functionManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), + new TestFunctionTranslator(functionAndTypeManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), emptyMap()); assertTrue(translatedExpression.getTranslated().isPresent()); assertEquals(translatedExpression.getTranslated().get(), "NOT_2 true"); @@ -154,7 +154,7 @@ public void testBasicOperator() TranslatedExpression translatedExpression = translateWith( specialForm, - new TestFunctionTranslator(functionManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), + new TestFunctionTranslator(functionAndTypeManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), emptyMap()); assertTrue(translatedExpression.getTranslated().isPresent()); assertEquals(translatedExpression.getTranslated().get(), "col1 -|- col2"); @@ -169,7 +169,7 @@ public void testLessThanOperator() TranslatedExpression translatedExpression = translateWith( specialForm, - new TestFunctionTranslator(functionManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), + new TestFunctionTranslator(functionAndTypeManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), emptyMap()); assertTrue(translatedExpression.getTranslated().isPresent()); assertEquals(translatedExpression.getTranslated().get(), "col1 LT col2"); @@ -184,7 +184,7 @@ public void testUntranslatableSpecialForm() TranslatedExpression translatedExpression = translateWith( specialForm, - new TestFunctionTranslator(functionManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), + new TestFunctionTranslator(functionAndTypeManager, buildFunctionTranslator(ImmutableSet.of(TestFunctions.class))), emptyMap()); assertFalse(translatedExpression.getTranslated().isPresent()); } @@ -192,13 +192,13 @@ public void testUntranslatableSpecialForm() private class TestFunctionTranslator extends RowExpressionTranslator> { - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final FunctionTranslator functionTranslator; - TestFunctionTranslator(FunctionManager functionManager, FunctionTranslator functionTranslator) + TestFunctionTranslator(FunctionAndTypeManager functionAndTypeManager, FunctionTranslator functionTranslator) { this.functionTranslator = requireNonNull(functionTranslator); - this.functionManager = requireNonNull(functionManager); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager); } @Override @@ -213,7 +213,7 @@ public TranslatedExpression translateCall(CallExpression callExpression, List> translatedExpressions = callExpression.getArguments().stream() .map(expression -> rowExpressionTreeTranslator.rewrite(expression, context)) .collect(Collectors.toList()); - FunctionMetadata functionMetadata = functionManager.getFunctionMetadata(callExpression.getFunctionHandle()); + FunctionMetadata functionMetadata = functionAndTypeManager.getFunctionMetadata(callExpression.getFunctionHandle()); try { return functionTranslator.translate(functionMetadata, callExpression, translatedExpressions); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/relational/TestSubExpressions.java b/presto-main/src/test/java/com/facebook/presto/sql/relational/TestSubExpressions.java index 8b4e188bc742..34c15a796e69 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/relational/TestSubExpressions.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/relational/TestSubExpressions.java @@ -15,7 +15,7 @@ import com.facebook.presto.common.function.OperatorType; import com.facebook.presto.common.type.FunctionType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.relation.LambdaDefinitionExpression; import com.facebook.presto.spi.relation.RowExpression; @@ -38,7 +38,7 @@ public class TestSubExpressions { - private static final FunctionManager FUNCTION_MANAGER = createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_MANAGER = createTestMetadataManager().getFunctionAndTypeManager(); @Test void testExtract() diff --git a/presto-main/src/test/java/com/facebook/presto/type/BenchmarkDecimalOperators.java b/presto-main/src/test/java/com/facebook/presto/type/BenchmarkDecimalOperators.java index 17b4586b734e..d8a82c623c33 100644 --- a/presto-main/src/test/java/com/facebook/presto/type/BenchmarkDecimalOperators.java +++ b/presto-main/src/test/java/com/facebook/presto/type/BenchmarkDecimalOperators.java @@ -609,7 +609,7 @@ private RowExpression rowExpression(String value) Expression expression = createExpression(value, metadata, TypeProvider.copyOf(symbolTypes)); Map, Type> expressionTypes = getExpressionTypes(TEST_SESSION, metadata, SQL_PARSER, TypeProvider.copyOf(symbolTypes), expression, emptyList(), WarningCollector.NOOP); - RowExpression rowExpression = SqlToRowExpressionTranslator.translate(expression, expressionTypes, sourceLayout, metadata.getFunctionManager(), metadata.getTypeManager(), TEST_SESSION); + RowExpression rowExpression = SqlToRowExpressionTranslator.translate(expression, expressionTypes, sourceLayout, metadata.getFunctionAndTypeManager(), metadata.getTypeManager(), TEST_SESSION); RowExpressionOptimizer optimizer = new RowExpressionOptimizer(metadata); return optimizer.optimize(rowExpression, OPTIMIZED, TEST_SESSION.toConnectorSession()); } diff --git a/presto-main/src/test/java/com/facebook/presto/type/TestTypeRegistry.java b/presto-main/src/test/java/com/facebook/presto/type/TestTypeRegistry.java index 6d3b20f1b3e6..13f4d96cb05a 100644 --- a/presto-main/src/test/java/com/facebook/presto/type/TestTypeRegistry.java +++ b/presto-main/src/test/java/com/facebook/presto/type/TestTypeRegistry.java @@ -17,7 +17,7 @@ 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.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.OperatorNotFoundException; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.google.common.collect.ImmutableSet; @@ -69,7 +69,7 @@ public class TestTypeRegistry { private final TypeManager typeRegistry = new TypeRegistry(); - private final FunctionManager functionManager = new FunctionManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); + private final FunctionAndTypeManager functionAndTypeManager = new FunctionAndTypeManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); @Test public void testNonexistentType() @@ -253,7 +253,7 @@ public void testCastOperatorsExistForCoercions() for (Type resultType : types) { if (typeRegistry.canCoerce(sourceType, resultType) && sourceType != UNKNOWN && resultType != UNKNOWN) { try { - functionManager.lookupCast(CAST, sourceType.getTypeSignature(), resultType.getTypeSignature()); + functionAndTypeManager.lookupCast(CAST, sourceType.getTypeSignature(), resultType.getTypeSignature()); } catch (OperatorNotFoundException e) { fail(format("'%s' -> '%s' coercion exists but there is no cast operator", sourceType, resultType)); @@ -268,16 +268,16 @@ public void testOperatorsImplemented() { for (Type type : typeRegistry.getTypes()) { if (type.isComparable()) { - functionManager.resolveOperator(EQUAL, fromTypes(type, type)); - functionManager.resolveOperator(NOT_EQUAL, fromTypes(type, type)); - functionManager.resolveOperator(IS_DISTINCT_FROM, fromTypes(type, type)); - functionManager.resolveOperator(HASH_CODE, fromTypes(type)); + functionAndTypeManager.resolveOperator(EQUAL, fromTypes(type, type)); + functionAndTypeManager.resolveOperator(NOT_EQUAL, fromTypes(type, type)); + functionAndTypeManager.resolveOperator(IS_DISTINCT_FROM, fromTypes(type, type)); + functionAndTypeManager.resolveOperator(HASH_CODE, fromTypes(type)); } if (type.isOrderable()) { - functionManager.resolveOperator(LESS_THAN, fromTypes(type, type)); - functionManager.resolveOperator(LESS_THAN_OR_EQUAL, fromTypes(type, type)); - functionManager.resolveOperator(GREATER_THAN_OR_EQUAL, fromTypes(type, type)); - functionManager.resolveOperator(GREATER_THAN, fromTypes(type, type)); + functionAndTypeManager.resolveOperator(LESS_THAN, fromTypes(type, type)); + functionAndTypeManager.resolveOperator(LESS_THAN_OR_EQUAL, fromTypes(type, type)); + functionAndTypeManager.resolveOperator(GREATER_THAN_OR_EQUAL, fromTypes(type, type)); + functionAndTypeManager.resolveOperator(GREATER_THAN, fromTypes(type, type)); } } } diff --git a/presto-main/src/test/java/com/facebook/presto/type/khyperloglog/TestKHyperLogLogAggregationFunction.java b/presto-main/src/test/java/com/facebook/presto/type/khyperloglog/TestKHyperLogLogAggregationFunction.java index 3108142dfe36..99be41b24dc5 100644 --- a/presto-main/src/test/java/com/facebook/presto/type/khyperloglog/TestKHyperLogLogAggregationFunction.java +++ b/presto-main/src/test/java/com/facebook/presto/type/khyperloglog/TestKHyperLogLogAggregationFunction.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.block.Block; import com.facebook.presto.common.type.SqlVarbinary; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; import com.facebook.presto.spi.function.AggregationFunction; @@ -41,7 +41,7 @@ public class TestKHyperLogLogAggregationFunction { - private static final FunctionManager functionManager = MetadataManager.createTestMetadataManager().getFunctionManager(); + private static final FunctionAndTypeManager FUNCTION_AND_TYPE_MANAGER = MetadataManager.createTestMetadataManager().getFunctionAndTypeManager(); private static final String NAME = KHyperLogLogAggregationFunction.class.getAnnotation(AggregationFunction.class).value(); @Test @@ -213,7 +213,7 @@ private Slice stringToSlice(String s) private static InternalAggregationFunction getAggregation(Type... arguments) { - return functionManager.getAggregateFunctionImplementation(functionManager.lookupFunction(NAME, fromTypes(arguments))); + return FUNCTION_AND_TYPE_MANAGER.getAggregateFunctionImplementation(FUNCTION_AND_TYPE_MANAGER.lookupFunction(NAME, fromTypes(arguments))); } private List generateLongs(int size) diff --git a/presto-main/src/test/java/com/facebook/presto/util/StructuralTestUtil.java b/presto-main/src/test/java/com/facebook/presto/util/StructuralTestUtil.java index 27f51b4e6dc9..6340baa98d02 100644 --- a/presto-main/src/test/java/com/facebook/presto/util/StructuralTestUtil.java +++ b/presto-main/src/test/java/com/facebook/presto/util/StructuralTestUtil.java @@ -23,7 +23,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.type.TypeRegistry; import com.google.common.collect.ImmutableList; @@ -41,7 +41,7 @@ public final class StructuralTestUtil static { // associate TYPE_MANAGER with a function manager - new FunctionManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); } private StructuralTestUtil() {} diff --git a/presto-ml/src/test/java/com/facebook/presto/ml/TestEvaluateClassifierPredictions.java b/presto-ml/src/test/java/com/facebook/presto/ml/TestEvaluateClassifierPredictions.java index 09a4f0672785..1d2118c4d86f 100644 --- a/presto-ml/src/test/java/com/facebook/presto/ml/TestEvaluateClassifierPredictions.java +++ b/presto-ml/src/test/java/com/facebook/presto/ml/TestEvaluateClassifierPredictions.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.Page; import com.facebook.presto.common.block.Block; import com.facebook.presto.common.block.BlockBuilder; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.aggregation.Accumulator; import com.facebook.presto.operator.aggregation.InternalAggregationFunction; @@ -37,14 +37,14 @@ public class TestEvaluateClassifierPredictions { private final MetadataManager metadata = MetadataManager.createTestMetadataManager(); - private final FunctionManager functionManager = metadata.getFunctionManager(); + private final FunctionAndTypeManager functionAndTypeManager = metadata.getFunctionAndTypeManager(); @Test public void testEvaluateClassifierPredictions() { metadata.registerBuiltInFunctions(extractFunctions(new MLPlugin().getFunctions())); - InternalAggregationFunction aggregation = functionManager.getAggregateFunctionImplementation( - functionManager.lookupFunction("evaluate_classifier_predictions", fromTypes(BIGINT, BIGINT))); + InternalAggregationFunction aggregation = functionAndTypeManager.getAggregateFunctionImplementation( + functionAndTypeManager.lookupFunction("evaluate_classifier_predictions", fromTypes(BIGINT, BIGINT))); Accumulator accumulator = aggregation.bind(ImmutableList.of(0, 1), Optional.empty()).createAccumulator(); accumulator.addInput(getPage()); BlockBuilder finalOut = accumulator.getFinalType().createBlockBuilder(null, 1); diff --git a/presto-ml/src/test/java/com/facebook/presto/ml/TestLearnAggregations.java b/presto-ml/src/test/java/com/facebook/presto/ml/TestLearnAggregations.java index ee7236bc574b..f753f6a86533 100644 --- a/presto-ml/src/test/java/com/facebook/presto/ml/TestLearnAggregations.java +++ b/presto-ml/src/test/java/com/facebook/presto/ml/TestLearnAggregations.java @@ -24,7 +24,7 @@ import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.common.type.VarcharType; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.ml.type.ClassifierParametricType; import com.facebook.presto.ml.type.ClassifierType; import com.facebook.presto.ml.type.ModelType; @@ -60,7 +60,7 @@ public class TestLearnAggregations typeRegistry.addType(RegressorType.REGRESSOR); // associate typeRegistry with a function manager - new FunctionManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); typeManager = typeRegistry; } diff --git a/presto-orc/src/test/java/com/facebook/presto/orc/OrcTester.java b/presto-orc/src/test/java/com/facebook/presto/orc/OrcTester.java index 7588c91dbead..0cfad0fef64d 100644 --- a/presto-orc/src/test/java/com/facebook/presto/orc/OrcTester.java +++ b/presto-orc/src/test/java/com/facebook/presto/orc/OrcTester.java @@ -38,7 +38,7 @@ import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.common.type.VarbinaryType; import com.facebook.presto.common.type.VarcharType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.orc.TrackingTupleDomainFilter.TestBigintRange; import com.facebook.presto.orc.TrackingTupleDomainFilter.TestDoubleRange; import com.facebook.presto.orc.TupleDomainFilter.BigintRange; @@ -202,7 +202,7 @@ public class OrcTester static { // associate TYPE_MANAGER with a function manager - new FunctionManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); } public enum Format diff --git a/presto-orc/src/test/java/com/facebook/presto/orc/TestMapFlatBatchStreamReader.java b/presto-orc/src/test/java/com/facebook/presto/orc/TestMapFlatBatchStreamReader.java index 6f5f208ed300..62074d8eb705 100644 --- a/presto-orc/src/test/java/com/facebook/presto/orc/TestMapFlatBatchStreamReader.java +++ b/presto-orc/src/test/java/com/facebook/presto/orc/TestMapFlatBatchStreamReader.java @@ -33,7 +33,7 @@ import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.common.type.VarbinaryType; import com.facebook.presto.common.type.VarcharType; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.orc.cache.StorageOrcFileTailSource; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.type.TypeRegistry; @@ -77,7 +77,7 @@ public class TestMapFlatBatchStreamReader static { // associate TYPE_MANAGER with a function manager - new FunctionManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); } private static final Type LIST_TYPE = TYPE_MANAGER.getParameterizedType( diff --git a/presto-orc/src/test/java/com/facebook/presto/orc/TestOrcMapNullKey.java b/presto-orc/src/test/java/com/facebook/presto/orc/TestOrcMapNullKey.java index 5d60f4456d55..fe91b8391a1e 100644 --- a/presto-orc/src/test/java/com/facebook/presto/orc/TestOrcMapNullKey.java +++ b/presto-orc/src/test/java/com/facebook/presto/orc/TestOrcMapNullKey.java @@ -20,7 +20,7 @@ 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.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.orc.metadata.CompressionKind; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.type.TypeRegistry; @@ -59,7 +59,7 @@ public class TestOrcMapNullKey public TestOrcMapNullKey() { // Associate TYPE_MANAGER with a function manager. - new FunctionManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); } @DataProvider(name = "mapNullKeysEnabled") diff --git a/presto-orc/src/test/java/com/facebook/presto/orc/TestOrcReaderMemoryUsage.java b/presto-orc/src/test/java/com/facebook/presto/orc/TestOrcReaderMemoryUsage.java index f7817249a457..0b3c847224c1 100644 --- a/presto-orc/src/test/java/com/facebook/presto/orc/TestOrcReaderMemoryUsage.java +++ b/presto-orc/src/test/java/com/facebook/presto/orc/TestOrcReaderMemoryUsage.java @@ -19,7 +19,7 @@ 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.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.orc.metadata.CompressionKind; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.type.TypeRegistry; @@ -59,7 +59,7 @@ public class TestOrcReaderMemoryUsage public TestOrcReaderMemoryUsage() { // Associate TYPE_MANAGER with a function manager. - new FunctionManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); } @Test diff --git a/presto-orc/src/test/java/com/facebook/presto/orc/TestTupleDomainFilterUtils.java b/presto-orc/src/test/java/com/facebook/presto/orc/TestTupleDomainFilterUtils.java index f8f4147dc893..d21a22a08356 100644 --- a/presto-orc/src/test/java/com/facebook/presto/orc/TestTupleDomainFilterUtils.java +++ b/presto-orc/src/test/java/com/facebook/presto/orc/TestTupleDomainFilterUtils.java @@ -21,7 +21,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.orc.TupleDomainFilter.BigintMultiRange; import com.facebook.presto.orc.TupleDomainFilter.BigintRange; @@ -119,7 +119,7 @@ public class TestTupleDomainFilterUtils static { // associate TYPE_MANAGER with a function manager - new FunctionManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); } private static final Session TEST_SESSION = testSessionBuilder() 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 1fd5becaa920..dad20801a448 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 @@ -20,7 +20,7 @@ import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.metadata.SessionPropertyManager; @@ -89,7 +89,7 @@ public class TestPinotQueryBase { protected static final TypeManager typeManager = new TypeRegistry(); - protected static final FunctionManager functionMetadataManager = new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + protected static final FunctionAndTypeManager functionMetadataManager = new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); protected static final StandardFunctionResolution standardFunctionResolution = new FunctionResolution(functionMetadataManager); protected static ConnectorId pinotConnectorId = new ConnectorId("id"); diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/OrcTestingUtil.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/OrcTestingUtil.java index c448c2377bdf..611e7ab715b4 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/OrcTestingUtil.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/OrcTestingUtil.java @@ -15,7 +15,7 @@ import com.facebook.presto.block.BlockEncodingManager; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.orc.FileOrcDataSource; @@ -93,9 +93,9 @@ public static OrcBatchRecordReader createRecordReader(OrcReader orcReader, Map columnIds, List colum throws IOException { TypeRegistry typeManager = new TypeRegistry(); - new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); return new OrcFileWriter(columnIds, columnTypes, new OutputStreamDataSink(new FileOutputStream(file)), true, true, new OrcWriterStats(), typeManager, ZSTD); } diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcFileRewriter.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcFileRewriter.java index e7d28ec70912..f50334df8e7b 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcFileRewriter.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcFileRewriter.java @@ -25,7 +25,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignature; import com.facebook.presto.common.type.TypeSignatureParameter; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.orc.OrcBatchRecordReader; import com.facebook.presto.orc.OrcDataSource; import com.facebook.presto.orc.OrcReader; @@ -125,7 +125,7 @@ public void testRewrite() { TypeManager typeManager = new TypeRegistry(); // associate typeManager with a function manager - new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); ArrayType arrayType = new ArrayType(BIGINT); ArrayType arrayOfArrayType = new ArrayType(arrayType); @@ -721,7 +721,7 @@ private static FileWriter createFileWriter(List columnIds, List colu private static OrcFileRewriter createFileRewriter() { TypeRegistry typeManager = new TypeRegistry(); - new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); return new OrcFileRewriter( READER_ATTRIBUTES, true, diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestShardWriter.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestShardWriter.java index 90d93c5f5691..5728db94fd47 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestShardWriter.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestShardWriter.java @@ -26,7 +26,7 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignature; import com.facebook.presto.common.type.TypeSignatureParameter; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.orc.OrcBatchRecordReader; import com.facebook.presto.orc.OrcDataSource; import com.facebook.presto.spi.PrestoException; @@ -111,7 +111,7 @@ public void testWriter(boolean useOptimizedOrcWriter) { TypeManager typeManager = new TypeRegistry(); // associate typeManager with a function manager - new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); List columnIds = ImmutableList.of(1L, 2L, 4L, 6L, 7L, 8L, 9L, 10L, 11L, 12L, 13L); ArrayType arrayType = new ArrayType(BIGINT); diff --git a/presto-rcfile/src/test/java/com/facebook/presto/rcfile/RcFileTester.java b/presto-rcfile/src/test/java/com/facebook/presto/rcfile/RcFileTester.java index 5cec03a28bb9..1a6118100cfd 100644 --- a/presto-rcfile/src/test/java/com/facebook/presto/rcfile/RcFileTester.java +++ b/presto-rcfile/src/test/java/com/facebook/presto/rcfile/RcFileTester.java @@ -32,7 +32,7 @@ import com.facebook.presto.common.type.TypeSignatureParameter; import com.facebook.presto.common.type.VarcharType; import com.facebook.presto.hadoop.HadoopNative; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.rcfile.binary.BinaryRcFileEncoding; import com.facebook.presto.rcfile.text.TextRcFileEncoding; import com.facebook.presto.sql.analyzer.FeaturesConfig; @@ -192,7 +192,7 @@ public class RcFileTester static { // associate TYPE_MANAGER with a function manager - new FunctionManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); HadoopNative.requireHadoopNative(); } 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 c7f4fed99cc7..ce3b43e002cb 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 @@ -65,7 +65,7 @@ import com.facebook.presto.metadata.CatalogManager; import com.facebook.presto.metadata.ColumnPropertyManager; import com.facebook.presto.metadata.ConnectorMetadataUpdaterManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.HandleJsonModule; import com.facebook.presto.metadata.InternalNodeManager; import com.facebook.presto.metadata.Metadata; @@ -269,7 +269,7 @@ protected void setup(Binder binder) jsonBinder(binder).addDeserializerBinding(Block.class).to(BlockJsonSerde.Deserializer.class); // metadata - binder.bind(FunctionManager.class).in(Scopes.SINGLETON); + binder.bind(FunctionAndTypeManager.class).in(Scopes.SINGLETON); binder.bind(MetadataManager.class).in(Scopes.SINGLETON); binder.bind(Metadata.class).to(MetadataManager.class).in(Scopes.SINGLETON); binder.bind(StaticFunctionNamespaceStore.class).in(Scopes.SINGLETON); diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkQueryExecutionFactory.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkQueryExecutionFactory.java index 242e8c540dca..af70f8e37c94 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkQueryExecutionFactory.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkQueryExecutionFactory.java @@ -295,7 +295,7 @@ public IPrestoSparkQueryExecution create( PreparedQuery preparedQuery = queryPreparer.prepareQuery(session, sql, warningCollector); planAndMore = queryPlanner.createQueryPlan(session, preparedQuery, warningCollector); SubPlan fragmentedPlan = planFragmenter.fragmentQueryPlan(session, planAndMore.getPlan(), warningCollector); - log.info(textDistributedPlan(fragmentedPlan, metadata.getFunctionManager(), session, true)); + log.info(textDistributedPlan(fragmentedPlan, metadata.getFunctionAndTypeManager(), session, true)); TableWriteInfo tableWriteInfo = getTableWriteInfo(session, fragmentedPlan); JavaSparkContext javaSparkContext = new JavaSparkContext(sparkContext); diff --git a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/PrestoSparkTaskExecutorFactory.java b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/PrestoSparkTaskExecutorFactory.java index 1ebbba48ae11..fc933414e014 100644 --- a/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/PrestoSparkTaskExecutorFactory.java +++ b/presto-spark-base/src/main/java/com/facebook/presto/spark/execution/PrestoSparkTaskExecutorFactory.java @@ -36,7 +36,7 @@ import com.facebook.presto.memory.MemoryPool; import com.facebook.presto.memory.NodeMemoryConfig; import com.facebook.presto.memory.QueryContext; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.operator.FragmentResultCacheManager; import com.facebook.presto.operator.OutputFactory; @@ -125,7 +125,7 @@ public class PrestoSparkTaskExecutorFactory private final SessionPropertyManager sessionPropertyManager; private final BlockEncodingManager blockEncodingManager; - private final FunctionManager functionManager; + private final FunctionAndTypeManager functionAndTypeManager; private final JsonCodec taskDescriptorJsonCodec; private final JsonCodec taskSourceJsonCodec; @@ -159,7 +159,7 @@ public class PrestoSparkTaskExecutorFactory public PrestoSparkTaskExecutorFactory( SessionPropertyManager sessionPropertyManager, BlockEncodingManager blockEncodingManager, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, JsonCodec taskDescriptorJsonCodec, JsonCodec taskSourceJsonCodec, JsonCodec taskInfoJsonCodec, @@ -179,7 +179,7 @@ public PrestoSparkTaskExecutorFactory( this( sessionPropertyManager, blockEncodingManager, - functionManager, + functionAndTypeManager, taskDescriptorJsonCodec, taskSourceJsonCodec, taskInfoJsonCodec, @@ -205,7 +205,7 @@ public PrestoSparkTaskExecutorFactory( public PrestoSparkTaskExecutorFactory( SessionPropertyManager sessionPropertyManager, BlockEncodingManager blockEncodingManager, - FunctionManager functionManager, + FunctionAndTypeManager functionAndTypeManager, JsonCodec taskDescriptorJsonCodec, JsonCodec taskSourceJsonCodec, JsonCodec taskInfoJsonCodec, @@ -229,7 +229,7 @@ public PrestoSparkTaskExecutorFactory( { this.sessionPropertyManager = requireNonNull(sessionPropertyManager, "sessionPropertyManager is null"); this.blockEncodingManager = requireNonNull(blockEncodingManager, "blockEncodingManager is null"); - this.functionManager = requireNonNull(functionManager, "functionManager is null"); + this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null"); this.taskDescriptorJsonCodec = requireNonNull(taskDescriptorJsonCodec, "sparkTaskDescriptorJsonCodec is null"); this.taskSourceJsonCodec = requireNonNull(taskSourceJsonCodec, "taskSourceJsonCodec is null"); this.taskInfoJsonCodec = requireNonNull(taskInfoJsonCodec, "taskInfoJsonCodec is null"); @@ -317,7 +317,7 @@ public IPrestoSparkTaskExecutor doCreate( // TODO: Remove this once we can display the plan on Spark UI. - log.info(PlanPrinter.textPlanFragment(fragment, functionManager, session, true)); + log.info(PlanPrinter.textPlanFragment(fragment, functionAndTypeManager, session, true)); MemoryPool memoryPool = new MemoryPool(new MemoryPoolId("spark-executor-memory-pool"), maxTotalMemory); SpillSpaceTracker spillSpaceTracker = new SpillSpaceTracker(maxSpillMemory); diff --git a/presto-spark-base/src/test/java/com/facebook/presto/spark/PrestoSparkQueryRunner.java b/presto-spark-base/src/test/java/com/facebook/presto/spark/PrestoSparkQueryRunner.java index b4a51b8a3fa6..14e6a8ddd48f 100644 --- a/presto-spark-base/src/test/java/com/facebook/presto/spark/PrestoSparkQueryRunner.java +++ b/presto-spark-base/src/test/java/com/facebook/presto/spark/PrestoSparkQueryRunner.java @@ -442,7 +442,7 @@ public void createCatalog(String catalogName, String connectorName, Map properties) { - metadata.getFunctionManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties); + metadata.getFunctionAndTypeManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties); } @Override 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 47bd542026fa..9f2ccf48c946 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 @@ -378,7 +378,7 @@ public void createCatalog(String catalogName, String connectorName, Map properties) { for (TestingPrestoServer server : servers) { - server.getMetadata().getFunctionManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties); + server.getMetadata().getFunctionAndTypeManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties); } } diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/PlanDeterminismChecker.java b/presto-tests/src/main/java/com/facebook/presto/tests/PlanDeterminismChecker.java index 530562bcacd4..2e67625c12ce 100644 --- a/presto-tests/src/main/java/com/facebook/presto/tests/PlanDeterminismChecker.java +++ b/presto-tests/src/main/java/com/facebook/presto/tests/PlanDeterminismChecker.java @@ -66,7 +66,7 @@ private String getPlanText(Session session, String sql) return PlanPrinter.textLogicalPlan( plan.getRoot(), plan.getTypes(), - localQueryRunner.getMetadata().getFunctionManager(), + localQueryRunner.getMetadata().getFunctionAndTypeManager(), plan.getStatsAndCosts(), transactionSession, 0, diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/StandaloneQueryRunner.java b/presto-tests/src/main/java/com/facebook/presto/tests/StandaloneQueryRunner.java index 2386e39141cf..49eac9c7222f 100644 --- a/presto-tests/src/main/java/com/facebook/presto/tests/StandaloneQueryRunner.java +++ b/presto-tests/src/main/java/com/facebook/presto/tests/StandaloneQueryRunner.java @@ -240,7 +240,7 @@ public void createCatalog(String catalogName, String connectorName, Map properties) { - server.getMetadata().getFunctionManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties); + server.getMetadata().getFunctionAndTypeManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties); } @Override diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/StatefulSleepingSum.java b/presto-tests/src/main/java/com/facebook/presto/tests/StatefulSleepingSum.java index 47eff1a2084c..42cfe9650fde 100644 --- a/presto-tests/src/main/java/com/facebook/presto/tests/StatefulSleepingSum.java +++ b/presto-tests/src/main/java/com/facebook/presto/tests/StatefulSleepingSum.java @@ -16,7 +16,7 @@ import com.facebook.presto.common.function.QualifiedFunctionName; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.SqlScalarFunction; import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation; import com.facebook.presto.spi.function.FunctionKind; @@ -75,7 +75,7 @@ public String getDescription() } @Override - public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionManager functionManager) + public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager) { int args = 4; return new BuiltInScalarFunctionImplementation( diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/StructuralTestUtil.java b/presto-tests/src/main/java/com/facebook/presto/tests/StructuralTestUtil.java index a43bf6dfb660..7d99304f2fff 100644 --- a/presto-tests/src/main/java/com/facebook/presto/tests/StructuralTestUtil.java +++ b/presto-tests/src/main/java/com/facebook/presto/tests/StructuralTestUtil.java @@ -24,7 +24,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.common.type.TypeSignatureParameter; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.type.TypeRegistry; import com.google.common.collect.ImmutableList; @@ -42,7 +42,7 @@ public final class StructuralTestUtil static { // associate TYPE_MANAGER with a function manager - new FunctionManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(TYPE_MANAGER, new BlockEncodingManager(), new FeaturesConfig()); } private StructuralTestUtil() {} diff --git a/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/VerifierModule.java b/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/VerifierModule.java index db5f3dd64484..f560bd93fb8b 100644 --- a/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/VerifierModule.java +++ b/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/VerifierModule.java @@ -20,7 +20,7 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.metadata.CatalogManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.HandleJsonModule; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.sql.parser.SqlParser; @@ -108,7 +108,7 @@ protected final void setup(Binder binder) binder.bind(CatalogManager.class).in(Scopes.SINGLETON); // function - binder.bind(FunctionManager.class).in(SINGLETON); + binder.bind(FunctionAndTypeManager.class).in(SINGLETON); // handle resolver binder.install(new HandleJsonModule()); diff --git a/presto-verifier/src/test/java/com/facebook/presto/verifier/VerifierTestUtil.java b/presto-verifier/src/test/java/com/facebook/presto/verifier/VerifierTestUtil.java index 8331ef33cddd..151ea5911b54 100644 --- a/presto-verifier/src/test/java/com/facebook/presto/verifier/VerifierTestUtil.java +++ b/presto-verifier/src/test/java/com/facebook/presto/verifier/VerifierTestUtil.java @@ -16,7 +16,7 @@ import com.facebook.presto.Session; import com.facebook.presto.block.BlockEncodingManager; import com.facebook.presto.common.type.TypeManager; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.plugin.memory.MemoryPlugin; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.sql.parser.ParsingOptions; @@ -149,7 +149,7 @@ public static ChecksumValidator createChecksumValidator(VerifierConfig verifierC public static TypeManager createTypeManager() { TypeManager typeManager = new TypeRegistry(); - new FunctionManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(typeManager, new BlockEncodingManager(), new FeaturesConfig()); return typeManager; } } diff --git a/presto-verifier/src/test/java/com/facebook/presto/verifier/checksum/TestChecksumValidator.java b/presto-verifier/src/test/java/com/facebook/presto/verifier/checksum/TestChecksumValidator.java index 6e60e194dc32..b3c6aef1aae1 100644 --- a/presto-verifier/src/test/java/com/facebook/presto/verifier/checksum/TestChecksumValidator.java +++ b/presto-verifier/src/test/java/com/facebook/presto/verifier/checksum/TestChecksumValidator.java @@ -17,7 +17,7 @@ import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.common.type.SqlVarbinary; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.FunctionManager; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.sql.parser.SqlParser; import com.facebook.presto.sql.parser.SqlParserOptions; @@ -60,7 +60,7 @@ public class TestChecksumValidator private static final TypeRegistry typeRegistry = new TypeRegistry(); static { - new FunctionManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); + new FunctionAndTypeManager(typeRegistry, new BlockEncodingManager(), new FeaturesConfig()); } private static final Column BIGINT_COLUMN = createColumn("bigint", BIGINT);