diff --git a/presto-hive-function-namespace/src/test/java/com/facebook/presto/hive/functions/HiveFunctionsTestUtils.java b/presto-hive-function-namespace/src/test/java/com/facebook/presto/hive/functions/HiveFunctionsTestUtils.java index f69fbf53c434..c06f42ca9e69 100644 --- a/presto-hive-function-namespace/src/test/java/com/facebook/presto/hive/functions/HiveFunctionsTestUtils.java +++ b/presto-hive-function-namespace/src/test/java/com/facebook/presto/hive/functions/HiveFunctionsTestUtils.java @@ -38,7 +38,8 @@ public static TestingPrestoServer createTestingPrestoServer() "hive-functions", "hive", getNamespaceManagerCreationProperties(), - server.getPluginNodeManager()); + server.getPluginNodeManager(), + false); server.refreshNodes(); return server; } diff --git a/presto-hive-function-namespace/src/test/java/com/facebook/presto/hive/functions/TestHiveAggregationFunctions.java b/presto-hive-function-namespace/src/test/java/com/facebook/presto/hive/functions/TestHiveAggregationFunctions.java index 981477c77229..4cc475413306 100644 --- a/presto-hive-function-namespace/src/test/java/com/facebook/presto/hive/functions/TestHiveAggregationFunctions.java +++ b/presto-hive-function-namespace/src/test/java/com/facebook/presto/hive/functions/TestHiveAggregationFunctions.java @@ -150,7 +150,8 @@ private static TestingPrestoServer createServer() "hive-functions", "hive", Collections.emptyMap(), - server.getPluginNodeManager()); + server.getPluginNodeManager(), + false); server.refreshNodes(); return server; } diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/FunctionAndTypeManager.java b/presto-main/src/main/java/com/facebook/presto/metadata/FunctionAndTypeManager.java index 0d07bb362579..3db69932aaf2 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/FunctionAndTypeManager.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/FunctionAndTypeManager.java @@ -33,7 +33,6 @@ import com.facebook.presto.common.type.TypeWithName; import com.facebook.presto.common.type.UserDefinedType; import com.facebook.presto.operator.window.WindowFunctionSupplier; -import com.facebook.presto.server.ServerConfig; import com.facebook.presto.spi.NodeManager; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.function.AggregationFunctionImplementation; @@ -66,7 +65,6 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.UncheckedExecutionException; import org.weakref.jmx.Managed; @@ -106,7 +104,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.ImmutableList.toImmutableList; -import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static java.lang.String.format; import static java.util.Collections.emptyList; @@ -122,7 +119,6 @@ public class FunctionAndTypeManager implements FunctionMetadataManager, TypeManager { - private static final String IS_DEFAULT_NAMESPACE = "default.namespace"; private final TransactionManager transactionManager; private final BlockEncodingSerde blockEncodingSerde; private final BuiltInTypeAndFunctionNamespaceManager builtInTypeAndFunctionNamespaceManager; @@ -135,7 +131,6 @@ public class FunctionAndTypeManager private final LoadingCache functionCache; private final CacheStatsMBean cacheStatsMBean; private final boolean nativeExecution; - private final boolean coordinatorSidecarEnabled; private CatalogSchemaName currentDefaultNamespace = DEFAULT_NAMESPACE; @Inject @@ -144,7 +139,6 @@ public FunctionAndTypeManager( BlockEncodingSerde blockEncodingSerde, FeaturesConfig featuresConfig, FunctionsConfig functionsConfig, - ServerConfig serverConfig, HandleResolver handleResolver, Set types) { @@ -165,7 +159,6 @@ public FunctionAndTypeManager( this.functionSignatureMatcher = new FunctionSignatureMatcher(this); this.typeCoercer = new TypeCoercer(functionsConfig, this); this.nativeExecution = featuresConfig.isNativeExecutionEnabled(); - this.coordinatorSidecarEnabled = serverConfig.isCoordinatorSidecarEnabled(); } public static FunctionAndTypeManager createTestFunctionAndTypeManager() @@ -175,7 +168,6 @@ public static FunctionAndTypeManager createTestFunctionAndTypeManager() new BlockEncodingManager(), new FeaturesConfig(), new FunctionsConfig(), - new ServerConfig(), new HandleResolver(), ImmutableSet.of()); } @@ -288,14 +280,16 @@ public void loadFunctionNamespaceManager( String functionNamespaceManagerName, String catalogName, Map properties, - NodeManager nodeManager) + NodeManager nodeManager, + boolean isDefaultNamespace) { requireNonNull(functionNamespaceManagerName, "functionNamespaceManagerName is null"); FunctionNamespaceManagerFactory factory = functionNamespaceManagerFactories.get(functionNamespaceManagerName); checkState(factory != null, "No factory for function namespace manager %s", functionNamespaceManagerName); - ImmutableMap newProperties = - ImmutableMap.copyOf(constructNewPropertyMapIfDefaultNamespaceConfigured(properties, catalogName)); - FunctionNamespaceManager functionNamespaceManager = factory.create(catalogName, newProperties, new FunctionNamespaceManagerContext(this, nodeManager, this)); + if (isDefaultNamespace) { + this.currentDefaultNamespace = new CatalogSchemaName(catalogName, "default"); + } + FunctionNamespaceManager functionNamespaceManager = factory.create(catalogName, properties, new FunctionNamespaceManagerContext(this, nodeManager, this)); functionNamespaceManager.setBlockEncodingSerde(blockEncodingSerde); transactionManager.registerFunctionNamespaceManager(catalogName, functionNamespaceManager); @@ -386,19 +380,10 @@ public List listFunctions(Session session, Optional likePat ImmutableList.Builder functions = new ImmutableList.Builder<>(); if (!isListBuiltInFunctionsOnly(session)) { functions.addAll(SessionFunctionUtils.listFunctions(session.getSessionFunctions())); - - // If coordinator sidecar is enabled, filter on the current default namespace - if (coordinatorSidecarEnabled) { - functions.addAll(functionNamespaceManagers.entrySet().stream() - .flatMap(manager -> manager.getValue().listFunctions(likePattern, escape).stream() - .filter((functionName) -> functionName.getSignature().getName().getCatalogSchemaName().equals(currentDefaultNamespace))) - .collect(toImmutableList())); - } - else { - functions.addAll(functionNamespaceManagers.values().stream() - .flatMap(manager -> manager.listFunctions(likePattern, escape).stream()) - .collect(toImmutableList())); - } + functions.addAll(functionNamespaceManagers.entrySet().stream() + .flatMap(manager -> manager.getValue().listFunctions(likePattern, escape).stream() + .filter((functionName) -> functionName.getSignature().getName().getCatalogSchemaName().equals(currentDefaultNamespace))) + .collect(toImmutableList())); } else { functions.addAll(listBuiltInFunctions()); @@ -671,11 +656,6 @@ public FunctionHandle lookupCast(CastType castType, Type fromType, Type toType) return builtInTypeAndFunctionNamespaceManager.getFunctionHandle(Optional.empty(), signature); } - public boolean isCoordinatorSidecarEnabled() - { - return coordinatorSidecarEnabled; - } - public CatalogSchemaName getCurrentDefaultNamespace() { return currentDefaultNamespace; @@ -777,21 +757,6 @@ private Optional> getServingFunc return Optional.ofNullable(functionNamespaceManagers.get(typeSignatureBase.getTypeName().getCatalogName())); } - private Map constructNewPropertyMapIfDefaultNamespaceConfigured(Map properties, String catalogName) - { - if (!properties.containsKey(IS_DEFAULT_NAMESPACE)) { - return properties; - } - // If default namespace is configured, update the default namespace - String isDefaultNamespace = properties.get(IS_DEFAULT_NAMESPACE); - if (Boolean.parseBoolean(isDefaultNamespace)) { - this.currentDefaultNamespace = new CatalogSchemaName(catalogName, "default"); - } - return properties.entrySet().stream() - .filter(entry -> !entry.getKey().equals(IS_DEFAULT_NAMESPACE)) - .collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue)); - } - @Override @SuppressWarnings("unchecked") public SpecializedFunctionKey getSpecializedFunctionKey(Signature signature) 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 b1ee09770f76..7f9812c48f89 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 @@ -27,7 +27,6 @@ import com.facebook.presto.common.type.Type; import com.facebook.presto.common.type.TypeSignature; import com.facebook.presto.execution.QueryManager; -import com.facebook.presto.server.ServerConfig; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ColumnMetadata; import com.facebook.presto.spi.ConnectorId; @@ -214,39 +213,34 @@ public static MetadataManager createTestMetadataManager() public static MetadataManager createTestMetadataManager(FeaturesConfig featuresConfig) { - return createTestMetadataManager(new CatalogManager(), featuresConfig, new FunctionsConfig(), new ServerConfig()); + return createTestMetadataManager(new CatalogManager(), featuresConfig, new FunctionsConfig()); } public static MetadataManager createTestMetadataManager(FunctionsConfig functionsConfig) { - return createTestMetadataManager(new CatalogManager(), new FeaturesConfig(), functionsConfig, new ServerConfig()); - } - - public static MetadataManager createTestMetadataManager(ServerConfig serverConfig) - { - return createTestMetadataManager(new CatalogManager(), new FeaturesConfig(), new FunctionsConfig(), serverConfig); + return createTestMetadataManager(new CatalogManager(), new FeaturesConfig(), functionsConfig); } public static MetadataManager createTestMetadataManager(CatalogManager catalogManager) { - return createTestMetadataManager(catalogManager, new FeaturesConfig(), new FunctionsConfig(), new ServerConfig()); + return createTestMetadataManager(catalogManager, new FeaturesConfig(), new FunctionsConfig()); } - public static MetadataManager createTestMetadataManager(CatalogManager catalogManager, FeaturesConfig featuresConfig, FunctionsConfig functionsConfig, ServerConfig serverConfig) + public static MetadataManager createTestMetadataManager(CatalogManager catalogManager, FeaturesConfig featuresConfig, FunctionsConfig functionsConfig) { - return createTestMetadataManager(createTestTransactionManager(catalogManager), featuresConfig, functionsConfig, serverConfig); + return createTestMetadataManager(createTestTransactionManager(catalogManager), featuresConfig, functionsConfig); } public static MetadataManager createTestMetadataManager(TransactionManager transactionManager) { - return createTestMetadataManager(transactionManager, new FeaturesConfig(), new FunctionsConfig(), new ServerConfig()); + return createTestMetadataManager(transactionManager, new FeaturesConfig(), new FunctionsConfig()); } - public static MetadataManager createTestMetadataManager(TransactionManager transactionManager, FeaturesConfig featuresConfig, FunctionsConfig functionsConfig, ServerConfig serverConfig) + public static MetadataManager createTestMetadataManager(TransactionManager transactionManager, FeaturesConfig featuresConfig, FunctionsConfig functionsConfig) { BlockEncodingManager blockEncodingManager = new BlockEncodingManager(); return new MetadataManager( - new FunctionAndTypeManager(transactionManager, blockEncodingManager, featuresConfig, functionsConfig, serverConfig, new HandleResolver(), ImmutableSet.of()), + new FunctionAndTypeManager(transactionManager, blockEncodingManager, featuresConfig, functionsConfig, new HandleResolver(), ImmutableSet.of()), blockEncodingManager, createTestingSessionPropertyManager(), new SchemaPropertyManager(), 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 3f9ad173facb..2822a85143c5 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 @@ -29,6 +29,7 @@ import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Strings.isNullOrEmpty; import static com.google.common.io.Files.getNameWithoutExtension; +import static java.lang.Boolean.parseBoolean; import static java.lang.String.format; public class StaticFunctionNamespaceStore @@ -83,15 +84,15 @@ private void loadFunctionNamespaceManager(String catalogName, Map(properties); String functionNamespaceManagerName = properties.remove(FUNCTION_NAMESPACE_MANAGER_NAME); checkState(!isNullOrEmpty(functionNamespaceManagerName), "%s property must be present", FUNCTION_NAMESPACE_MANAGER_NAME); - - if (Boolean.parseBoolean(properties.get(DEFAULT_NAMESPACE))) { + boolean defaultNamespace = parseBoolean(properties.remove(DEFAULT_NAMESPACE)); + if (defaultNamespace) { if (isDefaultNamespace) { throw new IllegalStateException( format("Only one function namespace manager can have the %s property set.", DEFAULT_NAMESPACE)); } isDefaultNamespace = true; } - functionAndTypeManager.loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties, nodeManager); + functionAndTypeManager.loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties, nodeManager, defaultNamespace); log.info("-- Added function namespace manager [%s] --", catalogName); } 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 c86900f03697..a900bb364d16 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 @@ -893,13 +893,13 @@ public PlanOptimizers( // Optimizers above this do not need to care about aggregations with the type other than SINGLE // This optimizer must be run after all exchange-related optimizers builder.add(new IterativeOptimizer( - metadata, - ruleStats, - statsCalculator, - costCalculator, - ImmutableSet.of( - new PushPartialAggregationThroughJoin(), - new PushPartialAggregationThroughExchange(metadata.getFunctionAndTypeManager(), featuresConfig.isNativeExecutionEnabled()))), + metadata, + ruleStats, + statsCalculator, + costCalculator, + ImmutableSet.of( + new PushPartialAggregationThroughJoin(), + new PushPartialAggregationThroughExchange(metadata.getFunctionAndTypeManager(), featuresConfig.isNativeExecutionEnabled()))), // MergePartialAggregationsWithFilter should immediately follow PushPartialAggregationThroughExchange new MergePartialAggregationsWithFilter(metadata.getFunctionAndTypeManager()), new IterativeOptimizer( 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 b68e0aab8d8f..b099f7766621 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 @@ -854,7 +854,6 @@ public PlanWithProperties visitTableFinish(TableFinishNode node, PreferredProper PlanNode child = planChild(node, PreferredProperties.any()).getNode(); ExchangeNode gather; - // in case the input is a union (see PushTableWriteThroughUnion), don't add another exchange if (child instanceof ExchangeNode) { ExchangeNode exchangeNode = (ExchangeNode) child; 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 d16b12f35834..404f6cc7833a 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,6 +17,8 @@ import com.facebook.presto.common.function.OperatorType; import com.facebook.presto.common.type.CharType; import com.facebook.presto.common.type.Type; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.StandardErrorCode; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.function.StandardFunctionResolution; import com.facebook.presto.sql.analyzer.FunctionAndTypeResolver; @@ -85,6 +87,20 @@ public FunctionHandle likeVarcharFunction() return functionAndTypeResolver.lookupFunction("LIKE", fromTypes(VARCHAR, LIKE_PATTERN)); } + public boolean supportsLikePatternFunction() + { + try { + functionAndTypeResolver.lookupFunction("LIKE_PATTERN", fromTypes(VARCHAR, VARCHAR)); + return true; + } + catch (PrestoException e) { + if (e.getErrorCode() == StandardErrorCode.FUNCTION_NOT_FOUND.toErrorCode()) { + return false; + } + throw e; + } + } + public FunctionHandle likeVarcharVarcharFunction() { return functionAndTypeResolver.lookupFunction("LIKE", fromTypes(VARCHAR, VARCHAR)); 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 38b313f23760..57caface8f45 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 @@ -272,7 +272,6 @@ private static class Visitor private final SqlFunctionProperties sqlFunctionProperties; private final Map sessionFunctions; private final FunctionResolution functionResolution; - private final boolean coordinatorSidecarEnabled; private Visitor( Map, Type> types, @@ -292,7 +291,6 @@ private Visitor( this.sqlFunctionProperties = requireNonNull(sqlFunctionProperties); this.functionResolution = new FunctionResolution(functionAndTypeResolver); this.sessionFunctions = requireNonNull(sessionFunctions); - this.coordinatorSidecarEnabled = functionAndTypeManager.isCoordinatorSidecarEnabled(); } private Type getType(Expression node) @@ -910,7 +908,7 @@ protected RowExpression visitLikePredicate(LikePredicate node, Context context) if (node.getEscape().isPresent()) { RowExpression escape = process(node.getEscape().get(), context); - if (coordinatorSidecarEnabled) { + if (!functionResolution.supportsLikePatternFunction()) { return call(value.getSourceLocation(), "LIKE", functionResolution.likeVarcharVarcharVarcharFunction(), BOOLEAN, value, pattern, escape); } return likeFunctionCall(value, call(getSourceLocation(node), "LIKE_PATTERN", functionResolution.likePatternFunction(), LIKE_PATTERN, pattern, escape)); @@ -921,7 +919,7 @@ protected RowExpression visitLikePredicate(LikePredicate node, Context context) return prefixOrSuffixMatch; } - if (coordinatorSidecarEnabled) { + if (!functionResolution.supportsLikePatternFunction()) { return likeFunctionCall(value, pattern); } @@ -970,7 +968,7 @@ else if (LIKE_SIMPLE_EXISTS_PATTERN.matcher(patternString).matches()) { private RowExpression likeFunctionCall(RowExpression value, RowExpression pattern) { if (value.getType() instanceof VarcharType) { - if (coordinatorSidecarEnabled) { + if (!functionResolution.supportsLikePatternFunction()) { return call(value.getSourceLocation(), "LIKE", functionResolution.likeVarcharVarcharFunction(), BOOLEAN, value, pattern); } return call(value.getSourceLocation(), "LIKE", functionResolution.likeVarcharFunction(), BOOLEAN, value, pattern); 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 d4012953b944..e71c229f2876 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 @@ -123,7 +123,6 @@ import com.facebook.presto.server.NodeStatusNotificationManager; import com.facebook.presto.server.PluginManager; import com.facebook.presto.server.PluginManagerConfig; -import com.facebook.presto.server.ServerConfig; import com.facebook.presto.server.SessionPropertyDefaults; import com.facebook.presto.server.security.PasswordAuthenticatorManager; import com.facebook.presto.spi.ConnectorId; @@ -424,7 +423,7 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig, featuresConfig.setIgnoreStatsCalculatorFailures(false); this.metadata = new MetadataManager( - new FunctionAndTypeManager(transactionManager, blockEncodingManager, featuresConfig, functionsConfig, new ServerConfig(), new HandleResolver(), ImmutableSet.of()), + new FunctionAndTypeManager(transactionManager, blockEncodingManager, featuresConfig, functionsConfig, new HandleResolver(), ImmutableSet.of()), blockEncodingManager, createTestingSessionPropertyManager( new SystemSessionProperties( @@ -753,9 +752,9 @@ public void createCatalog(String catalogName, String connectorName, Map properties) + public void loadFunctionNamespaceManager(String functionNamespaceManagerName, String catalogName, Map properties, boolean isDefaultNamespace) { - metadata.getFunctionAndTypeManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties, pluginNodeManager); + metadata.getFunctionAndTypeManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties, pluginNodeManager, isDefaultNamespace); } public LocalQueryRunner printPlan() diff --git a/presto-main/src/main/java/com/facebook/presto/testing/QueryRunner.java b/presto-main/src/main/java/com/facebook/presto/testing/QueryRunner.java index d2121ce335f8..7e5c83b725c9 100644 --- a/presto-main/src/main/java/com/facebook/presto/testing/QueryRunner.java +++ b/presto-main/src/main/java/com/facebook/presto/testing/QueryRunner.java @@ -95,7 +95,7 @@ default void installCoordinatorPlugin(CoordinatorPlugin plugin) void createCatalog(String catalogName, String connectorName, Map properties); - void loadFunctionNamespaceManager(String functionNamespaceManagerName, String catalogName, Map properties); + void loadFunctionNamespaceManager(String functionNamespaceManagerName, String catalogName, Map properties, boolean isDefaultNamespace); default void loadSessionPropertyProvider(String sessionPropertyProviderName) { 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 93c9e8c50536..b7f8ad2e1e84 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 @@ -27,8 +27,9 @@ import com.facebook.presto.common.type.VarbinaryType; import com.facebook.presto.functionNamespace.json.JsonFileBasedFunctionNamespaceManagerFactory; import com.facebook.presto.metadata.FunctionAndTypeManager; +import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.metadata.MetadataManager; import com.facebook.presto.operator.scalar.FunctionAssertions; -import com.facebook.presto.server.testing.TestingPrestoServer; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.function.AggregationFunctionMetadata; @@ -58,6 +59,7 @@ import com.facebook.presto.sql.tree.NodeRef; import com.facebook.presto.sql.tree.QualifiedName; import com.facebook.presto.sql.tree.StringLiteral; +import com.facebook.presto.testing.TestingNodeManager; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -70,7 +72,6 @@ import org.joda.time.DateTimeZone; import org.joda.time.LocalDate; import org.joda.time.LocalTime; -import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -80,7 +81,6 @@ import java.util.concurrent.TimeUnit; import java.util.stream.IntStream; -import static com.facebook.airlift.testing.Closeables.closeQuietly; import static com.facebook.presto.SessionTestUtils.TEST_SESSION; import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.BooleanType.BOOLEAN; @@ -175,31 +175,15 @@ public class TestExpressionInterpreter .build()); private static final SqlParser SQL_PARSER = new SqlParser(); - private static final TestingPrestoServer server; - - static { - try { - server = new TestingPrestoServer(); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - private static final TestingRowExpressionTranslator TRANSLATOR = new TestingRowExpressionTranslator(server.getMetadata()); + private static final Metadata METADATA = MetadataManager.createTestMetadataManager(); + private static final TestingRowExpressionTranslator TRANSLATOR = new TestingRowExpressionTranslator(METADATA); private static final BlockEncodingSerde blockEncodingSerde = new BlockEncodingManager(); @BeforeClass public void setup() { - server.getMetadata().getFunctionAndTypeManager().registerBuiltInFunctions(ImmutableList.of(APPLY_FUNCTION)); - setupJsonFunctionNamespaceManager(server.getMetadata().getFunctionAndTypeManager()); - } - - @AfterClass(alwaysRun = true) - public void tearDownServer() - { - closeQuietly(server); + METADATA.getFunctionAndTypeManager().registerBuiltInFunctions(ImmutableList.of(APPLY_FUNCTION)); + setupJsonFunctionNamespaceManager(METADATA.getFunctionAndTypeManager()); } @Test @@ -432,14 +416,14 @@ public void testNonDeterministicFunctionCall() @Test public void testCppFunctionCall() { - server.getMetadata().getFunctionAndTypeManager().createFunction(SQUARE_UDF_CPP, false); + METADATA.getFunctionAndTypeManager().createFunction(SQUARE_UDF_CPP, false); assertOptimizedEquals("json.test_schema.square(-5)", "json.test_schema.square(-5)"); } @Test public void testCppAggregateFunctionCall() { - server.getMetadata().getFunctionAndTypeManager().createFunction(AVG_UDAF_CPP, false); + METADATA.getFunctionAndTypeManager().createFunction(AVG_UDAF_CPP, false); assertOptimizedEquals("json.test_schema.avg(1.0)", "json.test_schema.avg(1.0)"); } @@ -451,7 +435,8 @@ private void setupJsonFunctionNamespaceManager(FunctionAndTypeManager functionAn JsonFileBasedFunctionNamespaceManagerFactory.NAME, "json", ImmutableMap.of("supported-function-languages", "CPP", "function-implementation-type", "CPP"), - server.getPluginNodeManager()); + new TestingNodeManager(), + false); } @Test @@ -1671,7 +1656,7 @@ private static Object optimize(@Language("SQL") String expression) private static Expression expression(String expression) { - return FunctionAssertions.createExpression(expression, server.getMetadata(), SYMBOL_TYPES); + return FunctionAssertions.createExpression(expression, METADATA, SYMBOL_TYPES); } private static RowExpression toRowExpression(Expression expression) @@ -1681,8 +1666,8 @@ private static RowExpression toRowExpression(Expression expression) private static Object optimize(Expression expression) { - Map, Type> expressionTypes = getExpressionTypes(TEST_SESSION, server.getMetadata(), SQL_PARSER, SYMBOL_TYPES, expression, emptyMap(), WarningCollector.NOOP); - ExpressionInterpreter interpreter = expressionOptimizer(expression, server.getMetadata(), TEST_SESSION, expressionTypes); + Map, Type> expressionTypes = getExpressionTypes(TEST_SESSION, METADATA, SQL_PARSER, SYMBOL_TYPES, expression, emptyMap(), WarningCollector.NOOP); + ExpressionInterpreter interpreter = expressionOptimizer(expression, METADATA, TEST_SESSION, expressionTypes); return interpreter.optimize(variable -> { Symbol symbol = new Symbol(variable.getName()); Object value = symbolConstant(symbol); @@ -1695,7 +1680,7 @@ private static Object optimize(Expression expression) private static Object optimize(RowExpression expression, Level level) { - return new RowExpressionInterpreter(expression, server.getMetadata().getFunctionAndTypeManager(), TEST_SESSION.toConnectorSession(), level).optimize(variable -> { + return new RowExpressionInterpreter(expression, METADATA.getFunctionAndTypeManager(), TEST_SESSION.toConnectorSession(), level).optimize(variable -> { Symbol symbol = new Symbol(variable.getName()); Object value = symbolConstant(symbol); if (value == null) { @@ -1827,10 +1812,10 @@ else if (left instanceof SpecialFormExpression) { private static boolean isRemovableCast(Object value) { if (value instanceof CallExpression && - new FunctionResolution(server.getMetadata().getFunctionAndTypeManager().getFunctionAndTypeResolver()).isCastFunction(((CallExpression) value).getFunctionHandle())) { + new FunctionResolution(METADATA.getFunctionAndTypeManager().getFunctionAndTypeResolver()).isCastFunction(((CallExpression) value).getFunctionHandle())) { Type targetType = ((CallExpression) value).getType(); Type sourceType = ((CallExpression) value).getArguments().get(0).getType(); - return server.getMetadata().getFunctionAndTypeManager().canCoerce(sourceType, targetType); + return METADATA.getFunctionAndTypeManager().canCoerce(sourceType, targetType); } return false; } @@ -1852,7 +1837,7 @@ private static Object evaluate(String expression, boolean deterministic) { assertRoundTrip(expression); - Expression parsedExpression = FunctionAssertions.createExpression(expression, server.getMetadata(), SYMBOL_TYPES); + Expression parsedExpression = FunctionAssertions.createExpression(expression, METADATA, SYMBOL_TYPES); return evaluate(parsedExpression, deterministic); } @@ -1866,9 +1851,9 @@ private static void assertRoundTrip(String expression) private static Object evaluate(Expression expression, boolean deterministic) { - Map, Type> expressionTypes = getExpressionTypes(TEST_SESSION, server.getMetadata(), SQL_PARSER, SYMBOL_TYPES, expression, emptyMap(), WarningCollector.NOOP); - Object expressionResult = expressionInterpreter(expression, server.getMetadata(), TEST_SESSION, expressionTypes).evaluate(); - Object rowExpressionResult = rowExpressionInterpreter(TRANSLATOR.translateAndOptimize(expression), server.getMetadata().getFunctionAndTypeManager(), TEST_SESSION.toConnectorSession()).evaluate(); + Map, Type> expressionTypes = getExpressionTypes(TEST_SESSION, METADATA, SQL_PARSER, SYMBOL_TYPES, expression, emptyMap(), WarningCollector.NOOP); + Object expressionResult = expressionInterpreter(expression, METADATA, TEST_SESSION, expressionTypes).evaluate(); + Object rowExpressionResult = rowExpressionInterpreter(TRANSLATOR.translateAndOptimize(expression), METADATA.getFunctionAndTypeManager(), TEST_SESSION.toConnectorSession()).evaluate(); if (deterministic) { assertExpressionAndRowExpressionEquals(expressionResult, rowExpressionResult); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestLogicalPlanner.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestLogicalPlanner.java index d3c644b793ca..2117e67cdbcf 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestLogicalPlanner.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestLogicalPlanner.java @@ -1816,7 +1816,8 @@ private void setupJsonFunctionNamespaceManager(QueryRunner queryRunner) ImmutableMap.of( "supported-function-languages", "CPP", "function-implementation-type", "CPP", - "json-based-function-manager.path-to-function-definition", "")); + "json-based-function-manager.path-to-function-definition", ""), + false); } @Test diff --git a/presto-main/src/test/java/com/facebook/presto/type/AbstractTestType.java b/presto-main/src/test/java/com/facebook/presto/type/AbstractTestType.java index deb92a0c5ada..dfc25091c6ae 100644 --- a/presto-main/src/test/java/com/facebook/presto/type/AbstractTestType.java +++ b/presto-main/src/test/java/com/facebook/presto/type/AbstractTestType.java @@ -24,7 +24,6 @@ import com.facebook.presto.common.type.UnknownType; import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.HandleResolver; -import com.facebook.presto.server.ServerConfig; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.sql.analyzer.FunctionsConfig; import com.google.common.collect.ImmutableMap; @@ -71,7 +70,6 @@ public abstract class AbstractTestType blockEncodingSerde, new FeaturesConfig(), new FunctionsConfig(), - new ServerConfig(), new HandleResolver(), ImmutableSet.of()); diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/ContainerQueryRunner.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/ContainerQueryRunner.java index d32eea55c392..364631daac59 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/ContainerQueryRunner.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/ContainerQueryRunner.java @@ -267,7 +267,7 @@ public void createCatalog(String catalogName, String connectorName, Map properties) + public void loadFunctionNamespaceManager(String functionNamespaceManagerName, String catalogName, Map properties, boolean isDefaultNamespace) { throw new UnsupportedOperationException(); } diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/PrestoNativeQueryRunnerUtils.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/PrestoNativeQueryRunnerUtils.java index 825fbb1bc438..8feea7f3d8c1 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/PrestoNativeQueryRunnerUtils.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/PrestoNativeQueryRunnerUtils.java @@ -597,7 +597,8 @@ public static void setupJsonFunctionNamespaceManager(QueryRunner queryRunner, St ImmutableMap.of( "supported-function-languages", "CPP", "function-implementation-type", "CPP", - "json-based-function-manager.path-to-function-definition", jsonDefinitionPath)); + "json-based-function-manager.path-to-function-definition", jsonDefinitionPath), + false); } private static Table createHiveSymlinkTable(String databaseName, String tableName, List columns, String location) diff --git a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/functionNamespace/NativeFunctionNamespaceManager.java b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/functionNamespace/NativeFunctionNamespaceManager.java index bddef6e72e30..024f21d0ec3a 100644 --- a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/functionNamespace/NativeFunctionNamespaceManager.java +++ b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/functionNamespace/NativeFunctionNamespaceManager.java @@ -330,12 +330,10 @@ public final FunctionHandle getFunctionHandle(Optional properties) + public void loadFunctionNamespaceManager(String functionNamespaceManagerName, String catalogName, Map properties, boolean isDefaultNamespace) { - metadata.getFunctionAndTypeManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties, nodeManager); + metadata.getFunctionAndTypeManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties, nodeManager, isDefaultNamespace); } @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 fd1dc967b2a6..e406f0976f60 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 @@ -729,10 +729,10 @@ public void createCatalog(String catalogName, String connectorName, Map properties) + public void loadFunctionNamespaceManager(String functionNamespaceManagerName, String catalogName, Map properties, boolean isDefaultNamespace) { for (TestingPrestoServer server : servers) { - server.getMetadata().getFunctionAndTypeManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties, server.getPluginNodeManager()); + server.getMetadata().getFunctionAndTypeManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties, server.getPluginNodeManager(), isDefaultNamespace); } } @@ -929,7 +929,7 @@ private void enableTestFunctionNamespaces(List catalogNames, Map properties, - boolean coordinatorOnly) + boolean coordinatorOnly, + boolean isDefaultNamespace) { for (TestingPrestoServer server : servers) { if (coordinatorOnly && !server.isCoordinator()) { continue; } - server.getMetadata().getFunctionAndTypeManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties, server.getPluginNodeManager()); + server.getMetadata().getFunctionAndTypeManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties, server.getPluginNodeManager(), isDefaultNamespace); } } 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 0b6409f5dfad..37b53161ad33 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 @@ -245,9 +245,9 @@ public void createCatalog(String catalogName, String connectorName, Map properties) + public void loadFunctionNamespaceManager(String functionNamespaceManagerName, String catalogName, Map properties, boolean isDefaultNamespace) { - server.getMetadata().getFunctionAndTypeManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties, server.getPluginNodeManager()); + server.getMetadata().getFunctionAndTypeManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties, server.getPluginNodeManager(), isDefaultNamespace); } @Override diff --git a/presto-thrift-connector/src/test/java/com/facebook/presto/connector/thrift/integration/ThriftQueryRunner.java b/presto-thrift-connector/src/test/java/com/facebook/presto/connector/thrift/integration/ThriftQueryRunner.java index c9a4f82b7cd8..3da1bd2bfc29 100644 --- a/presto-thrift-connector/src/test/java/com/facebook/presto/connector/thrift/integration/ThriftQueryRunner.java +++ b/presto-thrift-connector/src/test/java/com/facebook/presto/connector/thrift/integration/ThriftQueryRunner.java @@ -291,9 +291,9 @@ public void createCatalog(String catalogName, String connectorName, Map properties) + public void loadFunctionNamespaceManager(String functionNamespaceManagerName, String catalogName, Map properties, boolean isDefaultNamespace) { - source.loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties); + source.loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties, isDefaultNamespace); } @Override 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 3e105f0fd18e..256978486920 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 @@ -22,7 +22,6 @@ import com.facebook.presto.metadata.CatalogManager; import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.metadata.HandleJsonModule; -import com.facebook.presto.server.ServerConfig; import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.sql.analyzer.FunctionsConfig; import com.facebook.presto.sql.parser.SqlParser; @@ -129,7 +128,6 @@ protected final void setup(Binder binder) // type configBinder(binder).bindConfig(FeaturesConfig.class); configBinder(binder).bindConfig(FunctionsConfig.class); - configBinder(binder).bindConfig(ServerConfig.class); binder.bind(TypeManager.class).to(FunctionAndTypeManager.class).in(SINGLETON); newSetBinder(binder, Type.class);