Skip to content

Commit

Permalink
Remove TypeManager dependency from FunctionListBuilder
Browse files Browse the repository at this point in the history
After integrating multiple other patches this dependency
which was effect of using Types instead of TypeSignatures
in many places may be removed.
  • Loading branch information
fiedukow authored and cberner committed Jul 5, 2016
1 parent 433f90d commit 7ee2f70
Show file tree
Hide file tree
Showing 15 changed files with 16 additions and 56 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import com.facebook.presto.operator.window.WindowFunction;
import com.facebook.presto.operator.window.WindowFunctionSupplier;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.spi.type.TypeManager;
import com.facebook.presto.spi.type.TypeSignature;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
Expand Down Expand Up @@ -55,12 +54,6 @@ public class FunctionListBuilder
JsonPath.class);

private final List<SqlFunction> functions = new ArrayList<>();
private final TypeManager typeManager;

public FunctionListBuilder(TypeManager typeManager)
{
this.typeManager = requireNonNull(typeManager, "typeManager is null");
}

public FunctionListBuilder window(String name, Type returnType, List<? extends Type> argumentTypes, Class<? extends WindowFunction> functionClass)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -328,7 +328,7 @@ public WindowFunctionSupplier load(SpecializedFunctionKey key)
}
});

FunctionListBuilder builder = new FunctionListBuilder(typeManager)
FunctionListBuilder builder = new FunctionListBuilder()
.window("row_number", BIGINT, ImmutableList.<Type>of(), RowNumberFunction.class)
.window("rank", BIGINT, ImmutableList.<Type>of(), RankFunction.class)
.window("dense_rank", BIGINT, ImmutableList.<Type>of(), DenseRankFunction.class)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ public void testMagicLiteralFunction()
@Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "\\QFunction already registered: custom_add(bigint,bigint):bigint\\E")
public void testDuplicateFunctions()
{
List<SqlFunction> functions = new FunctionListBuilder(new TypeRegistry())
List<SqlFunction> functions = new FunctionListBuilder()
.scalar(CustomFunctions.class)
.getFunctions()
.stream()
Expand All @@ -124,7 +124,7 @@ public void testDuplicateFunctions()
public void testConflictingScalarAggregation()
throws Exception
{
List<SqlFunction> functions = new FunctionListBuilder(new TypeRegistry())
List<SqlFunction> functions = new FunctionListBuilder()
.scalar(ScalarSum.class)
.getFunctions();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
import com.facebook.presto.spi.block.BlockBuilderStatus;
import com.facebook.presto.spi.type.StandardTypes;
import com.facebook.presto.type.SqlType;
import com.facebook.presto.type.TypeRegistry;
import com.google.common.collect.ImmutableList;
import org.testng.annotations.BeforeClass;

Expand All @@ -34,7 +33,7 @@ public class TestCountNullAggregation
@BeforeClass
public void setup()
{
functionRegistry.addFunctions(new FunctionListBuilder(new TypeRegistry()).aggregate(CountNull.class).getFunctions());
functionRegistry.addFunctions(new FunctionListBuilder().aggregate(CountNull.class).getFunctions());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -138,7 +138,7 @@ protected void assertInvalidCast(String projection, String message)
protected void registerScalar(Class<?> clazz)
{
Metadata metadata = functionAssertions.getMetadata();
List<SqlFunction> functions = new FunctionListBuilder(metadata.getTypeManager())
List<SqlFunction> functions = new FunctionListBuilder()
.scalar(clazz)
.getFunctions();
metadata.getFunctionRegistry().addFunctions(functions);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,7 +110,7 @@ public static class BenchmarkData
public void setup()
{
MetadataManager metadata = MetadataManager.createTestMetadataManager();
metadata.addFunctions(new FunctionListBuilder(metadata.getTypeManager()).scalar(BenchmarkArrayDistinct.class).getFunctions());
metadata.addFunctions(new FunctionListBuilder().scalar(BenchmarkArrayDistinct.class).getFunctions());
ExpressionCompiler compiler = new ExpressionCompiler(metadata);
ImmutableList.Builder<RowExpression> projectionsBuilder = ImmutableList.builder();
Block[] blocks = new Block[TYPES.size()];
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -112,8 +112,8 @@ public static class BenchmarkData
public void setup()
{
MetadataManager metadata = MetadataManager.createTestMetadataManager();
metadata.addFunctions(new FunctionListBuilder(metadata.getTypeManager()).scalar(BenchmarkOldArrayHash.class).getFunctions());
metadata.addFunctions(new FunctionListBuilder(metadata.getTypeManager()).scalar(BenchmarkAnotherArrayHash.class).getFunctions());
metadata.addFunctions(new FunctionListBuilder().scalar(BenchmarkOldArrayHash.class).getFunctions());
metadata.addFunctions(new FunctionListBuilder().scalar(BenchmarkAnotherArrayHash.class).getFunctions());
ExpressionCompiler compiler = new ExpressionCompiler(metadata);
ImmutableList.Builder<RowExpression> projectionsBuilder = ImmutableList.builder();
Block[] blocks = new Block[1];
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ public static class BenchmarkData
public void setup()
{
MetadataManager metadata = MetadataManager.createTestMetadataManager();
metadata.addFunctions(new FunctionListBuilder(metadata.getTypeManager()).scalar(BenchmarkArraySort.class).getFunctions());
metadata.addFunctions(new FunctionListBuilder().scalar(BenchmarkArraySort.class).getFunctions());
ExpressionCompiler compiler = new ExpressionCompiler(metadata);
ImmutableList.Builder<RowExpression> projectionsBuilder = ImmutableList.builder();
Block[] blocks = new Block[TYPES.size()];
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -216,7 +216,7 @@ public FunctionAssertions addFunctions(List<SqlFunction> functionInfos)

public FunctionAssertions addScalarFunctions(Class<?> clazz)
{
metadata.addFunctions(new FunctionListBuilder(metadata.getTypeManager()).scalar(clazz).getFunctions());
metadata.addFunctions(new FunctionListBuilder().scalar(clazz).getFunctions());
return this;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ public MLFunctionFactory(TypeManager typeManager)
@Override
public List<SqlFunction> listFunctions()
{
return new FunctionListBuilder(typeManager)
return new FunctionListBuilder()
.aggregate(LearnClassifierAggregation.class)
.aggregate(LearnVarcharClassifierAggregation.class)
.aggregate(LearnRegressorAggregation.class)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,26 +16,16 @@
import com.facebook.presto.metadata.FunctionFactory;
import com.facebook.presto.metadata.FunctionListBuilder;
import com.facebook.presto.metadata.SqlFunction;
import com.facebook.presto.spi.type.TypeManager;

import java.util.List;

import static java.util.Objects.requireNonNull;

public class MongoFunctionFactory
implements FunctionFactory
{
private final TypeManager typeManager;

public MongoFunctionFactory(TypeManager typeManager)
{
this.typeManager = requireNonNull(typeManager, "typeManager is null");
}

@Override
public List<SqlFunction> listFunctions()
{
return new FunctionListBuilder(typeManager)
return new FunctionListBuilder()
.scalar(ObjectIdFunctions.class)
.getFunctions();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ public <T> List<T> getServices(Class<T> type)
return ImmutableList.of(type.cast(OBJECT_ID));
}
if (type == FunctionFactory.class) {
return ImmutableList.of(type.cast(new MongoFunctionFactory(typeManager)));
return ImmutableList.of(type.cast(new MongoFunctionFactory()));
}
return ImmutableList.of();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,24 +16,16 @@
import com.facebook.presto.metadata.FunctionFactory;
import com.facebook.presto.metadata.FunctionListBuilder;
import com.facebook.presto.metadata.SqlFunction;
import com.facebook.presto.spi.type.TypeManager;

import java.util.List;

public class TeradataFunctionFactory
implements FunctionFactory
{
private final TypeManager typeManager;

public TeradataFunctionFactory(TypeManager typeManager)
{
this.typeManager = typeManager;
}

@Override
public List<SqlFunction> listFunctions()
{
return new FunctionListBuilder(typeManager)
return new FunctionListBuilder()
.scalar(TeradataStringFunctions.class)
.scalar(TeradataDateFunctions.class)
.getFunctions();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,31 +15,18 @@

import com.facebook.presto.metadata.FunctionFactory;
import com.facebook.presto.spi.Plugin;
import com.facebook.presto.spi.type.TypeManager;
import com.google.common.collect.ImmutableList;

import javax.inject.Inject;

import java.util.List;

import static java.util.Objects.requireNonNull;

public class TeradataFunctionsPlugin
implements Plugin
{
private TypeManager typeManager;

@Inject
public void setTypeManager(TypeManager typeManager)
{
this.typeManager = requireNonNull(typeManager, "typeManager is null");
}

@Override
public <T> List<T> getServices(Class<T> type)
{
if (type == FunctionFactory.class) {
return ImmutableList.of(type.cast(new TeradataFunctionFactory(typeManager)));
return ImmutableList.of(type.cast(new TeradataFunctionFactory()));
}
return ImmutableList.of();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import com.facebook.presto.testing.QueryRunner;
import com.facebook.presto.type.SqlIntervalDayTime;
import com.facebook.presto.type.SqlIntervalYearMonth;
import com.facebook.presto.type.TypeRegistry;
import com.facebook.presto.util.DateTimeZoneIndex;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ImmutableList;
Expand Down Expand Up @@ -85,7 +84,7 @@ public abstract class AbstractTestQueries
extends AbstractTestQueryFramework
{
// We can just use the default type registry, since we don't use any parametric types
protected static final List<SqlFunction> CUSTOM_FUNCTIONS = new FunctionListBuilder(new TypeRegistry())
protected static final List<SqlFunction> CUSTOM_FUNCTIONS = new FunctionListBuilder()
.aggregate(CustomSum.class)
.window("custom_rank", BIGINT, ImmutableList.<Type>of(), CustomRank.class)
.scalar(CustomAdd.class)
Expand Down

0 comments on commit 7ee2f70

Please sign in to comment.