Skip to content

Commit

Permalink
Change FunctionManager to FunctionAndTypeManager
Browse files Browse the repository at this point in the history
  • Loading branch information
rongrong authored and Rongrong Zhong committed Oct 13, 2020
1 parent 4720be3 commit 63af935
Show file tree
Hide file tree
Showing 385 changed files with 1,857 additions and 1,803 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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()),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -220,7 +220,7 @@ protected final OperatorFactory createHashProjectOperator(int operatorId, PlanNo
projections.add(new PageProjectionWithOutputs(new InputPageProjection(channel), new int[] {channel}));
}

Optional<RowExpression> hashExpression = HashGenerationOptimizer.getHashExpression(localQueryRunner.getMetadata().getFunctionManager(), variables.build());
Optional<RowExpression> hashExpression = HashGenerationOptimizer.getHashExpression(localQueryRunner.getMetadata().getFunctionAndTypeManager(), variables.build());
verify(hashExpression.isPresent());
RowExpression translatedHashExpression = translate(hashExpression.get(), variableToInputMapping.build());

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -41,9 +41,9 @@ public CountAggregationBenchmark(LocalQueryRunner localQueryRunner)
protected List<? extends OperatorFactory> 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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -42,9 +42,9 @@ public DoubleSumAggregationBenchmark(LocalQueryRunner localQueryRunner)
protected List<? extends OperatorFactory> 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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ protected List<? extends OperatorFactory> 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));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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<OGCGeometry> makeGeometries()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1412,7 +1412,7 @@ protected void doTestMismatchSchemaTable(
ROW_EXPRESSION_SERVICE,
FUNCTION_RESOLUTION,
hivePartitionManager,
METADATA.getFunctionManager(),
METADATA.getFunctionAndTypeManager(),
tableHandle,
predicate,
Optional.empty()).getLayout().getHandle();
Expand Down Expand Up @@ -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();
Expand Down Expand Up @@ -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();
Expand Down
Loading

0 comments on commit 63af935

Please sign in to comment.