From 7e42bc006905d07f060e442f472ded92ade97c0d Mon Sep 17 00:00:00 2001 From: Leiqing Cai Date: Thu, 6 Feb 2020 13:04:00 -0800 Subject: [PATCH 1/6] Remove dependency of presto-sql-function from presto-main - Rename module presto-sql-function to presto-function-namespace-managers. - Move InMemoryFunctionNamespaceManager and TestFunctionNamespaceManager to presto-function-namespace-managers. --- pom.xml | 8 +------- .../pom.xml | 4 ++-- ...actSqlInvokedFunctionNamespaceManager.java | 2 +- ...InvokedFunctionNamespaceManagerConfig.java | 2 +- ...uidFunctionNamespaceTransactionHandle.java | 2 +- .../InMemoryFunctionNamespaceManager.java | 6 +++--- .../testing/SqlInvokedFunctionTestUtils.java | 2 +- ...estSqlInvokedFunctionNamespaceManager.java | 19 +++++++++---------- ...InvokedFunctionNamespaceManagerConfig.java | 2 +- presto-main/pom.xml | 5 ----- 10 files changed, 20 insertions(+), 32 deletions(-) rename {presto-sql-function => presto-function-namespace-managers}/pom.xml (93%) rename {presto-sql-function/src/main/java/com/facebook/presto/sqlfunction => presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace}/AbstractSqlInvokedFunctionNamespaceManager.java (99%) rename {presto-sql-function/src/main/java/com/facebook/presto/sqlfunction => presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace}/SqlInvokedFunctionNamespaceManagerConfig.java (97%) rename {presto-sql-function/src/main/java/com/facebook/presto/sqlfunction => presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace}/UuidFunctionNamespaceTransactionHandle.java (97%) rename {presto-main/src/main/java/com/facebook/presto => presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace}/testing/InMemoryFunctionNamespaceManager.java (95%) rename {presto-sql-function/src/main/java/com/facebook/presto/sqlfunction => presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace}/testing/SqlInvokedFunctionTestUtils.java (98%) rename presto-main/src/test/java/com/facebook/presto/metadata/TestFunctionNamespaceManager.java => presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/TestSqlInvokedFunctionNamespaceManager.java (91%) rename {presto-sql-function/src/test/java/com/facebook/presto/sqlfunction => presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace}/TestSqlInvokedFunctionNamespaceManagerConfig.java (97%) diff --git a/pom.xml b/pom.xml index 1226e3cc4b573..bcf8a4fb7dd6e 100644 --- a/pom.xml +++ b/pom.xml @@ -145,7 +145,7 @@ presto-proxy presto-kudu presto-elasticsearch - presto-sql-function + presto-function-namespace-managers presto-expressions presto-benchmark-runner presto-spark-classloader-interface @@ -477,12 +477,6 @@ ${project.version} - - com.facebook.presto - presto-sql-function - ${project.version} - - com.facebook.presto presto-spark-classloader-interface diff --git a/presto-sql-function/pom.xml b/presto-function-namespace-managers/pom.xml similarity index 93% rename from presto-sql-function/pom.xml rename to presto-function-namespace-managers/pom.xml index 62ac7065d1029..953941d78b151 100644 --- a/presto-sql-function/pom.xml +++ b/presto-function-namespace-managers/pom.xml @@ -9,8 +9,8 @@ 4.0.0 - presto-sql-function - presto-sql-function + presto-function-namespace-managers + presto-function-namespace-managers ${project.parent.basedir} diff --git a/presto-sql-function/src/main/java/com/facebook/presto/sqlfunction/AbstractSqlInvokedFunctionNamespaceManager.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/AbstractSqlInvokedFunctionNamespaceManager.java similarity index 99% rename from presto-sql-function/src/main/java/com/facebook/presto/sqlfunction/AbstractSqlInvokedFunctionNamespaceManager.java rename to presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/AbstractSqlInvokedFunctionNamespaceManager.java index b28839bb4ada2..06a4f476eb05c 100644 --- a/presto-sql-function/src/main/java/com/facebook/presto/sqlfunction/AbstractSqlInvokedFunctionNamespaceManager.java +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/AbstractSqlInvokedFunctionNamespaceManager.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.sqlfunction; +package com.facebook.presto.functionNamespace; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.function.FunctionMetadata; diff --git a/presto-sql-function/src/main/java/com/facebook/presto/sqlfunction/SqlInvokedFunctionNamespaceManagerConfig.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/SqlInvokedFunctionNamespaceManagerConfig.java similarity index 97% rename from presto-sql-function/src/main/java/com/facebook/presto/sqlfunction/SqlInvokedFunctionNamespaceManagerConfig.java rename to presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/SqlInvokedFunctionNamespaceManagerConfig.java index 194c64ef46ec6..65e79d93c53dd 100644 --- a/presto-sql-function/src/main/java/com/facebook/presto/sqlfunction/SqlInvokedFunctionNamespaceManagerConfig.java +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/SqlInvokedFunctionNamespaceManagerConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.sqlfunction; +package com.facebook.presto.functionNamespace; import com.facebook.airlift.configuration.Config; import io.airlift.units.Duration; diff --git a/presto-sql-function/src/main/java/com/facebook/presto/sqlfunction/UuidFunctionNamespaceTransactionHandle.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/UuidFunctionNamespaceTransactionHandle.java similarity index 97% rename from presto-sql-function/src/main/java/com/facebook/presto/sqlfunction/UuidFunctionNamespaceTransactionHandle.java rename to presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/UuidFunctionNamespaceTransactionHandle.java index 609b338b45ed6..3e813005fac4a 100644 --- a/presto-sql-function/src/main/java/com/facebook/presto/sqlfunction/UuidFunctionNamespaceTransactionHandle.java +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/UuidFunctionNamespaceTransactionHandle.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.sqlfunction; +package com.facebook.presto.functionNamespace; import com.facebook.presto.spi.function.FunctionNamespaceTransactionHandle; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/presto-main/src/main/java/com/facebook/presto/testing/InMemoryFunctionNamespaceManager.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/InMemoryFunctionNamespaceManager.java similarity index 95% rename from presto-main/src/main/java/com/facebook/presto/testing/InMemoryFunctionNamespaceManager.java rename to presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/InMemoryFunctionNamespaceManager.java index 13130709a7bc1..41d82f732bfb0 100644 --- a/presto-main/src/main/java/com/facebook/presto/testing/InMemoryFunctionNamespaceManager.java +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/InMemoryFunctionNamespaceManager.java @@ -11,8 +11,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.testing; +package com.facebook.presto.functionNamespace.testing; +import com.facebook.presto.functionNamespace.AbstractSqlInvokedFunctionNamespaceManager; +import com.facebook.presto.functionNamespace.SqlInvokedFunctionNamespaceManagerConfig; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.function.AlterRoutineCharacteristics; import com.facebook.presto.spi.function.FunctionMetadata; @@ -22,8 +24,6 @@ import com.facebook.presto.spi.function.SqlFunctionId; import com.facebook.presto.spi.function.SqlInvokedFunction; import com.facebook.presto.spi.type.TypeSignature; -import com.facebook.presto.sqlfunction.AbstractSqlInvokedFunctionNamespaceManager; -import com.facebook.presto.sqlfunction.SqlInvokedFunctionNamespaceManagerConfig; import javax.annotation.concurrent.ThreadSafe; diff --git a/presto-sql-function/src/main/java/com/facebook/presto/sqlfunction/testing/SqlInvokedFunctionTestUtils.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java similarity index 98% rename from presto-sql-function/src/main/java/com/facebook/presto/sqlfunction/testing/SqlInvokedFunctionTestUtils.java rename to presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java index 107657b83ff7f..6f73026446597 100644 --- a/presto-sql-function/src/main/java/com/facebook/presto/sqlfunction/testing/SqlInvokedFunctionTestUtils.java +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.sqlfunction.testing; +package com.facebook.presto.functionNamespace.testing; import com.facebook.presto.spi.CatalogSchemaName; import com.facebook.presto.spi.function.QualifiedFunctionName; diff --git a/presto-main/src/test/java/com/facebook/presto/metadata/TestFunctionNamespaceManager.java b/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/TestSqlInvokedFunctionNamespaceManager.java similarity index 91% rename from presto-main/src/test/java/com/facebook/presto/metadata/TestFunctionNamespaceManager.java rename to presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/TestSqlInvokedFunctionNamespaceManager.java index 60f9438c0e290..b1dab49144f8c 100644 --- a/presto-main/src/test/java/com/facebook/presto/metadata/TestFunctionNamespaceManager.java +++ b/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/TestSqlInvokedFunctionNamespaceManager.java @@ -11,15 +11,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.metadata; +package com.facebook.presto.functionNamespace; +import com.facebook.presto.functionNamespace.testing.InMemoryFunctionNamespaceManager; import com.facebook.presto.spi.ErrorCodeSupplier; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.function.FunctionMetadata; import com.facebook.presto.spi.function.FunctionNamespaceTransactionHandle; import com.facebook.presto.spi.function.SqlInvokedFunction; -import com.facebook.presto.sqlfunction.SqlInvokedFunctionNamespaceManagerConfig; -import com.facebook.presto.testing.InMemoryFunctionNamespaceManager; import com.google.common.collect.ImmutableSet; import io.airlift.units.Duration; import org.testng.annotations.Test; @@ -27,21 +26,21 @@ import java.util.Collection; import java.util.Optional; +import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.FUNCTION_POWER_TOWER_DOUBLE; +import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.FUNCTION_POWER_TOWER_DOUBLE_UPDATED; +import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.FUNCTION_POWER_TOWER_INT; +import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.POWER_TOWER; import static com.facebook.presto.spi.StandardErrorCode.GENERIC_USER_ERROR; -import static com.facebook.presto.sqlfunction.testing.SqlInvokedFunctionTestUtils.FUNCTION_POWER_TOWER_DOUBLE; -import static com.facebook.presto.sqlfunction.testing.SqlInvokedFunctionTestUtils.FUNCTION_POWER_TOWER_DOUBLE_UPDATED; -import static com.facebook.presto.sqlfunction.testing.SqlInvokedFunctionTestUtils.FUNCTION_POWER_TOWER_INT; -import static com.facebook.presto.sqlfunction.testing.SqlInvokedFunctionTestUtils.POWER_TOWER; -import static com.facebook.presto.testing.assertions.Assert.assertEquals; import static com.google.common.collect.Iterables.getOnlyElement; import static java.lang.String.format; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotSame; import static org.testng.Assert.assertSame; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; -public class TestFunctionNamespaceManager +public class TestSqlInvokedFunctionNamespaceManager { @Test public void testCreateFunction() @@ -123,7 +122,7 @@ public void testCaching() assertEquals(function1, function2); assertNotSame(function1, function2); - // fetchFunctionMetadataDirect does not produce the same metdata reference + // fetchFunctionMetadataDirect does not produce the same metadata reference FunctionMetadata metadata1 = functionNamespaceManager.fetchFunctionMetadataDirect(function1.getRequiredFunctionHandle()); FunctionMetadata metadata2 = functionNamespaceManager.fetchFunctionMetadataDirect(function2.getRequiredFunctionHandle()); assertEquals(metadata1, metadata2); diff --git a/presto-sql-function/src/test/java/com/facebook/presto/sqlfunction/TestSqlInvokedFunctionNamespaceManagerConfig.java b/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/TestSqlInvokedFunctionNamespaceManagerConfig.java similarity index 97% rename from presto-sql-function/src/test/java/com/facebook/presto/sqlfunction/TestSqlInvokedFunctionNamespaceManagerConfig.java rename to presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/TestSqlInvokedFunctionNamespaceManagerConfig.java index 46d4510886645..f72ad0ab9215c 100644 --- a/presto-sql-function/src/test/java/com/facebook/presto/sqlfunction/TestSqlInvokedFunctionNamespaceManagerConfig.java +++ b/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/TestSqlInvokedFunctionNamespaceManagerConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.sqlfunction; +package com.facebook.presto.functionNamespace; import com.google.common.collect.ImmutableMap; import io.airlift.units.Duration; diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 9aea0928a7376..481ad68764ae9 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -71,11 +71,6 @@ presto-memory-context - - com.facebook.presto - presto-sql-function - - com.facebook.presto presto-expressions From 9a25731ee5f73c0eb0a8dfdfb515f23132f4e956 Mon Sep 17 00:00:00 2001 From: Leiqing Cai Date: Thu, 6 Feb 2020 13:04:00 -0800 Subject: [PATCH 2/6] Require function namespace manager to bind with catalog names --- ...actSqlInvokedFunctionNamespaceManager.java | 29 ++++++++++- .../InMemoryFunctionNamespaceManager.java | 4 +- .../testing/SqlInvokedFunctionTestUtils.java | 4 +- ...estSqlInvokedFunctionNamespaceManager.java | 4 +- .../presto/metadata/FunctionManager.java | 51 ++++--------------- .../StaticFunctionNamespaceStore.java | 10 ++-- .../DelegatingTransactionManager.java | 8 +-- .../InMemoryTransactionManager.java | 18 +++---- .../transaction/NoOpTransactionManager.java | 2 +- .../transaction/TransactionManager.java | 4 +- .../FunctionNamespaceManagerFactory.java | 2 +- .../presto/tests/DistributedQueryRunner.java | 4 +- 12 files changed, 69 insertions(+), 71 deletions(-) diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/AbstractSqlInvokedFunctionNamespaceManager.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/AbstractSqlInvokedFunctionNamespaceManager.java index 06a4f476eb05c..24b07c5ec01ec 100644 --- a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/AbstractSqlInvokedFunctionNamespaceManager.java +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/AbstractSqlInvokedFunctionNamespaceManager.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.functionNamespace; +import com.facebook.presto.spi.CatalogSchemaName; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.function.FunctionMetadata; import com.facebook.presto.spi.function.FunctionNamespaceManager; @@ -43,6 +44,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.MILLISECONDS; public abstract class AbstractSqlInvokedFunctionNamespaceManager @@ -50,12 +52,14 @@ public abstract class AbstractSqlInvokedFunctionNamespaceManager { private final ConcurrentMap transactions = new ConcurrentHashMap<>(); + private final String catalogName; private final LoadingCache> functions; private final LoadingCache metadataByHandle; private final LoadingCache implementationByHandle; - public AbstractSqlInvokedFunctionNamespaceManager(SqlInvokedFunctionNamespaceManagerConfig config) + public AbstractSqlInvokedFunctionNamespaceManager(String catalogName, SqlInvokedFunctionNamespaceManagerConfig config) { + this.catalogName = requireNonNull(catalogName, "catalogName is null"); this.functions = CacheBuilder.newBuilder() .expireAfterWrite(config.getFunctionCacheExpiration().toMillis(), MILLISECONDS) .build(new CacheLoader>() @@ -125,6 +129,7 @@ public final void abort(FunctionNamespaceTransactionHandle transactionHandle) @Override public final Collection getFunctions(Optional transactionHandle, QualifiedFunctionName functionName) { + checkCatalog(functionName); checkArgument(transactionHandle.isPresent(), "missing transactionHandle"); return transactions.get(transactionHandle.get()).loadAndGetFunctionsTransactional(functionName); } @@ -132,6 +137,7 @@ public final Collection getFunctions(Optional transactionHandle, Signature signature) { + checkCatalog(signature.getName()); checkArgument(transactionHandle.isPresent(), "missing transactionHandle"); // This is the only assumption in this class that we're dealing with sql-invoked regular function. SqlFunctionId functionId = new SqlFunctionId(signature.getName(), signature.getArgumentTypes()); @@ -141,6 +147,7 @@ public final FunctionHandle getFunctionHandle(Optional latestFunctions = new ConcurrentHashMap<>(); - public InMemoryFunctionNamespaceManager(SqlInvokedFunctionNamespaceManagerConfig config) + public InMemoryFunctionNamespaceManager(String catalogName, SqlInvokedFunctionNamespaceManagerConfig config) { - super(config); + super(catalogName, config); } @Override diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java index 6f73026446597..a1fc2184ad4ef 100644 --- a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java @@ -34,7 +34,9 @@ private SqlInvokedFunctionTestUtils() { } - public static final QualifiedFunctionName POWER_TOWER = QualifiedFunctionName.of(new CatalogSchemaName("unittest", "memory"), "power_tower"); + public static final String TEST_CATALOG = "unittest"; + + public static final QualifiedFunctionName POWER_TOWER = QualifiedFunctionName.of(new CatalogSchemaName(TEST_CATALOG, "memory"), "power_tower"); public static final SqlInvokedFunction FUNCTION_POWER_TOWER_DOUBLE = new SqlInvokedFunction( POWER_TOWER, diff --git a/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/TestSqlInvokedFunctionNamespaceManager.java b/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/TestSqlInvokedFunctionNamespaceManager.java index b1dab49144f8c..3d9bd34fdc289 100644 --- a/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/TestSqlInvokedFunctionNamespaceManager.java +++ b/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/TestSqlInvokedFunctionNamespaceManager.java @@ -30,6 +30,7 @@ import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.FUNCTION_POWER_TOWER_DOUBLE_UPDATED; import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.FUNCTION_POWER_TOWER_INT; import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.POWER_TOWER; +import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.TEST_CATALOG; import static com.facebook.presto.spi.StandardErrorCode.GENERIC_USER_ERROR; import static com.google.common.collect.Iterables.getOnlyElement; import static java.lang.String.format; @@ -75,6 +76,7 @@ public void testCreateFunctionFailed() public void testTransactionalGetFunction() { InMemoryFunctionNamespaceManager functionNamespaceManager = new InMemoryFunctionNamespaceManager( + TEST_CATALOG, new SqlInvokedFunctionNamespaceManagerConfig() .setFunctionCacheExpiration(new Duration(0, MILLISECONDS)) .setFunctionInstanceCacheExpiration(new Duration(0, MILLISECONDS))); @@ -145,7 +147,7 @@ public void testCaching() private static InMemoryFunctionNamespaceManager createFunctionNamespaceManager() { - return new InMemoryFunctionNamespaceManager(new SqlInvokedFunctionNamespaceManagerConfig()); + return new InMemoryFunctionNamespaceManager(TEST_CATALOG, new SqlInvokedFunctionNamespaceManagerConfig()); } private static void assertPrestoException(Runnable runnable, ErrorCodeSupplier expectedErrorCode, String expectedMessageRegex) diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/FunctionManager.java b/presto-main/src/main/java/com/facebook/presto/metadata/FunctionManager.java index fb66a4d7403cf..4b77da433dce1 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/FunctionManager.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/FunctionManager.java @@ -22,7 +22,6 @@ import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.block.BlockEncodingSerde; import com.facebook.presto.spi.function.AlterRoutineCharacteristics; -import com.facebook.presto.spi.function.CatalogSchemaPrefix; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.function.FunctionKind; import com.facebook.presto.spi.function.FunctionMetadata; @@ -110,7 +109,6 @@ public class FunctionManager private final FunctionInvokerProvider functionInvokerProvider; private final Map functionNamespaceManagerFactories = new ConcurrentHashMap<>(); private final HandleResolver handleResolver; - private final Map functionNamespaces = new ConcurrentHashMap<>(); private final Map> functionNamespaceManagers = new ConcurrentHashMap<>(); private final LoadingCache functionCache; private final CacheStatsMBean cacheStatsMBean; @@ -126,15 +124,14 @@ public FunctionManager( this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); this.builtInFunctionNamespaceManager = new BuiltInFunctionNamespaceManager(typeManager, blockEncodingSerde, featuresConfig, this); - this.functionNamespaces.put(DEFAULT_NAMESPACE.asCatalogSchemaPrefix(), BuiltInFunctionNamespaceManager.ID); - this.functionNamespaceManagers.put(BuiltInFunctionNamespaceManager.ID, builtInFunctionNamespaceManager); + this.functionNamespaceManagers.put(DEFAULT_NAMESPACE.getCatalogName(), builtInFunctionNamespaceManager); this.functionInvokerProvider = new FunctionInvokerProvider(this); this.handleResolver = handleResolver; if (typeManager instanceof TypeRegistry) { ((TypeRegistry) typeManager).setFunctionManager(this); } // TODO: Provide a more encapsulated way for TransactionManager to register FunctionNamespaceManager - transactionManager.registerFunctionNamespaceManager(BuiltInFunctionNamespaceManager.ID, builtInFunctionNamespaceManager); + transactionManager.registerFunctionNamespaceManager(DEFAULT_NAMESPACE.getCatalogName(), builtInFunctionNamespaceManager); this.functionCache = CacheBuilder.newBuilder() .recordStats() .maximumSize(1000) @@ -158,23 +155,17 @@ public CacheStatsMBean getFunctionResolutionCacheStats() public void loadFunctionNamespaceManager( String functionNamespaceManagerName, - String functionNamespaceManagerId, - List catalogSchemaPrefixes, + String catalogName, Map properties) { requireNonNull(functionNamespaceManagerName, "functionNamespaceManagerName is null"); FunctionNamespaceManagerFactory factory = functionNamespaceManagerFactories.get(functionNamespaceManagerName); checkState(factory != null, "No factory for function namespace manager %s", functionNamespaceManagerName); - FunctionNamespaceManager functionNamespaceManager = factory.create(properties); + FunctionNamespaceManager functionNamespaceManager = factory.create(catalogName, properties); - transactionManager.registerFunctionNamespaceManager(functionNamespaceManagerId, functionNamespaceManager); - if (functionNamespaceManagers.putIfAbsent(functionNamespaceManagerId, functionNamespaceManager) != null) { - throw new IllegalArgumentException(format("Function namespace manager [%s] is already registered", functionNamespaceManagerId)); - } - for (String catalogSchemaPrefix : catalogSchemaPrefixes) { - if (functionNamespaces.putIfAbsent(CatalogSchemaPrefix.of(catalogSchemaPrefix), functionNamespaceManagerId) != null) { - throw new IllegalArgumentException(format("Function namespace [%s] is already registered to function namespace manager [%s]", catalogSchemaPrefix, functionNamespaceManagerId)); - } + transactionManager.registerFunctionNamespaceManager(catalogName, functionNamespaceManager); + if (functionNamespaceManagers.putIfAbsent(catalogName, functionNamespaceManager) != null) { + throw new IllegalArgumentException(format("Function namespace manager is already registered for catalog [%s]", catalogName)); } } @@ -272,14 +263,13 @@ public FunctionHandle resolveFunction(Optional transactionId, Qua private FunctionHandle resolveFunctionInternal(Optional transactionId, QualifiedFunctionName functionName, List parameterTypes) { - Optional functionNamespaceManagerId = getServingFunctionNamespaceManagerId(functionName.getFunctionNamespace()); - if (!functionNamespaceManagerId.isPresent()) { + 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, functionNamespaceManagerId.get())); - FunctionNamespaceManager functionNamespaceManager = functionNamespaceManagers.get(functionNamespaceManagerId.get()); + .map(id -> transactionManager.getFunctionNamespaceTransaction(id, functionName.getFunctionNamespace().getCatalogName())); Collection candidates = functionNamespaceManager.getFunctions(transactionHandle, functionName); try { @@ -455,26 +445,7 @@ private FunctionHandle lookupFunction( private Optional> getServingFunctionNamespaceManager(CatalogSchemaName functionNamespace) { - return getServingFunctionNamespaceManagerId(functionNamespace).map(functionNamespaceManagers::get); - } - - private Optional getServingFunctionNamespaceManagerId(CatalogSchemaName functionNamespace) - { - if (functionNamespace.equals(DEFAULT_NAMESPACE)) { - return Optional.of(BuiltInFunctionNamespaceManager.ID); - } - - CatalogSchemaPrefix bestMatch = null; - String servingFunctionNamespaceManagerId = null; - - for (Map.Entry entry : functionNamespaces.entrySet()) { - CatalogSchemaPrefix prefix = entry.getKey(); - if (prefix.includes(functionNamespace) && (bestMatch == null || bestMatch.includes(prefix))) { - bestMatch = prefix; - servingFunctionNamespaceManagerId = entry.getValue(); - } - } - return Optional.ofNullable(servingFunctionNamespaceManagerId); + return Optional.ofNullable(functionNamespaceManagers.get(functionNamespace.getCatalogName())); } private String constructFunctionNotFoundErrorMessage(QualifiedFunctionName functionName, List parameterTypes, Collection candidates) 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 d5004d5f077fb..0ec39dc141599 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 @@ -62,19 +62,15 @@ public void loadFunctionNamespaceManagers() private void loadFunctionNamespaceManager(File file) throws Exception { - String functionNamespaceManagerId = getNameWithoutExtension(file.getName()); - + String catalogName = getNameWithoutExtension(file.getName()); log.info("-- Loading function namespace manager from %s --", file); Map properties = new HashMap<>(loadProperties(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()); - String functionNamespaces = properties.remove("serving-namespaces"); - checkState(functionNamespaces != null, "Function namespace configuration %s does not contain serving-namespaces", file.getAbsoluteFile()); - - functionManager.loadFunctionNamespaceManager(functionNamespaceManagerName, functionNamespaceManagerId, SPLITTER.splitToList(functionNamespaces), properties); - log.info("-- Added function namespaces [%s] using function namespace manager [%s] --", functionNamespaces, functionNamespaceManagerId); + functionManager.loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties); + log.info("-- Added function namespace manager [%s] --", catalogName); } private static List listFiles(File dir) diff --git a/presto-main/src/main/java/com/facebook/presto/transaction/DelegatingTransactionManager.java b/presto-main/src/main/java/com/facebook/presto/transaction/DelegatingTransactionManager.java index bae513607222d..117efac282cdb 100644 --- a/presto-main/src/main/java/com/facebook/presto/transaction/DelegatingTransactionManager.java +++ b/presto-main/src/main/java/com/facebook/presto/transaction/DelegatingTransactionManager.java @@ -153,14 +153,14 @@ public void activateTransaction(Session session, boolean transactionControl, Acc } @Override - public void registerFunctionNamespaceManager(String functionNamespaceManagerName, FunctionNamespaceManager functionNamespaceManager) + public void registerFunctionNamespaceManager(String catalogName, FunctionNamespaceManager functionNamespaceManager) { - delegate.registerFunctionNamespaceManager(functionNamespaceManagerName, functionNamespaceManager); + delegate.registerFunctionNamespaceManager(catalogName, functionNamespaceManager); } @Override - public FunctionNamespaceTransactionHandle getFunctionNamespaceTransaction(TransactionId transactionId, String functionNamespaceManagerName) + public FunctionNamespaceTransactionHandle getFunctionNamespaceTransaction(TransactionId transactionId, String catalogName) { - return delegate.getFunctionNamespaceTransaction(transactionId, functionNamespaceManagerName); + return delegate.getFunctionNamespaceTransaction(transactionId, catalogName); } } diff --git a/presto-main/src/main/java/com/facebook/presto/transaction/InMemoryTransactionManager.java b/presto-main/src/main/java/com/facebook/presto/transaction/InMemoryTransactionManager.java index 02f18f5cc2c9c..4106f9b39aabf 100644 --- a/presto-main/src/main/java/com/facebook/presto/transaction/InMemoryTransactionManager.java +++ b/presto-main/src/main/java/com/facebook/presto/transaction/InMemoryTransactionManager.java @@ -235,16 +235,16 @@ public ConnectorTransactionHandle getConnectorTransaction(TransactionId transact } @Override - public synchronized void registerFunctionNamespaceManager(String functionNamespaceManagerId, FunctionNamespaceManager functionNamespaceManager) + public synchronized void registerFunctionNamespaceManager(String catalogName, FunctionNamespaceManager functionNamespaceManager) { - checkArgument(!functionNamespaceManagers.containsKey(functionNamespaceManagerId), "FunctionNamespaceManager %s is already registered", functionNamespaceManagerId); - functionNamespaceManagers.put(functionNamespaceManagerId, functionNamespaceManager); + checkArgument(!functionNamespaceManagers.containsKey(catalogName), "FunctionNamespaceManager is already registered for catalog [%s]", catalogName); + functionNamespaceManagers.put(catalogName, functionNamespaceManager); } @Override - public FunctionNamespaceTransactionHandle getFunctionNamespaceTransaction(TransactionId transactionId, String functionNamespaceManagerId) + public FunctionNamespaceTransactionHandle getFunctionNamespaceTransaction(TransactionId transactionId, String catalogName) { - return getTransactionMetadata(transactionId).getFunctionNamespaceTransaction(functionNamespaceManagerId).getTransactionHandle(); + return getTransactionMetadata(transactionId).getFunctionNamespaceTransaction(catalogName).getTransactionHandle(); } private void checkConnectorWrite(TransactionId transactionId, ConnectorId connectorId) @@ -472,14 +472,14 @@ private synchronized CatalogMetadata getTransactionCatalogMetadata(ConnectorId c return catalogMetadata; } - private synchronized FunctionNamespaceTransactionMetadata getFunctionNamespaceTransaction(String functionNamespaceManagerId) + private synchronized FunctionNamespaceTransactionMetadata getFunctionNamespaceTransaction(String catalogName) { checkOpenTransaction(); return functionNamespaceTransactions.computeIfAbsent( - functionNamespaceManagerId, id -> { - verify(id != null, "Unknown function namespace manager: %s", id); - FunctionNamespaceManager functionNamespaceManager = functionNamespaceManagers.get(id); + catalogName, catalog -> { + verify(catalog != null, "catalog is null"); + FunctionNamespaceManager functionNamespaceManager = functionNamespaceManagers.get(catalog); FunctionNamespaceTransactionHandle transactionHandle = functionNamespaceManager.beginTransaction(); return new FunctionNamespaceTransactionMetadata(functionNamespaceManager, transactionHandle); }); diff --git a/presto-main/src/main/java/com/facebook/presto/transaction/NoOpTransactionManager.java b/presto-main/src/main/java/com/facebook/presto/transaction/NoOpTransactionManager.java index d989f57d9f4b7..8145036a5efa6 100644 --- a/presto-main/src/main/java/com/facebook/presto/transaction/NoOpTransactionManager.java +++ b/presto-main/src/main/java/com/facebook/presto/transaction/NoOpTransactionManager.java @@ -98,7 +98,7 @@ public ConnectorTransactionHandle getConnectorTransaction(TransactionId transact } @Override - public void registerFunctionNamespaceManager(String functionNamespaceManagerName, FunctionNamespaceManager functionNamespaceManager) + public void registerFunctionNamespaceManager(String catalogName, FunctionNamespaceManager functionNamespaceManager) { } diff --git a/presto-main/src/main/java/com/facebook/presto/transaction/TransactionManager.java b/presto-main/src/main/java/com/facebook/presto/transaction/TransactionManager.java index 43e143160f490..522953bf40f1e 100644 --- a/presto-main/src/main/java/com/facebook/presto/transaction/TransactionManager.java +++ b/presto-main/src/main/java/com/facebook/presto/transaction/TransactionManager.java @@ -54,9 +54,9 @@ public interface TransactionManager ConnectorTransactionHandle getConnectorTransaction(TransactionId transactionId, ConnectorId connectorId); - void registerFunctionNamespaceManager(String functionNamespaceManagerName, FunctionNamespaceManager functionNamespaceManager); + void registerFunctionNamespaceManager(String catalogNames, FunctionNamespaceManager functionNamespaceManager); - FunctionNamespaceTransactionHandle getFunctionNamespaceTransaction(TransactionId transactionId, String functionNamespaceManagerName); + FunctionNamespaceTransactionHandle getFunctionNamespaceTransaction(TransactionId transactionId, String catalogName); void checkAndSetActive(TransactionId transactionId); diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/function/FunctionNamespaceManagerFactory.java b/presto-spi/src/main/java/com/facebook/presto/spi/function/FunctionNamespaceManagerFactory.java index fb388ca9264ce..eaa5e7555de79 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/function/FunctionNamespaceManagerFactory.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/function/FunctionNamespaceManagerFactory.java @@ -21,5 +21,5 @@ public interface FunctionNamespaceManagerFactory FunctionHandleResolver getHandleResolver(); - FunctionNamespaceManager create(Map config); + FunctionNamespaceManager create(String catalogName, Map config); } 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 e17044aef7096..f4afe0a99433c 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 @@ -335,10 +335,10 @@ public void createCatalog(String catalogName, String connectorName, Map catalogSchemaPrefix, Map properties) + public void loadFunctionNamespaceManager(String functionNamespaceManagerName, String catalogName, Map properties) { for (TestingPrestoServer server : servers) { - server.getMetadata().getFunctionManager().loadFunctionNamespaceManager(functionNamespaceManagerName, functionNamespaceManagerId, catalogSchemaPrefix, properties); + server.getMetadata().getFunctionManager().loadFunctionNamespaceManager(functionNamespaceManagerName, catalogName, properties); } } From e49addd0a7a98d489b5c696f2cb5dc2f2d849602 Mon Sep 17 00:00:00 2001 From: Leiqing Cai Date: Thu, 6 Feb 2020 13:04:00 -0800 Subject: [PATCH 3/6] Use Return statement as the body for SQL-invoked functions --- .../testing/SqlInvokedFunctionTestUtils.java | 6 +- .../sql/analyzer/StatementAnalyzer.java | 4 +- .../sql/relational/SqlFunctionUtils.java | 3 +- .../com/facebook/presto/sql/parser/SqlBase.g4 | 8 ++ .../com/facebook/presto/sql/SqlFormatter.java | 15 +++- .../presto/sql/parser/AstBuilder.java | 17 +++- .../facebook/presto/sql/parser/SqlParser.java | 6 ++ .../facebook/presto/sql/tree/AstVisitor.java | 5 ++ .../presto/sql/tree/CreateFunction.java | 10 +-- .../com/facebook/presto/sql/tree/Return.java | 89 +++++++++++++++++++ .../presto/sql/parser/TestSqlParser.java | 7 +- 11 files changed, 152 insertions(+), 18 deletions(-) create mode 100644 presto-parser/src/main/java/com/facebook/presto/sql/tree/Return.java diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java index a1fc2184ad4ef..d5d81dfde5af0 100644 --- a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java @@ -44,7 +44,7 @@ private SqlInvokedFunctionTestUtils() parseTypeSignature(DOUBLE), "power tower", RoutineCharacteristics.builder().setDeterminism(DETERMINISTIC).build(), - "pow(x, x)", + "RETURN pow(x, x)", Optional.empty()); public static final SqlInvokedFunction FUNCTION_POWER_TOWER_DOUBLE_UPDATED = new SqlInvokedFunction( @@ -53,7 +53,7 @@ private SqlInvokedFunctionTestUtils() parseTypeSignature(DOUBLE), "power tower", RoutineCharacteristics.builder().setDeterminism(DETERMINISTIC).setNullCallClause(RETURNS_NULL_ON_NULL_INPUT).build(), - "pow(x, x)", + "RETURN pow(x, x)", Optional.empty()); public static final SqlInvokedFunction FUNCTION_POWER_TOWER_INT = new SqlInvokedFunction( @@ -62,6 +62,6 @@ private SqlInvokedFunctionTestUtils() parseTypeSignature(INTEGER), "power tower", RoutineCharacteristics.builder().setDeterminism(DETERMINISTIC).setNullCallClause(RETURNS_NULL_ON_NULL_INPUT).build(), - "pow(x, x)", + "RETURN pow(x, x)", Optional.empty()); } 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 ae16b61558ff8..6eaf1965fcc65 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 @@ -580,12 +580,12 @@ protected Scope visitCreateFunction(CreateFunction node, Optional scope) Scope functionScope = Scope.builder() .withRelationType(RelationId.anonymous(), new RelationType(fields)) .build(); - Type bodyType = analyzeExpression(node.getBody(), functionScope).getExpressionTypes().get(NodeRef.of(node.getBody())); + Type bodyType = analyzeExpression(node.getBody().getExpression(), functionScope).getExpressionTypes().get(NodeRef.of(node.getBody().getExpression())); if (!bodyType.equals(returnType)) { throw new SemanticException(TYPE_MISMATCH, node, "Function implementation type '%s' does not match declared return type '%s'", bodyType, returnType); } - Analyzer.verifyNoAggregateWindowOrGroupingFunctions(analysis.getFunctionHandles(), metadata.getFunctionManager(), node.getBody(), "CREATE FUNCTION body"); + Analyzer.verifyNoAggregateWindowOrGroupingFunctions(analysis.getFunctionHandles(), metadata.getFunctionManager(), node.getBody().getExpression(), "CREATE FUNCTION body"); // TODO: Check body contains no SQL invoked functions 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 cc65596f1bbbf..73e79f4f75004 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 @@ -104,7 +104,8 @@ private static Expression parseSqlFunctionExpression(SqlInvokedScalarFunctionImp ParsingOptions parsingOptions = ParsingOptions.builder() .setDecimalLiteralTreatment(sqlFunctionProperties.isParseDecimalLiteralAsDouble() ? AS_DOUBLE : AS_DECIMAL) .build(); - return new SqlParser().createExpression(functionImplementation.getImplementation(), parsingOptions); + // TODO: Use injector-created SqlParser, which could potentially be different from the adhoc SqlParser. + return new SqlParser().createRoutineBody(functionImplementation.getImplementation(), parsingOptions).getExpression(); } private static Map getFunctionArgumentTypes(FunctionMetadata functionMetadata, Metadata metadata) diff --git a/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 b/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 index b8c4eb20e2437..4025fd8401752 100644 --- a/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 +++ b/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 @@ -26,6 +26,10 @@ standaloneExpression : expression EOF ; +standaloneRoutineBody + : routineBody EOF + ; + statement : query #statementDefault | USE schema=identifier #use @@ -170,6 +174,10 @@ alterRoutineCharacteristic ; routineBody + : returnStatement + ; + +returnStatement : RETURN expression ; diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/SqlFormatter.java b/presto-parser/src/main/java/com/facebook/presto/sql/SqlFormatter.java index 6d3c22b3b3e1c..7e9b64b1e6687 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/SqlFormatter.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/SqlFormatter.java @@ -74,6 +74,7 @@ import com.facebook.presto.sql.tree.RenameSchema; import com.facebook.presto.sql.tree.RenameTable; import com.facebook.presto.sql.tree.ResetSession; +import com.facebook.presto.sql.tree.Return; import com.facebook.presto.sql.tree.Revoke; import com.facebook.presto.sql.tree.RevokeRoles; import com.facebook.presto.sql.tree.Rollback; @@ -570,8 +571,9 @@ protected Void visitCreateFunction(CreateFunction node, Integer indent) } builder.append("\n") .append(formatRoutineCharacteristics(node.getCharacteristics())) - .append("\nRETURN ") - .append(formatExpression(node.getBody(), parameters)); + .append("\n"); + + process(node.getBody(), 0); return null; } @@ -601,6 +603,15 @@ protected Void visitDropFunction(DropFunction node, Integer indent) return null; } + @Override + protected Void visitReturn(Return node, Integer indent) + { + append(indent, "RETURN "); + builder.append(formatExpression(node.getExpression(), parameters)); + + return null; + } + @Override protected Void visitDropView(DropView node, Integer context) { diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/parser/AstBuilder.java b/presto-parser/src/main/java/com/facebook/presto/sql/parser/AstBuilder.java index 70925d1faf874..0a27edd17e563 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/parser/AstBuilder.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/parser/AstBuilder.java @@ -118,6 +118,7 @@ import com.facebook.presto.sql.tree.RenameSchema; import com.facebook.presto.sql.tree.RenameTable; import com.facebook.presto.sql.tree.ResetSession; +import com.facebook.presto.sql.tree.Return; import com.facebook.presto.sql.tree.Revoke; import com.facebook.presto.sql.tree.RevokeRoles; import com.facebook.presto.sql.tree.Rollback; @@ -218,6 +219,12 @@ public Node visitStandaloneExpression(SqlBaseParser.StandaloneExpressionContext return visit(context.expression()); } + @Override + public Node visitStandaloneRoutineBody(SqlBaseParser.StandaloneRoutineBodyContext context) + { + return visit(context.routineBody()); + } + // ******************* statements ********************** @Override @@ -421,7 +428,7 @@ public Node visitCreateFunction(SqlBaseParser.CreateFunctionContext context) getType(context.returnType), comment, getRoutineCharacteristics(context.routineCharacteristics()), - (Expression) visit(context.routineBody())); + (Return) visit(context.routineBody())); } @Override @@ -445,7 +452,13 @@ public Node visitDropFunction(SqlBaseParser.DropFunctionContext context) @Override public Node visitRoutineBody(SqlBaseParser.RoutineBodyContext context) { - return visit(context.expression()); + return visit(context.returnStatement()); + } + + @Override + public Node visitReturnStatement(SqlBaseParser.ReturnStatementContext context) + { + return new Return((Expression) visit(context.expression())); } @Override diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/parser/SqlParser.java b/presto-parser/src/main/java/com/facebook/presto/sql/parser/SqlParser.java index b84838068d15c..cd7e209ea1cdb 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/parser/SqlParser.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/parser/SqlParser.java @@ -15,6 +15,7 @@ import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.Node; +import com.facebook.presto.sql.tree.Return; import com.facebook.presto.sql.tree.Statement; import org.antlr.v4.runtime.BaseErrorListener; import org.antlr.v4.runtime.CharStreams; @@ -112,6 +113,11 @@ public Expression createExpression(String expression, ParsingOptions parsingOpti return (Expression) invokeParser("expression", expression, SqlBaseParser::standaloneExpression, parsingOptions); } + public Return createRoutineBody(String routineBody, ParsingOptions parsingOptions) + { + return (Return) invokeParser("routineBody", routineBody, SqlBaseParser::standaloneRoutineBody, parsingOptions); + } + private Node invokeParser(String name, String sql, Function parseFunction, ParsingOptions parsingOptions) { try { diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/AstVisitor.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/AstVisitor.java index f7ddacbf642db..c3e5123f7c7a2 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/AstVisitor.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/AstVisitor.java @@ -761,4 +761,9 @@ protected R visitCurrentUser(CurrentUser node, C context) { return visitExpression(node, context); } + + protected R visitReturn(Return node, C context) + { + return visitStatement(node, context); + } } diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/CreateFunction.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/CreateFunction.java index 2031da7b45db7..02214a5265d4b 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/CreateFunction.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/CreateFunction.java @@ -31,19 +31,19 @@ public class CreateFunction private final String returnType; private final Optional comment; private final RoutineCharacteristics characteristics; - private final Expression body; + private final Return body; - public CreateFunction(QualifiedName functionName, boolean replace, List parameters, String returnType, Optional comment, RoutineCharacteristics characteristics, Expression body) + public CreateFunction(QualifiedName functionName, boolean replace, List parameters, String returnType, Optional comment, RoutineCharacteristics characteristics, Return body) { this(Optional.empty(), replace, functionName, parameters, returnType, comment, characteristics, body); } - public CreateFunction(NodeLocation location, boolean replace, QualifiedName functionName, List parameters, String returnType, Optional comment, RoutineCharacteristics characteristics, Expression body) + public CreateFunction(NodeLocation location, boolean replace, QualifiedName functionName, List parameters, String returnType, Optional comment, RoutineCharacteristics characteristics, Return body) { this(Optional.of(location), replace, functionName, parameters, returnType, comment, characteristics, body); } - private CreateFunction(Optional location, boolean replace, QualifiedName functionName, List parameters, String returnType, Optional comment, RoutineCharacteristics characteristics, Expression body) + private CreateFunction(Optional location, boolean replace, QualifiedName functionName, List parameters, String returnType, Optional comment, RoutineCharacteristics characteristics, Return body) { super(location); this.functionName = requireNonNull(functionName, "functionName is null"); @@ -85,7 +85,7 @@ public RoutineCharacteristics getCharacteristics() return characteristics; } - public Expression getBody() + public Return getBody() { return body; } diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Return.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Return.java new file mode 100644 index 0000000000000..a5d6357360bf4 --- /dev/null +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Return.java @@ -0,0 +1,89 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.sql.tree; + +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public class Return + extends Statement +{ + private final Expression expression; + + public Return(Expression expression) + { + this(Optional.empty(), expression); + } + + public Return(NodeLocation location, Expression expression) + { + this(Optional.of(location), expression); + } + + private Return(Optional location, Expression expression) + { + super(location); + this.expression = requireNonNull(expression, "Expression is null"); + } + + public Expression getExpression() + { + return expression; + } + + @Override + public R accept(AstVisitor visitor, C context) + { + return visitor.visitReturn(this, context); + } + + @Override + public List getChildren() + { + return ImmutableList.of(expression); + } + + @Override + public int hashCode() + { + return Objects.hash(expression); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + Return o = (Return) obj; + return Objects.equals(expression, o.expression); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("expression", expression) + .toString(); + } +} diff --git a/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestSqlParser.java b/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestSqlParser.java index af5efdf6c8277..07095490153d1 100644 --- a/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestSqlParser.java +++ b/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestSqlParser.java @@ -102,6 +102,7 @@ import com.facebook.presto.sql.tree.RenameSchema; import com.facebook.presto.sql.tree.RenameTable; import com.facebook.presto.sql.tree.ResetSession; +import com.facebook.presto.sql.tree.Return; import com.facebook.presto.sql.tree.Revoke; import com.facebook.presto.sql.tree.RevokeRoles; import com.facebook.presto.sql.tree.Rollback; @@ -1460,10 +1461,10 @@ public void testCreateFunction() "double", Optional.of("tangent trigonometric function"), new RoutineCharacteristics(SQL, DETERMINISTIC, RETURNS_NULL_ON_NULL_INPUT), - new ArithmeticBinaryExpression( + new Return(new ArithmeticBinaryExpression( DIVIDE, new FunctionCall(QualifiedName.of("sin"), ImmutableList.of(identifier("x"))), - new FunctionCall(QualifiedName.of("cos"), ImmutableList.of(identifier("x")))))); + new FunctionCall(QualifiedName.of("cos"), ImmutableList.of(identifier("x"))))))); CreateFunction createFunctionRand = new CreateFunction( QualifiedName.of("dev", "testing", "rand"), @@ -1472,7 +1473,7 @@ public void testCreateFunction() "double", Optional.empty(), new RoutineCharacteristics(SQL, NOT_DETERMINISTIC, CALLED_ON_NULL_INPUT), - new FunctionCall(QualifiedName.of("rand"), ImmutableList.of())); + new Return(new FunctionCall(QualifiedName.of("rand"), ImmutableList.of()))); assertStatement( "CREATE OR REPLACE FUNCTION dev.testing.rand ()\n" + "RETURNS double\n" + From ad0617d22309e22bc0fb380afecfc058ee04a685 Mon Sep 17 00:00:00 2001 From: Leiqing Cai Date: Thu, 6 Feb 2020 13:04:00 -0800 Subject: [PATCH 4/6] Introduce db based function namespace manager --- presto-function-namespace-managers/pom.xml | 57 +- ...actSqlInvokedFunctionNamespaceManager.java | 11 + .../InvalidFunctionHandleException.java | 27 + .../functionNamespace/ServingCatalog.java | 31 ++ .../mysql/DefineFunctionNamespacesTable.java | 28 + .../mysql/DefineSqlFunctionsTable.java | 28 + .../mysql/FunctionNamespaceDao.java | 265 ++++++++++ ...ctionNamespacesTableCustomizerFactory.java | 72 +++ .../mysql/MySqlFunctionNamespaceManager.java | 278 ++++++++++ .../MySqlFunctionNamespaceManagerConfig.java | 50 ++ .../MySqlFunctionNamespaceManagerModule.java | 57 ++ ...RoutineCharacteristicsArgumentFactory.java | 41 ++ .../mysql/SqlFunctionIdArgumentFactory.java | 37 ++ .../SqlFunctionsTableCustomizerFactory.java | 72 +++ .../mysql/SqlInvokedFunctionRecord.java | 42 ++ .../SqlInvokedFunctionRecordRowMapper.java | 31 ++ .../mysql/SqlInvokedFunctionRowMapper.java | 63 +++ .../mysql/SqlParametersArgumentFactory.java | 43 ++ .../mysql/TypeSignatureArgumentFactory.java | 37 ++ .../testing/SqlInvokedFunctionTestUtils.java | 16 +- .../TestMySqlFunctionNamespaceManager.java | 488 ++++++++++++++++++ ...stMySqlFunctionNamespaceManagerConfig.java | 48 ++ .../spi/function/RoutineCharacteristics.java | 33 +- 23 files changed, 1841 insertions(+), 14 deletions(-) create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/InvalidFunctionHandleException.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/ServingCatalog.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/DefineFunctionNamespacesTable.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/DefineSqlFunctionsTable.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/FunctionNamespaceDao.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/FunctionNamespacesTableCustomizerFactory.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManager.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManagerConfig.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManagerModule.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/RoutineCharacteristicsArgumentFactory.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlFunctionIdArgumentFactory.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlFunctionsTableCustomizerFactory.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlInvokedFunctionRecord.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlInvokedFunctionRecordRowMapper.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlInvokedFunctionRowMapper.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlParametersArgumentFactory.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/TypeSignatureArgumentFactory.java create mode 100644 presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/mysql/TestMySqlFunctionNamespaceManager.java create mode 100644 presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/mysql/TestMySqlFunctionNamespaceManagerConfig.java diff --git a/presto-function-namespace-managers/pom.xml b/presto-function-namespace-managers/pom.xml index 953941d78b151..4d3bf9d538ead 100644 --- a/presto-function-namespace-managers/pom.xml +++ b/presto-function-namespace-managers/pom.xml @@ -17,6 +17,16 @@ + + com.facebook.airlift + configuration + + + + com.facebook.airlift + json + + com.facebook.presto presto-spi @@ -40,8 +50,8 @@ - com.facebook.airlift - configuration + com.google.inject + guice @@ -49,7 +59,50 @@ units + + javax.annotation + javax.annotation-api + + + + javax.inject + javax.inject + + + + javax.validation + validation-api + + + + org.jdbi + jdbi3-core + + + + org.jdbi + jdbi3-sqlobject + + + + com.facebook.airlift + bootstrap + test + + + + com.facebook.airlift + testing + test + + + + com.facebook.presto + testing-mysql-server-5 + test + + org.testng testng diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/AbstractSqlInvokedFunctionNamespaceManager.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/AbstractSqlInvokedFunctionNamespaceManager.java index 24b07c5ec01ec..bf281e77db2fa 100644 --- a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/AbstractSqlInvokedFunctionNamespaceManager.java +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/AbstractSqlInvokedFunctionNamespaceManager.java @@ -21,6 +21,7 @@ import com.facebook.presto.spi.function.QualifiedFunctionName; import com.facebook.presto.spi.function.ScalarFunctionImplementation; import com.facebook.presto.spi.function.Signature; +import com.facebook.presto.spi.function.SqlFunction; import com.facebook.presto.spi.function.SqlFunctionHandle; import com.facebook.presto.spi.function.SqlFunctionId; import com.facebook.presto.spi.function.SqlInvokedFunction; @@ -160,6 +161,16 @@ public final ScalarFunctionImplementation getScalarFunctionImplementation(Functi return implementationByHandle.getUnchecked((SqlFunctionHandle) functionHandle); } + protected String getCatalogName() + { + return catalogName; + } + + protected void checkCatalog(SqlFunction function) + { + checkCatalog(function.getSignature().getName()); + } + protected void checkCatalog(QualifiedFunctionName functionName) { checkCatalog(functionName.getFunctionNamespace()); diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/InvalidFunctionHandleException.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/InvalidFunctionHandleException.java new file mode 100644 index 0000000000000..2084454046b0e --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/InvalidFunctionHandleException.java @@ -0,0 +1,27 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace; + +import com.facebook.presto.spi.function.FunctionHandle; + +import static java.lang.String.format; + +public class InvalidFunctionHandleException + extends RuntimeException +{ + public InvalidFunctionHandleException(FunctionHandle functionHandle) + { + super(format("Invalid FunctionHandle: %s", functionHandle)); + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/ServingCatalog.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/ServingCatalog.java new file mode 100644 index 0000000000000..6178f60cf45ea --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/ServingCatalog.java @@ -0,0 +1,31 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace; + +import javax.inject.Qualifier; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@Target({FIELD, PARAMETER, METHOD}) +@Qualifier +public @interface ServingCatalog +{ +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/DefineFunctionNamespacesTable.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/DefineFunctionNamespacesTable.java new file mode 100644 index 0000000000000..4b0f58c69b98b --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/DefineFunctionNamespacesTable.java @@ -0,0 +1,28 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import org.jdbi.v3.sqlobject.customizer.SqlStatementCustomizingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@SqlStatementCustomizingAnnotation(FunctionNamespacesTableCustomizerFactory.class) +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.METHOD, ElementType.PARAMETER}) +public @interface DefineFunctionNamespacesTable +{ +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/DefineSqlFunctionsTable.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/DefineSqlFunctionsTable.java new file mode 100644 index 0000000000000..56258213e046f --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/DefineSqlFunctionsTable.java @@ -0,0 +1,28 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import org.jdbi.v3.sqlobject.customizer.SqlStatementCustomizingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@SqlStatementCustomizingAnnotation(SqlFunctionsTableCustomizerFactory.class) +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.METHOD, ElementType.PARAMETER}) +public @interface DefineSqlFunctionsTable +{ +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/FunctionNamespaceDao.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/FunctionNamespaceDao.java new file mode 100644 index 0000000000000..bd690a2e46da4 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/FunctionNamespaceDao.java @@ -0,0 +1,265 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.facebook.presto.spi.function.RoutineCharacteristics; +import com.facebook.presto.spi.function.SqlFunctionId; +import com.facebook.presto.spi.function.SqlInvokedFunction; +import com.facebook.presto.spi.function.SqlParameter; +import com.facebook.presto.spi.type.TypeSignature; +import org.jdbi.v3.sqlobject.config.RegisterArgumentFactories; +import org.jdbi.v3.sqlobject.config.RegisterArgumentFactory; +import org.jdbi.v3.sqlobject.config.RegisterRowMapper; +import org.jdbi.v3.sqlobject.config.RegisterRowMappers; +import org.jdbi.v3.sqlobject.customizer.Bind; +import org.jdbi.v3.sqlobject.statement.SqlQuery; +import org.jdbi.v3.sqlobject.statement.SqlUpdate; + +import java.util.List; +import java.util.Optional; + +@RegisterRowMappers({ + @RegisterRowMapper(SqlInvokedFunctionRowMapper.class), + @RegisterRowMapper(SqlInvokedFunctionRecordRowMapper.class) +}) +@RegisterArgumentFactories({ + @RegisterArgumentFactory(SqlFunctionIdArgumentFactory.class), + @RegisterArgumentFactory(SqlParametersArgumentFactory.class), + @RegisterArgumentFactory(TypeSignatureArgumentFactory.class), + @RegisterArgumentFactory(RoutineCharacteristicsArgumentFactory.class), +}) +@DefineFunctionNamespacesTable +@DefineSqlFunctionsTable +public interface FunctionNamespaceDao +{ + @SqlUpdate("CREATE TABLE IF NOT EXISTS (\n" + + " catalog_name varchar(128) NOT NULL,\n" + + " schema_name varchar(128) NOT NULL,\n" + + " PRIMARY KEY (catalog_name, schema_name))") + void createFunctionNamespacesTableIfNotExists(); + + @SqlUpdate("CREATE TABLE IF NOT EXISTS (\n" + + " id bigint(20) NOT NULL AUTO_INCREMENT,\n" + + " function_id varchar(700) NOT NULL,\n" + + " version bigint(20) unsigned NOT NULL,\n" + + " catalog_name varchar(128) NOT NULL,\n" + + " schema_name varchar(128) NOT NULL,\n" + + " function_name varchar(256) NOT NULL,\n" + + " parameters varchar(40000) NOT NULL,\n" + + " return_type varchar(256) NOT NULL,\n" + + " routine_characteristics text NOT NULL,\n" + + " body mediumtext,\n" + + " description text,\n" + + " deleted boolean NOT NULL DEFAULT false,\n" + + " delete_time TIMESTAMP NULL DEFAULT NULL,\n" + + " create_time TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP,\n" + + " update_time TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,\n" + + " PRIMARY KEY (id),\n" + + " UNIQUE KEY function_id_version (function_id, version),\n" + + " KEY qualified_function_name (catalog_name, schema_name, function_name))") + void createSqlFunctionsTableIfNotExists(); + + @SqlQuery("SELECT\n" + + " count(1) > 0\n" + + "FROM \n" + + "WHERE catalog_name = :catalog_name\n" + + " AND schema_name = :schema_name") + boolean functionNamespaceExists( + @Bind("catalog_name") String catalogName, + @Bind("schema_name") String schemaName); + + @SqlQuery("SELECT\n" + + " t.catalog_name,\n" + + " t.schema_name,\n" + + " t.function_name,\n" + + " t.parameters,\n" + + " t.return_type,\n" + + " t.description,\n" + + " t.routine_characteristics,\n" + + " t.body,\n" + + " t.version\n" + + "FROM t\n" + + "JOIN (\n" + + " SELECT\n" + + " function_id,\n" + + " MAX(version) version\n" + + " FROM \n" + + " WHERE catalog_name = :catalog_name\n" + + " GROUP BY\n" + + " function_id\n" + + ") v\n" + + " ON t.function_id = v.function_id\n " + + " AND t.version = v.version\n" + + "WHERE\n" + + " NOT t.deleted") + List listFunctions(@Bind("catalog_name") String catalogNames); + + @SqlQuery("SELECT\n" + + " t.catalog_name,\n" + + " t.schema_name,\n" + + " t.function_name,\n" + + " t.parameters,\n" + + " t.return_type,\n" + + " t.description,\n" + + " t.routine_characteristics,\n" + + " t.body,\n" + + " t.version\n" + + "FROM t\n" + + "JOIN (\n" + + " SELECT\n" + + " function_id,\n" + + " MAX(version) version\n" + + " FROM \n" + + " WHERE catalog_name = :catalog_name\n" + + " AND schema_name = :schema_name\n" + + " AND function_name = :function_name\n" + + " GROUP BY\n" + + " function_id\n" + + ") v\n" + + " ON t.function_id = v.function_id\n " + + " AND t.version = v.version\n" + + "WHERE\n" + + " NOT t.deleted") + List getFunctions( + @Bind("catalog_name") String catalogName, + @Bind("schema_name") String schemaName, + @Bind("function_name") String functionName); + + @SqlQuery("SELECT\n" + + " catalog_name,\n" + + " schema_name,\n" + + " function_name,\n" + + " parameters,\n" + + " return_type,\n" + + " description,\n" + + " routine_characteristics,\n" + + " body,\n" + + " version\n" + + "FROM \n" + + "WHERE\n" + + " function_id = :function_id\n" + + " AND version = :version") + Optional getFunction( + @Bind("function_id") SqlFunctionId functionId, + @Bind("version") long version); + + @SqlQuery("SELECT\n" + + " t.catalog_name,\n" + + " t.schema_name,\n" + + " t.function_name,\n" + + " t.parameters,\n" + + " t.return_type,\n" + + " t.description,\n" + + " t.routine_characteristics,\n" + + " t.body,\n" + + " t.version,\n" + + " t.deleted\n" + + "FROM t\n" + + "JOIN (\n" + + " SELECT\n" + + " MAX(version) version\n" + + " FROM \n" + + " WHERE\n" + + " function_id = :function_id\n" + + ") v\n" + + "ON\n" + + " t.version = v.version\n" + + "WHERE\n" + + " t.function_id = :function_id\n" + + "FOR UPDATE") + Optional getLatestRecordForUpdate(@Bind("function_id") SqlFunctionId functionId); + + @SqlQuery("SELECT\n" + + " t.catalog_name,\n" + + " t.schema_name,\n" + + " t.function_name,\n" + + " t.parameters,\n" + + " t.return_type,\n" + + " t.description,\n" + + " t.routine_characteristics,\n" + + " t.body,\n" + + " t.version,\n" + + " t.deleted\n" + + "FROM t\n" + + "JOIN (\n" + + " SELECT\n" + + " function_id,\n" + + " MAX(version) version\n" + + " FROM \n" + + " WHERE\n" + + " catalog_name = :catalog_name\n" + + " AND schema_name = :schema_name\n" + + " AND function_name = :function_name\n" + + " GROUP BY\n" + + " function_id\n" + + ") v\n" + + "ON\n" + + " t.function_id = v.function_id\n" + + " AND t.version = v.version\n" + + "FOR UPDATE") + List getLatestRecordsForUpdate( + @Bind("catalog_name") String catalogName, + @Bind("schema_name") String schemaName, + @Bind("function_name") String functionName); + + @SqlUpdate("INSERT INTO (\n" + + " function_id,\n" + + " version,\n" + + " catalog_name,\n" + + " schema_name,\n" + + " function_name,\n" + + " parameters,\n" + + " return_type,\n" + + " description,\n" + + " routine_characteristics,\n" + + " body\n" + + " )\n" + + "VALUES\n" + + " (\n" + + " :function_id,\n" + + " :version,\n" + + " :catalog_name,\n" + + " :schema_name,\n" + + " :function_name,\n" + + " :parameters,\n" + + " :return_type,\n" + + " :description,\n" + + " :routine_characteristics,\n" + + " :body\n" + + " )") + void insertFunction( + @Bind("function_id") SqlFunctionId functionId, + @Bind("version") long version, + @Bind("catalog_name") String catalogName, + @Bind("schema_name") String schemaName, + @Bind("function_name") String functionName, + @Bind("parameters") List parameters, + @Bind("return_type") TypeSignature returnType, + @Bind("description") String description, + @Bind("routine_characteristics") RoutineCharacteristics routineCharacteristics, + @Bind("body") String body); + + @SqlUpdate("UPDATE\n" + + " \n" + + "SET\n" + + " deleted = :deleted\n," + + " delete_time = IF(:deleted, NOW(), null)\n" + + "WHERE\n" + + " function_id = :function_id\n" + + " AND version = :version") + int setDeletionStatus( + @Bind("function_id") SqlFunctionId functionId, + @Bind("version") long version, + @Bind("deleted") boolean deleted); +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/FunctionNamespacesTableCustomizerFactory.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/FunctionNamespacesTableCustomizerFactory.java new file mode 100644 index 0000000000000..368bce9b13c1f --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/FunctionNamespacesTableCustomizerFactory.java @@ -0,0 +1,72 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import org.jdbi.v3.core.config.JdbiConfig; +import org.jdbi.v3.sqlobject.customizer.SqlStatementCustomizer; +import org.jdbi.v3.sqlobject.customizer.SqlStatementCustomizerFactory; + +import java.lang.annotation.Annotation; +import java.lang.reflect.Method; + +import static java.util.Objects.requireNonNull; + +public class FunctionNamespacesTableCustomizerFactory + implements SqlStatementCustomizerFactory +{ + private static final String FUNCTION_NAMESPACES_TABLE_KEY = "function_namespaces_table"; + + @Override + public SqlStatementCustomizer createForType(Annotation annotation, Class sqlObjectType) + { + return statement -> statement.define(FUNCTION_NAMESPACES_TABLE_KEY, statement.getConfig(Config.class).getTableName()); + } + + @Override + public SqlStatementCustomizer createForMethod(Annotation annotation, Class sqlObjectType, Method method) + { + return createForType(annotation, sqlObjectType); + } + + public static class Config + implements JdbiConfig + { + private String tableName; + + public Config() + { + } + + private Config(Config other) + { + this.tableName = other.tableName; + } + + public String getTableName() + { + return tableName; + } + + public void setTableName(String tableName) + { + this.tableName = requireNonNull(tableName, "tableName is null"); + } + + @Override + public Config createCopy() + { + return new Config(this); + } + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManager.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManager.java new file mode 100644 index 0000000000000..d0b8b8ed692f5 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManager.java @@ -0,0 +1,278 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.facebook.presto.functionNamespace.AbstractSqlInvokedFunctionNamespaceManager; +import com.facebook.presto.functionNamespace.InvalidFunctionHandleException; +import com.facebook.presto.functionNamespace.ServingCatalog; +import com.facebook.presto.functionNamespace.SqlInvokedFunctionNamespaceManagerConfig; +import com.facebook.presto.spi.CatalogSchemaName; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.function.AlterRoutineCharacteristics; +import com.facebook.presto.spi.function.FunctionMetadata; +import com.facebook.presto.spi.function.QualifiedFunctionName; +import com.facebook.presto.spi.function.RoutineCharacteristics; +import com.facebook.presto.spi.function.ScalarFunctionImplementation; +import com.facebook.presto.spi.function.Signature; +import com.facebook.presto.spi.function.SqlFunction; +import com.facebook.presto.spi.function.SqlFunctionHandle; +import com.facebook.presto.spi.function.SqlFunctionId; +import com.facebook.presto.spi.function.SqlInvokedFunction; +import com.facebook.presto.spi.type.TypeSignature; +import org.jdbi.v3.core.Jdbi; + +import javax.annotation.PostConstruct; +import javax.inject.Inject; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Optional; + +import static com.facebook.presto.spi.StandardErrorCode.ALREADY_EXISTS; +import static com.facebook.presto.spi.StandardErrorCode.AMBIGUOUS_FUNCTION_CALL; +import static com.facebook.presto.spi.StandardErrorCode.GENERIC_USER_ERROR; +import static com.facebook.presto.spi.StandardErrorCode.NOT_FOUND; +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 java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; + +public class MySqlFunctionNamespaceManager + extends AbstractSqlInvokedFunctionNamespaceManager +{ + private static final int MAX_CATALOG_NAME_LENGTH = 128; + private static final int MAX_SCHEMA_NAME_LENGTH = 128; + private static final int MAX_FUNCTION_NAME_LENGTH = 256; + private static final int MAX_PARAMETER_TYPES_LENGTH = 500; + private static final int MAX_RETURN_TYPE_LENGTH = 256; + + private final Jdbi jdbi; + private final FunctionNamespaceDao functionNamespaceDao; + + @Inject + public MySqlFunctionNamespaceManager( + Jdbi jdbi, + FunctionNamespaceDao functionNamespaceDao, + SqlInvokedFunctionNamespaceManagerConfig managerConfig, + MySqlFunctionNamespaceManagerConfig dbManagerConfig, + @ServingCatalog String catalogName) + { + super(catalogName, managerConfig); + this.jdbi = requireNonNull(jdbi, "jdbi is null"); + this.functionNamespaceDao = requireNonNull(functionNamespaceDao, "functionNamespaceDao is null"); + + jdbi.getConfig(FunctionNamespacesTableCustomizerFactory.Config.class) + .setTableName(dbManagerConfig.getFunctionNamespacesTableName()); + jdbi.getConfig(SqlFunctionsTableCustomizerFactory.Config.class) + .setTableName(dbManagerConfig.getFunctionsTableName()); + } + + @PostConstruct + public void initialize() + { + functionNamespaceDao.createFunctionNamespacesTableIfNotExists(); + functionNamespaceDao.createSqlFunctionsTableIfNotExists(); + } + + @Override + public Collection listFunctions() + { + return functionNamespaceDao.listFunctions(getCatalogName()); + } + + @Override + protected Collection fetchFunctionsDirect(QualifiedFunctionName functionName) + { + checkCatalog(functionName); + return functionNamespaceDao.getFunctions( + functionName.getFunctionNamespace().getCatalogName(), + functionName.getFunctionNamespace().getSchemaName(), + functionName.getFunctionName()); + } + + @Override + protected FunctionMetadata fetchFunctionMetadataDirect(SqlFunctionHandle functionHandle) + { + checkCatalog(functionHandle); + Optional function = functionNamespaceDao.getFunction(functionHandle.getFunctionId(), functionHandle.getVersion()); + if (!function.isPresent()) { + throw new InvalidFunctionHandleException(functionHandle); + } + return sqlInvokedFunctionToMetadata(function.get()); + } + + @Override + protected ScalarFunctionImplementation fetchFunctionImplementationDirect(SqlFunctionHandle functionHandle) + { + checkCatalog(functionHandle); + Optional function = functionNamespaceDao.getFunction(functionHandle.getFunctionId(), functionHandle.getVersion()); + if (!function.isPresent()) { + throw new InvalidFunctionHandleException(functionHandle); + } + return sqlInvokedFunctionToImplementation(function.get()); + } + + @Override + public void createFunction(SqlInvokedFunction function, boolean replace) + { + checkCatalog(function); + checkArgument(!function.getVersion().isPresent(), "function '%s' is already versioned", function); + + QualifiedFunctionName functionName = function.getFunctionId().getFunctionName(); + checkFieldLength("Catalog name", functionName.getFunctionNamespace().getCatalogName(), MAX_CATALOG_NAME_LENGTH); + checkFieldLength("Schema name", functionName.getFunctionNamespace().getSchemaName(), MAX_SCHEMA_NAME_LENGTH); + if (!functionNamespaceDao.functionNamespaceExists(functionName.getFunctionNamespace().getCatalogName(), functionName.getFunctionNamespace().getSchemaName())) { + throw new PrestoException(NOT_FOUND, format("Function namespace not found: %s", functionName.getFunctionNamespace())); + } + checkFieldLength("Function name", functionName.getFunctionName(), MAX_FUNCTION_NAME_LENGTH); + checkFieldLength( + "Parameter types", + function.getFunctionId().getArgumentTypes().stream() + .map(TypeSignature::toString) + .collect(joining(",")), + MAX_PARAMETER_TYPES_LENGTH); + checkFieldLength("Return type", function.getSignature().getReturnType().toString(), MAX_RETURN_TYPE_LENGTH); + + jdbi.useTransaction(handle -> { + FunctionNamespaceDao transactionDao = handle.attach(FunctionNamespaceDao.class); + Optional latestVersion = transactionDao.getLatestRecordForUpdate(function.getFunctionId()); + if (!replace && latestVersion.isPresent() && !latestVersion.get().isDeleted()) { + throw new PrestoException(ALREADY_EXISTS, "Function already exists: " + function.getFunctionId()); + } + + if (!latestVersion.isPresent() || !latestVersion.get().getFunction().hasSameDefinitionAs(function)) { + long newVersion = latestVersion.map(SqlInvokedFunctionRecord::getFunction).flatMap(SqlInvokedFunction::getVersion).orElse(0L) + 1; + insertSqlInvokedFunction(transactionDao, function, newVersion); + } + else if (latestVersion.get().isDeleted()) { + SqlInvokedFunction latest = latestVersion.get().getFunction(); + checkState(latest.getVersion().isPresent(), "Function version missing: %s", latest.getFunctionId()); + transactionDao.setDeletionStatus(latest.getFunctionId(), latest.getVersion().get(), false); + } + }); + refreshFunctionsCache(functionName); + } + + @Override + public void alterFunction(QualifiedFunctionName functionName, Optional> parameterTypes, AlterRoutineCharacteristics alterRoutineCharacteristics) + { + checkCatalog(functionName); + jdbi.useTransaction(handle -> { + FunctionNamespaceDao transactionDao = handle.attach(FunctionNamespaceDao.class); + List functions = getSqlFunctions(transactionDao, functionName, parameterTypes); + + checkUnique(functions, functionName); + checkExists(functions, functionName, parameterTypes); + + SqlInvokedFunction latest = functions.get(0); + RoutineCharacteristics.Builder routineCharacteristics = RoutineCharacteristics.builder(latest.getRoutineCharacteristics()); + alterRoutineCharacteristics.getNullCallClause().ifPresent(routineCharacteristics::setNullCallClause); + SqlInvokedFunction altered = new SqlInvokedFunction( + latest.getFunctionId().getFunctionName(), + latest.getParameters(), + latest.getSignature().getReturnType(), + latest.getDescription(), + routineCharacteristics.build(), + latest.getBody(), + Optional.empty()); + if (!altered.hasSameDefinitionAs(latest)) { + checkState(latest.getVersion().isPresent(), "Function version missing: %s", latest.getFunctionId()); + insertSqlInvokedFunction(transactionDao, altered, latest.getVersion().get() + 1); + } + }); + refreshFunctionsCache(functionName); + } + + @Override + public void dropFunction(QualifiedFunctionName functionName, Optional> parameterTypes, boolean exists) + { + checkCatalog(functionName); + jdbi.useTransaction(handle -> { + FunctionNamespaceDao transactionDao = handle.attach(FunctionNamespaceDao.class); + List functions = getSqlFunctions(transactionDao, functionName, parameterTypes); + + checkUnique(functions, functionName); + checkExists(functions, functionName, parameterTypes); + + SqlInvokedFunction latest = functions.get(0); + checkState(latest.getVersion().isPresent(), "Function version missing: %s", latest.getFunctionId()); + transactionDao.setDeletionStatus(latest.getFunctionId(), latest.getVersion().get(), true); + }); + + refreshFunctionsCache(functionName); + } + + private List getSqlFunctions(FunctionNamespaceDao functionNamespaceDao, QualifiedFunctionName functionName, Optional> parameterTypes) + { + List records = new ArrayList<>(); + if (parameterTypes.isPresent()) { + functionNamespaceDao.getLatestRecordForUpdate(new SqlFunctionId(functionName, parameterTypes.get())).ifPresent(records::add); + } + else { + CatalogSchemaName functionNamespace = functionName.getFunctionNamespace(); + records = functionNamespaceDao.getLatestRecordsForUpdate(functionNamespace.getCatalogName(), functionNamespace.getSchemaName(), functionName.getFunctionName()); + } + return records.stream() + .filter(record -> !record.isDeleted()) + .map(SqlInvokedFunctionRecord::getFunction) + .collect(toImmutableList()); + } + + private void insertSqlInvokedFunction(FunctionNamespaceDao functionNamespaceDao, SqlInvokedFunction function, long version) + { + QualifiedFunctionName functionName = function.getFunctionId().getFunctionName(); + functionNamespaceDao.insertFunction( + function.getFunctionId(), + version, + functionName.getFunctionNamespace().getCatalogName(), + functionName.getFunctionNamespace().getSchemaName(), + functionName.getFunctionName(), + function.getParameters(), + function.getSignature().getReturnType(), + function.getDescription(), + function.getRoutineCharacteristics(), + function.getBody()); + } + + private static void checkFieldLength(String fieldName, String field, int maxLength) + { + if (field.length() > maxLength) { + throw new PrestoException(GENERIC_USER_ERROR, format("%s exceeds max length of %s: %s", fieldName, maxLength, field)); + } + } + + private static void checkUnique(List functions, QualifiedFunctionName functionName) + { + if (functions.size() > 1) { + String signatures = functions.stream() + .map(SqlFunction::getSignature) + .map(Signature::toString) + .collect(joining("; ")); + throw new PrestoException(AMBIGUOUS_FUNCTION_CALL, format("Function '%s' has multiple signatures: %s. Please specify parameter types.", functionName, signatures)); + } + } + + private static void checkExists(List functions, QualifiedFunctionName functionName, Optional> parameterTypes) + { + if (functions.isEmpty()) { + String formattedParameterTypes = parameterTypes.map(types -> types.stream() + .map(TypeSignature::toString) + .collect(joining(",", "(", ")"))).orElse(""); + throw new PrestoException(NOT_FOUND, format("Function not found: %s%s", functionName, formattedParameterTypes)); + } + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManagerConfig.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManagerConfig.java new file mode 100644 index 0000000000000..0350831893ba8 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManagerConfig.java @@ -0,0 +1,50 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.facebook.airlift.configuration.Config; + +import javax.validation.constraints.NotNull; + +public class MySqlFunctionNamespaceManagerConfig +{ + private String functionNamespacesTableName = "function_namespaces"; + private String functionsTableName = "sql_functions"; + + @NotNull + public String getFunctionNamespacesTableName() + { + return functionNamespacesTableName; + } + + @Config("function-namespaces-table-name") + public MySqlFunctionNamespaceManagerConfig setFunctionNamespacesTableName(String functionNamespacesTableName) + { + this.functionNamespacesTableName = functionNamespacesTableName; + return this; + } + + @NotNull + public String getFunctionsTableName() + { + return functionsTableName; + } + + @Config("functions-table-name") + public MySqlFunctionNamespaceManagerConfig setFunctionsTableName(String functionsTableName) + { + this.functionsTableName = functionsTableName; + return this; + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManagerModule.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManagerModule.java new file mode 100644 index 0000000000000..3e8623b8e8497 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManagerModule.java @@ -0,0 +1,57 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.facebook.presto.functionNamespace.ServingCatalog; +import com.facebook.presto.functionNamespace.SqlInvokedFunctionNamespaceManagerConfig; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.Singleton; +import com.google.inject.TypeLiteral; +import org.jdbi.v3.core.Jdbi; +import org.jdbi.v3.core.statement.SqlStatements; + +import static com.facebook.airlift.configuration.ConfigBinder.configBinder; +import static com.google.inject.Scopes.SINGLETON; +import static java.util.Objects.requireNonNull; + +public class MySqlFunctionNamespaceManagerModule + implements Module +{ + private final String catalogName; + + public MySqlFunctionNamespaceManagerModule(String catalogName) + { + this.catalogName = requireNonNull(catalogName, "catalogName is null"); + } + + @Override + public void configure(Binder binder) + { + binder.bind(new TypeLiteral() {}).annotatedWith(ServingCatalog.class).toInstance(catalogName); + + configBinder(binder).bindConfig(SqlInvokedFunctionNamespaceManagerConfig.class); + configBinder(binder).bindConfig(MySqlFunctionNamespaceManagerConfig.class); + binder.bind(MySqlFunctionNamespaceManager.class).in(SINGLETON); + } + + @Provides + @Singleton + public FunctionNamespaceDao provideFunctionNamespaceDao(Jdbi jdbi) + { + SqlStatements config = jdbi.getConfig(SqlStatements.class); + return jdbi.onDemand(FunctionNamespaceDao.class); + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/RoutineCharacteristicsArgumentFactory.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/RoutineCharacteristicsArgumentFactory.java new file mode 100644 index 0000000000000..363913a050037 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/RoutineCharacteristicsArgumentFactory.java @@ -0,0 +1,41 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.presto.spi.function.RoutineCharacteristics; +import org.jdbi.v3.core.argument.AbstractArgumentFactory; +import org.jdbi.v3.core.argument.Argument; +import org.jdbi.v3.core.argument.ObjectArgument; +import org.jdbi.v3.core.config.ConfigRegistry; + +import static com.facebook.airlift.json.JsonCodec.jsonCodec; +import static java.sql.Types.VARCHAR; + +public class RoutineCharacteristicsArgumentFactory + extends AbstractArgumentFactory +{ + private static final JsonCodec ROUTINE_CHARACTERISTICS_CODEC = jsonCodec(RoutineCharacteristics.class); + + public RoutineCharacteristicsArgumentFactory() + { + super(VARCHAR); + } + + @Override + protected Argument build(RoutineCharacteristics routineCharacteristics, ConfigRegistry config) + { + return new ObjectArgument(ROUTINE_CHARACTERISTICS_CODEC.toJson(routineCharacteristics), VARCHAR); + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlFunctionIdArgumentFactory.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlFunctionIdArgumentFactory.java new file mode 100644 index 0000000000000..b75f6ca226bdf --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlFunctionIdArgumentFactory.java @@ -0,0 +1,37 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.facebook.presto.spi.function.SqlFunctionId; +import org.jdbi.v3.core.argument.AbstractArgumentFactory; +import org.jdbi.v3.core.argument.Argument; +import org.jdbi.v3.core.argument.ObjectArgument; +import org.jdbi.v3.core.config.ConfigRegistry; + +import static java.sql.Types.VARCHAR; + +public class SqlFunctionIdArgumentFactory + extends AbstractArgumentFactory +{ + public SqlFunctionIdArgumentFactory() + { + super(VARCHAR); + } + + @Override + public Argument build(SqlFunctionId value, ConfigRegistry config) + { + return new ObjectArgument(value.getId(), VARCHAR); + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlFunctionsTableCustomizerFactory.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlFunctionsTableCustomizerFactory.java new file mode 100644 index 0000000000000..8ec833e12aa09 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlFunctionsTableCustomizerFactory.java @@ -0,0 +1,72 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import org.jdbi.v3.core.config.JdbiConfig; +import org.jdbi.v3.sqlobject.customizer.SqlStatementCustomizer; +import org.jdbi.v3.sqlobject.customizer.SqlStatementCustomizerFactory; + +import java.lang.annotation.Annotation; +import java.lang.reflect.Method; + +import static java.util.Objects.requireNonNull; + +public class SqlFunctionsTableCustomizerFactory + implements SqlStatementCustomizerFactory +{ + private static final String SQL_FUNCTIONS_TABLE_KEY = "sql_functions_table"; + + @Override + public SqlStatementCustomizer createForType(Annotation annotation, Class sqlObjectType) + { + return statement -> statement.define(SQL_FUNCTIONS_TABLE_KEY, statement.getConfig(Config.class).getTableName()); + } + + @Override + public SqlStatementCustomizer createForMethod(Annotation annotation, Class sqlObjectType, Method method) + { + return createForType(annotation, sqlObjectType); + } + + public static class Config + implements JdbiConfig + { + private String tableName; + + public Config() + { + } + + private Config(Config other) + { + this.tableName = other.tableName; + } + + public String getTableName() + { + return tableName; + } + + public void setTableName(String tableName) + { + this.tableName = requireNonNull(tableName, "tableName is null"); + } + + @Override + public Config createCopy() + { + return new Config(this); + } + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlInvokedFunctionRecord.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlInvokedFunctionRecord.java new file mode 100644 index 0000000000000..f7b4bc31eb214 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlInvokedFunctionRecord.java @@ -0,0 +1,42 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.facebook.presto.spi.function.SqlInvokedFunction; + +import static java.util.Objects.requireNonNull; + +public class SqlInvokedFunctionRecord +{ + private final SqlInvokedFunction function; + private final boolean deleted; + + public SqlInvokedFunctionRecord( + SqlInvokedFunction function, + boolean deleted) + { + this.function = requireNonNull(function, "function is null"); + this.deleted = deleted; + } + + public SqlInvokedFunction getFunction() + { + return function; + } + + public boolean isDeleted() + { + return deleted; + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlInvokedFunctionRecordRowMapper.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlInvokedFunctionRecordRowMapper.java new file mode 100644 index 0000000000000..c40ac6643e340 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlInvokedFunctionRecordRowMapper.java @@ -0,0 +1,31 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import org.jdbi.v3.core.mapper.RowMapper; +import org.jdbi.v3.core.statement.StatementContext; + +import java.sql.ResultSet; +import java.sql.SQLException; + +public class SqlInvokedFunctionRecordRowMapper + implements RowMapper +{ + @Override + public SqlInvokedFunctionRecord map(ResultSet rs, StatementContext ctx) + throws SQLException + { + return new SqlInvokedFunctionRecord(new SqlInvokedFunctionRowMapper().map(rs, ctx), rs.getBoolean("deleted")); + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlInvokedFunctionRowMapper.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlInvokedFunctionRowMapper.java new file mode 100644 index 0000000000000..0be69c0c77d27 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlInvokedFunctionRowMapper.java @@ -0,0 +1,63 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.presto.spi.CatalogSchemaName; +import com.facebook.presto.spi.function.QualifiedFunctionName; +import com.facebook.presto.spi.function.RoutineCharacteristics; +import com.facebook.presto.spi.function.SqlInvokedFunction; +import com.facebook.presto.spi.function.SqlParameter; +import org.jdbi.v3.core.mapper.RowMapper; +import org.jdbi.v3.core.statement.StatementContext; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.List; +import java.util.Optional; + +import static com.facebook.airlift.json.JsonCodec.jsonCodec; +import static com.facebook.airlift.json.JsonCodec.listJsonCodec; +import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; + +public class SqlInvokedFunctionRowMapper + implements RowMapper +{ + private static final JsonCodec> SQL_PARAMETERS_CODEC = listJsonCodec(SqlParameter.class); + private static final JsonCodec ROUTINE_CHARACTERISTICS_CODEC = jsonCodec(RoutineCharacteristics.class); + + @Override + public SqlInvokedFunction map(ResultSet rs, StatementContext ctx) + throws SQLException + { + String catalog = rs.getString("catalog_name"); + String schema = rs.getString("schema_name"); + String functionName = rs.getString("function_name"); + List parameters = SQL_PARAMETERS_CODEC.fromJson(rs.getString("parameters")); + String returnType = rs.getString("return_type"); + String description = rs.getString("description"); + RoutineCharacteristics routineCharacteristics = ROUTINE_CHARACTERISTICS_CODEC.fromJson(rs.getString("routine_characteristics")); + String body = rs.getString("body"); + long version = rs.getLong("version"); + + return new SqlInvokedFunction( + QualifiedFunctionName.of(new CatalogSchemaName(catalog, schema), functionName), + parameters, + parseTypeSignature(returnType), + description, + routineCharacteristics, + body, + Optional.of(version)); + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlParametersArgumentFactory.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlParametersArgumentFactory.java new file mode 100644 index 0000000000000..9094cb5d9c064 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/SqlParametersArgumentFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.presto.spi.function.SqlParameter; +import org.jdbi.v3.core.argument.AbstractArgumentFactory; +import org.jdbi.v3.core.argument.Argument; +import org.jdbi.v3.core.argument.ObjectArgument; +import org.jdbi.v3.core.config.ConfigRegistry; + +import java.util.List; + +import static com.facebook.airlift.json.JsonCodec.listJsonCodec; +import static java.sql.Types.VARCHAR; + +public class SqlParametersArgumentFactory + extends AbstractArgumentFactory> +{ + private static final JsonCodec> CODEC = listJsonCodec(SqlParameter.class); + + public SqlParametersArgumentFactory() + { + super(VARCHAR); + } + + @Override + public Argument build(List parameters, ConfigRegistry config) + { + return new ObjectArgument(CODEC.toJson(parameters), VARCHAR); + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/TypeSignatureArgumentFactory.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/TypeSignatureArgumentFactory.java new file mode 100644 index 0000000000000..01baa0140d997 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/TypeSignatureArgumentFactory.java @@ -0,0 +1,37 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.facebook.presto.spi.type.TypeSignature; +import org.jdbi.v3.core.argument.AbstractArgumentFactory; +import org.jdbi.v3.core.argument.Argument; +import org.jdbi.v3.core.argument.ObjectArgument; +import org.jdbi.v3.core.config.ConfigRegistry; + +import static java.sql.Types.VARCHAR; + +public class TypeSignatureArgumentFactory + extends AbstractArgumentFactory +{ + public TypeSignatureArgumentFactory() + { + super(VARCHAR); + } + + @Override + protected Argument build(TypeSignature typeSignature, ConfigRegistry config) + { + return new ObjectArgument(typeSignature.toString(), VARCHAR); + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java index d5d81dfde5af0..057e11324c0c6 100644 --- a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/testing/SqlInvokedFunctionTestUtils.java @@ -35,8 +35,10 @@ private SqlInvokedFunctionTestUtils() } public static final String TEST_CATALOG = "unittest"; + public static final String TEST_SCHEMA = "memory"; - public static final QualifiedFunctionName POWER_TOWER = QualifiedFunctionName.of(new CatalogSchemaName(TEST_CATALOG, "memory"), "power_tower"); + public static final QualifiedFunctionName POWER_TOWER = QualifiedFunctionName.of(new CatalogSchemaName(TEST_CATALOG, TEST_SCHEMA), "power_tower"); + public static final QualifiedFunctionName TANGENT = QualifiedFunctionName.of(new CatalogSchemaName(TEST_CATALOG, TEST_SCHEMA), "tangent"); public static final SqlInvokedFunction FUNCTION_POWER_TOWER_DOUBLE = new SqlInvokedFunction( POWER_TOWER, @@ -64,4 +66,16 @@ private SqlInvokedFunctionTestUtils() RoutineCharacteristics.builder().setDeterminism(DETERMINISTIC).setNullCallClause(RETURNS_NULL_ON_NULL_INPUT).build(), "RETURN pow(x, x)", Optional.empty()); + + public static final SqlInvokedFunction FUNCTION_TANGENT = new SqlInvokedFunction( + TANGENT, + ImmutableList.of(new SqlParameter("x", parseTypeSignature(DOUBLE))), + parseTypeSignature(DOUBLE), + "tangent", + RoutineCharacteristics.builder() + .setDeterminism(DETERMINISTIC) + .setNullCallClause(RETURNS_NULL_ON_NULL_INPUT) + .build(), + "RETURN sin(x) / cos(x)", + Optional.empty()); } diff --git a/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/mysql/TestMySqlFunctionNamespaceManager.java b/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/mysql/TestMySqlFunctionNamespaceManager.java new file mode 100644 index 0000000000000..75237076e1ffb --- /dev/null +++ b/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/mysql/TestMySqlFunctionNamespaceManager.java @@ -0,0 +1,488 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.facebook.airlift.bootstrap.Bootstrap; +import com.facebook.airlift.bootstrap.LifeCycleManager; +import com.facebook.presto.spi.CatalogSchemaName; +import com.facebook.presto.spi.ErrorCodeSupplier; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.function.AlterRoutineCharacteristics; +import com.facebook.presto.spi.function.FunctionHandle; +import com.facebook.presto.spi.function.FunctionMetadata; +import com.facebook.presto.spi.function.FunctionNamespaceTransactionHandle; +import com.facebook.presto.spi.function.QualifiedFunctionName; +import com.facebook.presto.spi.function.RoutineCharacteristics; +import com.facebook.presto.spi.function.SqlFunctionHandle; +import com.facebook.presto.spi.function.SqlFunctionId; +import com.facebook.presto.spi.function.SqlInvokedFunction; +import com.facebook.presto.spi.function.SqlParameter; +import com.facebook.presto.spi.type.TypeSignature; +import com.facebook.presto.testing.mysql.TestingMySqlServer; +import com.google.common.base.Joiner; +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 com.google.inject.Injector; +import org.jdbi.v3.core.Handle; +import org.jdbi.v3.core.Jdbi; +import org.jdbi.v3.sqlobject.SqlObjectPlugin; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.sql.DriverManager; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.facebook.airlift.testing.Closeables.closeQuietly; +import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.FUNCTION_POWER_TOWER_DOUBLE; +import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.FUNCTION_POWER_TOWER_DOUBLE_UPDATED; +import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.FUNCTION_POWER_TOWER_INT; +import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.FUNCTION_TANGENT; +import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.POWER_TOWER; +import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.TANGENT; +import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.TEST_CATALOG; +import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.TEST_SCHEMA; +import static com.facebook.presto.spi.StandardErrorCode.ALREADY_EXISTS; +import static com.facebook.presto.spi.function.FunctionKind.SCALAR; +import static com.facebook.presto.spi.function.RoutineCharacteristics.Determinism.DETERMINISTIC; +import static com.facebook.presto.spi.function.RoutineCharacteristics.NullCallClause.CALLED_ON_NULL_INPUT; +import static com.facebook.presto.spi.function.RoutineCharacteristics.NullCallClause.RETURNS_NULL_ON_NULL_INPUT; +import static com.facebook.presto.spi.type.StandardTypes.DOUBLE; +import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; +import static com.google.common.base.Throwables.throwIfUnchecked; +import static java.lang.String.format; +import static java.util.Collections.nCopies; +import static java.util.Comparator.comparing; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +@Test(singleThreaded = true) +public class TestMySqlFunctionNamespaceManager +{ + private static final String DB = "presto"; + + private TestingMySqlServer mySqlServer; + private Jdbi jdbi; + private Injector injector; + private MySqlFunctionNamespaceManager functionNamespaceManager; + + public Jdbi getJdbi() + { + return jdbi; + } + + @BeforeClass + public void setup() + throws Exception + { + this.mySqlServer = new TestingMySqlServer("testuser", "testpass", DB); + this.jdbi = Jdbi.create(() -> DriverManager.getConnection(mySqlServer.getJdbcUrl(DB))).installPlugin(new SqlObjectPlugin()); + + Bootstrap app = new Bootstrap( + new MySqlFunctionNamespaceManagerModule(TEST_CATALOG), + binder -> binder.bind(Jdbi.class).toInstance(getJdbi())); + + Map config = ImmutableMap.builder() + .put("function-cache-expiration", "0s") + .put("function-instance-cache-expiration", "0s") + .build(); + + try { + this.injector = app.strictConfig() + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + this.functionNamespaceManager = injector.getInstance(MySqlFunctionNamespaceManager.class); + } + catch (Exception e) { + throwIfUnchecked(e); + throw new RuntimeException(e); + } + } + + @BeforeMethod + public void setupFunctionNamespace() + { + createFunctionNamespace(TEST_CATALOG, TEST_SCHEMA); + } + + @AfterMethod + public void cleanup() + { + try (Handle handle = jdbi.open()) { + handle.execute("DELETE FROM sql_functions"); + handle.execute("DELETE FROM function_namespaces"); + } + } + + @AfterClass(alwaysRun = true) + public void tearDown() + { + closeQuietly(mySqlServer); + if (injector != null) { + injector.getInstance(LifeCycleManager.class).stop(); + } + } + + @Test + public void testListFunction() + { + createFunctionNamespace(TEST_CATALOG, "schema1"); + createFunctionNamespace(TEST_CATALOG, "schema2"); + SqlInvokedFunction function1 = constructTestFunction(QualifiedFunctionName.of(new CatalogSchemaName(TEST_CATALOG, "schema1"), "power_tower")); + SqlInvokedFunction function2 = constructTestFunction(QualifiedFunctionName.of(new CatalogSchemaName(TEST_CATALOG, "schema2"), "power_tower")); + + createFunction(function1, false); + createFunction(function2, false); + assertListFunctions(function1.withVersion(1), function2.withVersion(1)); + } + + private static SqlInvokedFunction constructTestFunction(QualifiedFunctionName functionName) + { + return new SqlInvokedFunction( + functionName, + ImmutableList.of(new SqlParameter("x", parseTypeSignature(DOUBLE))), + parseTypeSignature(DOUBLE), + "power tower", + RoutineCharacteristics.builder().setDeterminism(DETERMINISTIC).build(), + "pow(x, x)", + Optional.empty()); + } + + @Test + public void testCreateFunction() + { + assertListFunctions(); + + createFunction(FUNCTION_POWER_TOWER_DOUBLE, false); + assertListFunctions(FUNCTION_POWER_TOWER_DOUBLE.withVersion(1)); + + createFunction(FUNCTION_POWER_TOWER_DOUBLE_UPDATED, true); + assertListFunctions(FUNCTION_POWER_TOWER_DOUBLE_UPDATED.withVersion(2)); + assertGetFunctions(POWER_TOWER, FUNCTION_POWER_TOWER_DOUBLE_UPDATED.withVersion(2)); + + createFunction(FUNCTION_POWER_TOWER_INT, true); + assertListFunctions(FUNCTION_POWER_TOWER_DOUBLE_UPDATED.withVersion(2), FUNCTION_POWER_TOWER_INT.withVersion(1)); + assertGetFunctions(POWER_TOWER, FUNCTION_POWER_TOWER_DOUBLE_UPDATED.withVersion(2), FUNCTION_POWER_TOWER_INT.withVersion(1)); + + createFunction(FUNCTION_TANGENT, true); + assertListFunctions(FUNCTION_POWER_TOWER_DOUBLE_UPDATED.withVersion(2), FUNCTION_POWER_TOWER_INT.withVersion(1), FUNCTION_TANGENT.withVersion(1)); + assertGetFunctions(POWER_TOWER, FUNCTION_POWER_TOWER_DOUBLE_UPDATED.withVersion(2), FUNCTION_POWER_TOWER_INT.withVersion(1)); + assertGetFunctions(TANGENT, FUNCTION_TANGENT.withVersion(1)); + } + + @Test + public void testCreateFunctionFailedDuplicate() + { + createFunction(FUNCTION_POWER_TOWER_DOUBLE, true); + assertPrestoException(() -> createFunction(FUNCTION_POWER_TOWER_DOUBLE, false), ALREADY_EXISTS, ".*Function already exists: unittest\\.memory\\.power_tower\\(double\\)"); + assertPrestoException(() -> createFunction(FUNCTION_POWER_TOWER_DOUBLE_UPDATED, false), ALREADY_EXISTS, ".*Function already exists: unittest\\.memory\\.power_tower\\(double\\)"); + } + + public void testCreateFunctionRepeatedly() + { + createFunction(FUNCTION_POWER_TOWER_DOUBLE, false); + assertListFunctions(FUNCTION_POWER_TOWER_DOUBLE.withVersion(1)); + + createFunction(FUNCTION_POWER_TOWER_DOUBLE, true); + assertListFunctions(FUNCTION_POWER_TOWER_DOUBLE.withVersion(1)); + } + + @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = ".*function 'unittest\\.memory\\.power_tower\\(x double\\):double:1 \\{RETURN pow\\(x, x\\)\\} \\(SQL, DETERMINISTIC, CALLED_ON_NULL_INPUT\\)' is already versioned") + public void testCreateFunctionFailedVersioned() + { + createFunction(FUNCTION_POWER_TOWER_DOUBLE.withVersion(1), true); + } + + @Test(expectedExceptions = PrestoException.class, expectedExceptionsMessageRegExp = "Schema name exceeds max length of 128.*") + public void testCreateFunctionSchemaNameTooLong() + { + QualifiedFunctionName functionName = QualifiedFunctionName.of(new CatalogSchemaName(TEST_CATALOG, dummyString(129)), "tangent"); + createFunction(getFunctionTangent(functionName), false); + } + + @Test(expectedExceptions = PrestoException.class, expectedExceptionsMessageRegExp = "Function name exceeds max length of 256.*") + public void testCreateFunctionFunctionNameTooLong() + { + QualifiedFunctionName functionName = QualifiedFunctionName.of(new CatalogSchemaName(TEST_CATALOG, TEST_SCHEMA), dummyString(257)); + createFunction(getFunctionTangent(functionName), false); + } + + @Test(expectedExceptions = PrestoException.class, expectedExceptionsMessageRegExp = "Parameter types exceeds max length of 500.*") + public void testCreateFunctionFunctionIdTooLong() + { + List parameters = nCopies(72, new SqlParameter("x", parseTypeSignature(DOUBLE))); + createFunction( + new SqlInvokedFunction( + FUNCTION_TANGENT.getFunctionId().getFunctionName(), + parameters, + FUNCTION_TANGENT.getSignature().getReturnType(), + FUNCTION_TANGENT.getDescription(), + FUNCTION_TANGENT.getRoutineCharacteristics(), + FUNCTION_TANGENT.getBody(), + Optional.empty()), + false); + } + + @Test(expectedExceptions = PrestoException.class, expectedExceptionsMessageRegExp = "Return type exceeds max length of 256.*") + public void testCreateFunctionTypeNameTooLong() + { + TypeSignature returnType = parseTypeSignature(dummyString(257)); + createFunction( + new SqlInvokedFunction( + TANGENT, + FUNCTION_TANGENT.getParameters(), + returnType, + FUNCTION_TANGENT.getDescription(), + FUNCTION_TANGENT.getRoutineCharacteristics(), + FUNCTION_TANGENT.getBody(), + Optional.empty()), + false); + } + + @Test + public void testAlterFunction() + { + createFunction(FUNCTION_POWER_TOWER_DOUBLE, false); + createFunction(FUNCTION_POWER_TOWER_INT, false); + createFunction(FUNCTION_TANGENT, false); + + // Alter a specific function by name and parameter types + alterFunction(POWER_TOWER, Optional.of(ImmutableList.of(parseTypeSignature(DOUBLE))), new AlterRoutineCharacteristics(Optional.of(RETURNS_NULL_ON_NULL_INPUT))); + assertGetFunctions( + POWER_TOWER, + FUNCTION_POWER_TOWER_INT.withVersion(1L), + new SqlInvokedFunction( + POWER_TOWER, + ImmutableList.of(new SqlParameter("x", parseTypeSignature(DOUBLE))), + parseTypeSignature(DOUBLE), + "power tower", + RoutineCharacteristics.builder().setDeterminism(DETERMINISTIC).setNullCallClause(RETURNS_NULL_ON_NULL_INPUT).build(), + "RETURN pow(x, x)", + Optional.of(2L))); + + // Drop function and alter function by name + dropFunction(POWER_TOWER, Optional.of(ImmutableList.of(parseTypeSignature(DOUBLE))), false); + alterFunction(POWER_TOWER, Optional.empty(), new AlterRoutineCharacteristics(Optional.of(CALLED_ON_NULL_INPUT))); + + // Alter function by name + alterFunction(TANGENT, Optional.empty(), new AlterRoutineCharacteristics(Optional.of(CALLED_ON_NULL_INPUT))); + SqlInvokedFunction tangentV2 = new SqlInvokedFunction( + TANGENT, + FUNCTION_TANGENT.getParameters(), + FUNCTION_TANGENT.getSignature().getReturnType(), + FUNCTION_TANGENT.getDescription(), + RoutineCharacteristics.builder().setDeterminism(DETERMINISTIC).build(), + FUNCTION_TANGENT.getBody(), + Optional.of(2L)); + assertGetFunctions(TANGENT, tangentV2); + + // Alter function with no change + alterFunction(TANGENT, Optional.empty(), new AlterRoutineCharacteristics(Optional.of(CALLED_ON_NULL_INPUT))); + assertGetFunctions(TANGENT, tangentV2); + } + + @Test(expectedExceptions = PrestoException.class, expectedExceptionsMessageRegExp = "Function 'unittest\\.memory\\.power_tower' has multiple signatures: unittest\\.memory\\.power_tower\\(double\\):double; unittest\\.memory\\.power_tower\\(integer\\):integer\\. Please specify parameter types\\.") + public void testAlterFunctionAmbiguous() + { + createFunction(FUNCTION_POWER_TOWER_DOUBLE, false); + createFunction(FUNCTION_POWER_TOWER_INT, false); + alterFunction(POWER_TOWER, Optional.empty(), new AlterRoutineCharacteristics(Optional.of(RETURNS_NULL_ON_NULL_INPUT))); + } + + @Test(expectedExceptions = PrestoException.class, expectedExceptionsMessageRegExp = "Function not found: unittest\\.memory\\.power_tower\\(\\)") + public void testAlterFunctionNotFound() + { + createFunction(FUNCTION_POWER_TOWER_DOUBLE, false); + createFunction(FUNCTION_POWER_TOWER_INT, false); + alterFunction(POWER_TOWER, Optional.of(ImmutableList.of()), new AlterRoutineCharacteristics(Optional.of(RETURNS_NULL_ON_NULL_INPUT))); + } + + @Test + public void testDropFunction() + { + // Create functions + createFunction(FUNCTION_POWER_TOWER_DOUBLE, false); + createFunction(FUNCTION_POWER_TOWER_INT, false); + createFunction(FUNCTION_TANGENT, false); + + // Drop function by name + dropFunction(TANGENT, Optional.empty(), false); + assertGetFunctions(TANGENT); + + // Recreate functions + createFunction(FUNCTION_TANGENT, false); + + // Drop a specific function by name and parameter types + dropFunction(POWER_TOWER, Optional.of(ImmutableList.of(parseTypeSignature(DOUBLE))), true); + assertGetFunctions(POWER_TOWER, FUNCTION_POWER_TOWER_INT.withVersion(1)); + + dropFunction(TANGENT, Optional.of(ImmutableList.of(parseTypeSignature(DOUBLE))), true); + assertGetFunctions(POWER_TOWER, FUNCTION_POWER_TOWER_INT.withVersion(1)); + assertGetFunctions(TANGENT); + + // Recreate functions, power_double(double) is created with a different definition + createFunction(FUNCTION_POWER_TOWER_DOUBLE_UPDATED, false); + createFunction(FUNCTION_TANGENT, false); + + // Drop functions consecutively + dropFunction(POWER_TOWER, Optional.of(ImmutableList.of(parseTypeSignature(DOUBLE))), false); + dropFunction(POWER_TOWER, Optional.empty(), false); + } + + @Test(expectedExceptions = PrestoException.class, expectedExceptionsMessageRegExp = "Function 'unittest\\.memory\\.power_tower' has multiple signatures: unittest\\.memory\\.power_tower\\(double\\):double; unittest\\.memory\\.power_tower\\(integer\\):integer\\. Please specify parameter types\\.") + public void testDropFunctionAmbiguous() + { + createFunction(FUNCTION_POWER_TOWER_DOUBLE, false); + createFunction(FUNCTION_POWER_TOWER_INT, false); + dropFunction(POWER_TOWER, Optional.empty(), false); + } + + @Test(expectedExceptions = PrestoException.class, expectedExceptionsMessageRegExp = "Function not found: unittest\\.memory\\.power_tower\\(\\)") + public void testDropFunctionNotFound() + { + createFunction(FUNCTION_POWER_TOWER_DOUBLE, false); + createFunction(FUNCTION_POWER_TOWER_INT, false); + dropFunction(POWER_TOWER, Optional.of(ImmutableList.of()), true); + } + + @Test(expectedExceptions = PrestoException.class, expectedExceptionsMessageRegExp = "Function not found: unittest\\.memory\\.tangent\\(double\\)") + public void testDropFunctionFailed() + { + dropFunction(TANGENT, Optional.of(ImmutableList.of(parseTypeSignature(DOUBLE))), false); + } + + @Test(expectedExceptions = PrestoException.class, expectedExceptionsMessageRegExp = "Function not found: unittest\\.memory\\.invalid") + public void testDropFunctionsFailed() + { + dropFunction(QualifiedFunctionName.of(new CatalogSchemaName(TEST_CATALOG, TEST_SCHEMA), "invalid"), Optional.empty(), false); + } + + @Test + public void testGetFunctionMetadata() + { + createFunction(FUNCTION_POWER_TOWER_DOUBLE, true); + FunctionHandle handle1 = getLatestFunctionHandle(FUNCTION_POWER_TOWER_DOUBLE.getFunctionId()); + assertGetFunctionMetadata(handle1, FUNCTION_POWER_TOWER_DOUBLE); + + createFunction(FUNCTION_POWER_TOWER_DOUBLE_UPDATED, true); + FunctionHandle handle2 = getLatestFunctionHandle(FUNCTION_POWER_TOWER_DOUBLE_UPDATED.getFunctionId()); + assertGetFunctionMetadata(handle1, FUNCTION_POWER_TOWER_DOUBLE); + assertGetFunctionMetadata(handle2, FUNCTION_POWER_TOWER_DOUBLE_UPDATED); + } + + @Test(expectedExceptions = UncheckedExecutionException.class, expectedExceptionsMessageRegExp = ".*Invalid FunctionHandle: unittest\\.memory\\.power_tower\\(double\\):2") + public void testInvalidFunctionHandle() + { + createFunction(FUNCTION_POWER_TOWER_DOUBLE, true); + SqlFunctionHandle functionHandle = new SqlFunctionHandle(FUNCTION_POWER_TOWER_DOUBLE.getFunctionId(), 2); + functionNamespaceManager.getFunctionMetadata(functionHandle); + } + + private void createFunctionNamespace(String catalog, String schema) + { + try (Handle handle = jdbi.open()) { + handle.execute("INSERT INTO function_namespaces VALUES (?, ?)", catalog, schema); + } + } + + private void createFunction(SqlInvokedFunction function, boolean replace) + { + functionNamespaceManager.createFunction(function, replace); + } + + private void alterFunction(QualifiedFunctionName functionName, Optional> parameterTypes, AlterRoutineCharacteristics characteristics) + { + functionNamespaceManager.alterFunction(functionName, parameterTypes, characteristics); + } + + private void dropFunction(QualifiedFunctionName functionName, Optional> parameterTypes, boolean exists) + { + functionNamespaceManager.dropFunction(functionName, parameterTypes, exists); + } + + private FunctionHandle getLatestFunctionHandle(SqlFunctionId functionId) + { + FunctionNamespaceTransactionHandle transactionHandle = functionNamespaceManager.beginTransaction(); + Optional function = functionNamespaceManager.getFunctions(Optional.of(transactionHandle), functionId.getFunctionName()).stream() + .filter(candidate -> candidate.getFunctionId().equals(functionId)) + .max(comparing(SqlInvokedFunction::getRequiredVersion)); + assertTrue(function.isPresent()); + functionNamespaceManager.commit(transactionHandle); + return function.get().getRequiredFunctionHandle(); + } + + private void assertListFunctions(SqlInvokedFunction... functions) + { + assertEquals(ImmutableSet.copyOf(functionNamespaceManager.listFunctions()), ImmutableSet.copyOf(Arrays.asList(functions))); + } + + private void assertGetFunctions(QualifiedFunctionName functionName, SqlInvokedFunction... functions) + { + FunctionNamespaceTransactionHandle transactionHandle = functionNamespaceManager.beginTransaction(); + assertEquals(ImmutableSet.copyOf(functionNamespaceManager.getFunctions(Optional.of(transactionHandle), functionName)), ImmutableSet.copyOf(Arrays.asList(functions))); + functionNamespaceManager.commit(transactionHandle); + } + + private void assertGetFunctionMetadata(FunctionHandle functionHandle, SqlInvokedFunction expectFunction) + { + FunctionMetadata functionMetadata = functionNamespaceManager.getFunctionMetadata(functionHandle); + + assertEquals(functionMetadata.getName(), expectFunction.getSignature().getName()); + assertFalse(functionMetadata.getOperatorType().isPresent()); + assertEquals(functionMetadata.getArgumentTypes(), expectFunction.getSignature().getArgumentTypes()); + assertEquals(functionMetadata.getReturnType(), expectFunction.getSignature().getReturnType()); + assertEquals(functionMetadata.getFunctionKind(), SCALAR); + assertEquals(functionMetadata.isDeterministic(), expectFunction.isDeterministic()); + assertEquals(functionMetadata.isCalledOnNullInput(), expectFunction.isCalledOnNullInput()); + } + + private static void assertPrestoException(Runnable runnable, ErrorCodeSupplier errorCode, String expectedMessageRegex) + { + try { + runnable.run(); + fail("No Exception is thrown, expect PrestoException"); + } + catch (PrestoException e) { + assertEquals(e.getErrorCode(), errorCode.toErrorCode()); + assertTrue(e.getMessage().matches(expectedMessageRegex), format("Error message '%s' does not match '%s'", e.getMessage(), expectedMessageRegex)); + } + } + + private static String dummyString(int length) + { + return Joiner.on("").join(nCopies(length, "x")); + } + + private static SqlInvokedFunction getFunctionTangent(QualifiedFunctionName functionName) + { + return new SqlInvokedFunction( + functionName, + FUNCTION_TANGENT.getParameters(), + FUNCTION_TANGENT.getSignature().getReturnType(), + FUNCTION_TANGENT.getDescription(), + FUNCTION_TANGENT.getRoutineCharacteristics(), + FUNCTION_TANGENT.getBody(), + Optional.empty()); + } +} diff --git a/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/mysql/TestMySqlFunctionNamespaceManagerConfig.java b/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/mysql/TestMySqlFunctionNamespaceManagerConfig.java new file mode 100644 index 0000000000000..b726b9d3d4db3 --- /dev/null +++ b/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/mysql/TestMySqlFunctionNamespaceManagerConfig.java @@ -0,0 +1,48 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.Map; + +import static com.facebook.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static com.facebook.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static com.facebook.airlift.configuration.testing.ConfigAssertions.recordDefaults; + +public class TestMySqlFunctionNamespaceManagerConfig +{ + @Test + public void testDefault() + { + assertRecordedDefaults(recordDefaults(MySqlFunctionNamespaceManagerConfig.class) + .setFunctionNamespacesTableName("function_namespaces") + .setFunctionsTableName("sql_functions")); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("function-namespaces-table-name", "sql_function_namespaces") + .put("functions-table-name", "sql_invoked_functions") + .build(); + MySqlFunctionNamespaceManagerConfig expected = new MySqlFunctionNamespaceManagerConfig() + .setFunctionNamespacesTableName("sql_function_namespaces") + .setFunctionsTableName("sql_invoked_functions"); + + assertFullMapping(properties, expected); + } +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/function/RoutineCharacteristics.java b/presto-spi/src/main/java/com/facebook/presto/spi/function/RoutineCharacteristics.java index 022e766a81e27..2326152496273 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/function/RoutineCharacteristics.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/function/RoutineCharacteristics.java @@ -13,7 +13,11 @@ */ package com.facebook.presto.spi.function; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.Objects; +import java.util.Optional; import static com.facebook.presto.spi.function.RoutineCharacteristics.Determinism.DETERMINISTIC; import static com.facebook.presto.spi.function.RoutineCharacteristics.Determinism.NOT_DETERMINISTIC; @@ -45,26 +49,30 @@ public enum NullCallClause private final Determinism determinism; private final NullCallClause nullCallClause; - private RoutineCharacteristics( - Language language, - Determinism determinism, - NullCallClause nullCallClause) + @JsonCreator + public RoutineCharacteristics( + @JsonProperty("language") Optional language, + @JsonProperty("determinism") Optional determinism, + @JsonProperty("nullCallClause") Optional nullCallClause) { - this.language = requireNonNull(language, "language is null"); - this.determinism = requireNonNull(determinism, "determinism is null"); - this.nullCallClause = requireNonNull(nullCallClause, "nullCallClause is null"); + this.language = language.orElse(SQL); + this.determinism = determinism.orElse(NOT_DETERMINISTIC); + this.nullCallClause = nullCallClause.orElse(CALLED_ON_NULL_INPUT); } + @JsonProperty public Language getLanguage() { return language; } + @JsonProperty public Determinism getDeterminism() { return determinism; } + @JsonProperty public NullCallClause getNullCallClause() { return nullCallClause; @@ -119,9 +127,9 @@ public String toString() public static class Builder { - private Language language = SQL; - private Determinism determinism = NOT_DETERMINISTIC; - private NullCallClause nullCallClause = CALLED_ON_NULL_INPUT; + private Language language; + private Determinism determinism; + private NullCallClause nullCallClause; private Builder() {} @@ -152,7 +160,10 @@ public Builder setNullCallClause(NullCallClause nullCallClause) public RoutineCharacteristics build() { - return new RoutineCharacteristics(language, determinism, nullCallClause); + return new RoutineCharacteristics( + Optional.ofNullable(language), + Optional.ofNullable(determinism), + Optional.ofNullable(nullCallClause)); } } } From 50c436401608d35c356823065a6204e685132a1c Mon Sep 17 00:00:00 2001 From: Leiqing Cai Date: Thu, 6 Feb 2020 13:04:00 -0800 Subject: [PATCH 5/6] Introduce MySql-based function namespace manager Also, include function namespace managers plugin in Presto server --- presto-function-namespace-managers/pom.xml | 13 ++-- .../FunctionNamespaceManagerPlugin.java | 29 ++++++++ .../mysql/MySqlConnectionConfig.java | 36 ++++++++++ .../mysql/MySqlConnectionModule.java | 36 ++++++++++ .../MySqlFunctionNamespaceManagerFactory.java | 66 +++++++++++++++++++ .../mysql/TestMySqlConnectionConfig.java | 45 +++++++++++++ presto-main/etc/config.properties | 3 +- presto-server/src/main/assembly/presto.xml | 4 ++ 8 files changed, 225 insertions(+), 7 deletions(-) create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/FunctionNamespaceManagerPlugin.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlConnectionConfig.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlConnectionModule.java create mode 100644 presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManagerFactory.java create mode 100644 presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/mysql/TestMySqlConnectionConfig.java diff --git a/presto-function-namespace-managers/pom.xml b/presto-function-namespace-managers/pom.xml index 4d3bf9d538ead..7325016e8d2eb 100644 --- a/presto-function-namespace-managers/pom.xml +++ b/presto-function-namespace-managers/pom.xml @@ -11,12 +11,18 @@ presto-function-namespace-managers presto-function-namespace-managers + presto-plugin ${project.parent.basedir} + + com.facebook.airlift + bootstrap + + com.facebook.airlift configuration @@ -57,6 +63,7 @@ io.airlift units + provided @@ -85,12 +92,6 @@ - - com.facebook.airlift - bootstrap - test - - com.facebook.airlift testing diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/FunctionNamespaceManagerPlugin.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/FunctionNamespaceManagerPlugin.java new file mode 100644 index 0000000000000..d9c5545e4a000 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/FunctionNamespaceManagerPlugin.java @@ -0,0 +1,29 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace; + +import com.facebook.presto.functionNamespace.mysql.MySqlFunctionNamespaceManagerFactory; +import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.function.FunctionNamespaceManagerFactory; +import com.google.common.collect.ImmutableList; + +public class FunctionNamespaceManagerPlugin + implements Plugin +{ + @Override + public Iterable getFunctionNamespaceManagerFactories() + { + return ImmutableList.of(new MySqlFunctionNamespaceManagerFactory()); + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlConnectionConfig.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlConnectionConfig.java new file mode 100644 index 0000000000000..1952a395b4a69 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlConnectionConfig.java @@ -0,0 +1,36 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.facebook.airlift.configuration.Config; + +import javax.validation.constraints.NotNull; + +public class MySqlConnectionConfig +{ + private String databaseUrl; + + @NotNull + public String getDatabaseUrl() + { + return databaseUrl; + } + + @Config("database-url") + public MySqlConnectionConfig setDatabaseUrl(String databaseUrl) + { + this.databaseUrl = databaseUrl; + return this; + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlConnectionModule.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlConnectionModule.java new file mode 100644 index 0000000000000..3c62304cbd2a6 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlConnectionModule.java @@ -0,0 +1,36 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.facebook.airlift.configuration.AbstractConfigurationAwareModule; +import com.google.inject.Binder; +import org.jdbi.v3.core.Jdbi; +import org.jdbi.v3.sqlobject.SqlObjectPlugin; + +import java.sql.DriverManager; + +import static com.facebook.airlift.configuration.ConfigBinder.configBinder; + +public class MySqlConnectionModule + extends AbstractConfigurationAwareModule +{ + @Override + protected void setup(Binder binder) + { + configBinder(binder).bindConfig(MySqlConnectionConfig.class); + + String databaseUrl = buildConfigObject(MySqlConnectionConfig.class).getDatabaseUrl(); + binder.bind(Jdbi.class).toInstance(Jdbi.create(() -> DriverManager.getConnection(databaseUrl)).installPlugin(new SqlObjectPlugin())); + } +} diff --git a/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManagerFactory.java b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManagerFactory.java new file mode 100644 index 0000000000000..666de741650f1 --- /dev/null +++ b/presto-function-namespace-managers/src/main/java/com/facebook/presto/functionNamespace/mysql/MySqlFunctionNamespaceManagerFactory.java @@ -0,0 +1,66 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.facebook.airlift.bootstrap.Bootstrap; +import com.facebook.presto.spi.function.FunctionHandleResolver; +import com.facebook.presto.spi.function.FunctionNamespaceManager; +import com.facebook.presto.spi.function.FunctionNamespaceManagerFactory; +import com.facebook.presto.spi.function.SqlFunctionHandle; +import com.google.inject.Injector; + +import java.util.Map; + +import static com.google.common.base.Throwables.throwIfUnchecked; + +public class MySqlFunctionNamespaceManagerFactory + implements FunctionNamespaceManagerFactory +{ + public static final String NAME = "mysql"; + + private static final SqlFunctionHandle.Resolver HANDLE_RESOLVER = new SqlFunctionHandle.Resolver(); + + @Override + public String getName() + { + return NAME; + } + + @Override + public FunctionHandleResolver getHandleResolver() + { + return HANDLE_RESOLVER; + } + + @Override + public FunctionNamespaceManager create(String catalogName, Map config) + { + try { + Bootstrap app = new Bootstrap( + new MySqlFunctionNamespaceManagerModule(catalogName), + new MySqlConnectionModule()); + + Injector injector = app + .strictConfig() + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + return injector.getInstance(MySqlFunctionNamespaceManager.class); + } + catch (Exception e) { + throwIfUnchecked(e); + throw new RuntimeException(e); + } + } +} diff --git a/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/mysql/TestMySqlConnectionConfig.java b/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/mysql/TestMySqlConnectionConfig.java new file mode 100644 index 0000000000000..4f67e71ff31f2 --- /dev/null +++ b/presto-function-namespace-managers/src/test/java/com/facebook/presto/functionNamespace/mysql/TestMySqlConnectionConfig.java @@ -0,0 +1,45 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.functionNamespace.mysql; + +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.Map; + +import static com.facebook.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static com.facebook.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static com.facebook.airlift.configuration.testing.ConfigAssertions.recordDefaults; + +public class TestMySqlConnectionConfig +{ + @Test + public void testDefault() + { + assertRecordedDefaults(recordDefaults(MySqlConnectionConfig.class) + .setDatabaseUrl(null)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("database-url", "localhost:1080") + .build(); + MySqlConnectionConfig expected = new MySqlConnectionConfig() + .setDatabaseUrl("localhost:1080"); + + assertFullMapping(properties, expected); + } +} diff --git a/presto-main/etc/config.properties b/presto-main/etc/config.properties index a84531ecb4e82..e124d24f3d4e2 100644 --- a/presto-main/etc/config.properties +++ b/presto-main/etc/config.properties @@ -39,7 +39,8 @@ plugin.bundles=\ ../presto-mysql/pom.xml,\ ../presto-sqlserver/pom.xml, \ ../presto-postgresql/pom.xml, \ - ../presto-tpcds/pom.xml + ../presto-tpcds/pom.xml, \ + ../presto-function-namespace-managers/pom.xml presto.version=testversion node-scheduler.include-coordinator=true diff --git a/presto-server/src/main/assembly/presto.xml b/presto-server/src/main/assembly/presto.xml index c7b5995e17068..510ee1c735d6b 100644 --- a/presto-server/src/main/assembly/presto.xml +++ b/presto-server/src/main/assembly/presto.xml @@ -64,6 +64,10 @@ ${project.build.directory}/dependency/presto-session-property-managers-${project.version} plugin/session-property-managers + + ${project.build.directory}/dependency/presto-function-namespace-managers-${project.version} + plugin/function-namespace-managers + ${project.build.directory}/dependency/presto-jmx-${project.version} plugin/jmx From 27de3877889ae6fbc412504791cd2ce2257077ce Mon Sep 17 00:00:00 2001 From: Leiqing Cai Date: Thu, 6 Feb 2020 13:04:00 -0800 Subject: [PATCH 6/6] Add documentation for SQL functions - Add documentation for CREATE FUNCTION, ALTER FUNCTION, and DROP FUNCTION statements. - Add documentation for Function Namespace Managers configuration. --- presto-docs/src/main/sphinx/admin.rst | 1 + .../admin/function-namespace-managers.rst | 92 +++++++++++++++++++ presto-docs/src/main/sphinx/sql.rst | 3 + .../src/main/sphinx/sql/alter-function.rst | 43 +++++++++ .../src/main/sphinx/sql/create-function.rst | 84 +++++++++++++++++ .../src/main/sphinx/sql/drop-function.rst | 45 +++++++++ 6 files changed, 268 insertions(+) create mode 100644 presto-docs/src/main/sphinx/admin/function-namespace-managers.rst create mode 100644 presto-docs/src/main/sphinx/sql/alter-function.rst create mode 100644 presto-docs/src/main/sphinx/sql/create-function.rst create mode 100644 presto-docs/src/main/sphinx/sql/drop-function.rst diff --git a/presto-docs/src/main/sphinx/admin.rst b/presto-docs/src/main/sphinx/admin.rst index 8413652242e1c..b10bf7403799c 100644 --- a/presto-docs/src/main/sphinx/admin.rst +++ b/presto-docs/src/main/sphinx/admin.rst @@ -11,4 +11,5 @@ Administration admin/spill admin/resource-groups admin/session-property-managers + admin/function-namespace-managers admin/dist-sort diff --git a/presto-docs/src/main/sphinx/admin/function-namespace-managers.rst b/presto-docs/src/main/sphinx/admin/function-namespace-managers.rst new file mode 100644 index 0000000000000..c1ac327d215a4 --- /dev/null +++ b/presto-docs/src/main/sphinx/admin/function-namespace-managers.rst @@ -0,0 +1,92 @@ +=========================== +Function Namespace Managers +=========================== + +.. warning:: + + This is an experimental feature being actively developed. The way + Function Namespace Managers are configured might be changed. + +Function namespace managers support storing and retrieving SQL +functions, allowing the Presto engine to perform actions such as +creating, altering, deleting functions. + +A function namespace is in the format of ``catalog.schema`` (e.g. +``example.test``). It can be thought of as a schema for storing +functions. However, it is not a full fledged schema as it does not +support storing tables and views, but only functions. + +Each Presto function, whether built-in or user-defined, resides in +a function namespace. All built-in functions reside in the +``presto.default`` function namespace. The qualified function name of +a function is the function namespace in which it reside followed by +its function name (e.g. ``example.test.func``). Built-in functions can +be referenced in queries with their function namespaces omitted, while +user-defined functions needs to be referenced by its qualified function +name. A function is uniquely identified by its qualified function name +and parameter type list. + +Each function namespace manager binds to a catalog name and manages all +functions within that catalog. Using the catalog name of an existing +connector is discouraged, as the behavior is not defined nor tested, +and will be disallowed in the future. + +Currently, those catalog names do not correspond to real catalogs. +They cannot be specified as the catalog in a session, nor do they +support :doc:`/sql/create-schema`, :doc:`/sql/alter-schema`, +:doc:`/sql/drop-schema`, or :doc:`/sql/show-schemas`. Instead, +namespaces can be added using the methods described below. + + +Configuration +------------- + +Presto currently stores all function namespace manager related +information in MySQL. + +To instantiate a MySQL-based function namespace manager that manages +catalog ``example``, administrator needs to first have a running MySQL +server. Suppose the MySQL server can be reached at ``localhost:1080``, +add a file ``etc/function-namespace/example.properties`` with the +following contents:: + + function-namespace-manager.name=mysql + database-url=localhost:1080 + function-namespaces-table-name=example_function_namespaces + functions-table-name=example_sql_functions + +When Presto first starts with the above MySQL function namespace +manager configuration, two MySQL tables will be created if they do +not exist. + +- ``example_function_namespaces`` stores function namespaces of + the catalog ``example``. +- ``example_sql_functions`` stores SQL-invoked functions of the + catalog ``example``. + +Multiple function namespace managers can be instantiated by placing +multiple properties files under ``etc/function-namespace``. They +may be configured to use the same tables. If so, each manager will +only create and interact with entries of the catalog to which it binds. + +To create a new function namespace, insert into the +``example_function_namespaces`` table:: + + INSERT INTO example_function_namespaces (catalog_name, schema_name) + VALUES('example', 'test'); + + +Configuration Reference +----------------------- + +``function-namespace-manager.name`` is the type of the function namespace manager to instantiate. Currently, only ``mysql`` is supported. + +The following table lists all configuration properties supported by the MySQL function namespace manager. + +=========================================== ================================================================================================== +Name Description +=========================================== ================================================================================================== +``database-url`` The URL of the MySQL database used by the MySQL function namespace manager. +``function-namespaces-table-name`` The name of the table that stores all the function namespaces managed by this manager. +``functions-table-name`` The name of the table that stores all the functions managed by this manager. +=========================================== ================================================================================================== diff --git a/presto-docs/src/main/sphinx/sql.rst b/presto-docs/src/main/sphinx/sql.rst index 11442c4b58a6f..0217413bfb0bf 100644 --- a/presto-docs/src/main/sphinx/sql.rst +++ b/presto-docs/src/main/sphinx/sql.rst @@ -7,11 +7,13 @@ This chapter describes the SQL syntax used in Presto. .. toctree:: :maxdepth: 1 + sql/alter-function sql/alter-schema sql/alter-table sql/analyze sql/call sql/commit + sql/create-function sql/create-role sql/create-schema sql/create-table @@ -22,6 +24,7 @@ This chapter describes the SQL syntax used in Presto. sql/describe sql/describe-input sql/describe-output + sql/drop-function sql/drop-role sql/drop-schema sql/drop-table diff --git a/presto-docs/src/main/sphinx/sql/alter-function.rst b/presto-docs/src/main/sphinx/sql/alter-function.rst new file mode 100644 index 0000000000000..a9c11a7546f6e --- /dev/null +++ b/presto-docs/src/main/sphinx/sql/alter-function.rst @@ -0,0 +1,43 @@ +============== +ALTER FUNCTION +============== + +Synopsis +-------- + +.. code-block:: none + + ALTER FUNCTION qualified_function_name [ ( parameter_type[, ...] ) ] + RETURNS NULL ON NULL INPUT | CALLED ON NULL INPUT + + +Description +----------- + +Alter the definition of an existing function. + +Parameter type list must be specified if multiple signatures exist +for the specified function name. If exactly one signature exists for +the specified function name, parameter type list can be omitted. + +Currently, only modifying the null-call clause is supported. + + +Examples +-------- + +Change the null-call clause of a function ``example.default.tan(double)``:: + + ALTER FUNCTION prod.default.tan(double) + CALLED ON NULL INPUT + +If only one function exists for ``example.default.tan``, parameter type list may be omitted:: + + ALTER FUNCTION prod.default.tan + CALLED ON NULL INPUT + + +See Also +-------- + +:doc:`create-function`, :doc:`drop-function`, :doc:`show-functions` diff --git a/presto-docs/src/main/sphinx/sql/create-function.rst b/presto-docs/src/main/sphinx/sql/create-function.rst new file mode 100644 index 0000000000000..1d20088d5f6c0 --- /dev/null +++ b/presto-docs/src/main/sphinx/sql/create-function.rst @@ -0,0 +1,84 @@ +=============== +CREATE FUNCTION +=============== + +Synopsis +-------- + +.. code-block:: none + + CREATE [ OR REPLACE ] FUNCTION + qualified_function_name ( + parameter_name parameter_type + [, ...] + ) + RETURNS return_type + [ COMMENT function_description ] + [ LANGUAGE SQL ] + [ DETERMINISTIC | NOT DETERMINISTIC ] + [ RETURNS NULL ON NULL INPUT | CALLED ON NULL INPUT ] + [ RETURN expression ] + + +Description +----------- + +Create a new function with the specified definition. + +Each function is uniquely identified by its qualified function name +and its parameter type list. ``qualified_function_name`` needs to be in +the format of ``catalog.schema.function_name``. + +In order to create a function, the corresponding function namespace +(in the format ``catalog.schema``) must first be managed by a function +namespace manager (See :doc:`/admin/function-namespace-managers`). + +The optional ``OR REPLACE`` clause causes the query to quietly replace +the existing function if a function with the identical signature (function +name with parameter type list) exists. + +The ``return_type`` needs to match the actual type of the routine body +``expression``, without performing type coercion. + +A set of routine characteristics can be specified to decorate the +function and specify its behavior. Each kind of routine characteristic +can be specified at most once. + +============================ ======================== ================================================================ +Routine Characteristic Default Value Description +============================ ======================== ================================================================ +Language clause SQL The language in which the function is defined. +Deterministic characteristic NOT DETERMINISTIC Whether the function is deterministic. ``NOT DETERMINISTIC`` + means that the function is possibly non-deterministic. +Null-call clause CALLED ON NULL INPUT The behavior of the function in which ``null`` is supplied as + the value of at least one argument. +============================ ======================== ================================================================ + + +Examples +-------- + +Create a new function ``example.default.tan(double)``:: + + CREATE FUNCTION example.default.tan(x double) + RETURNS double + DETERMINISTIC + RETURNS NULL ON NULL INPUT + RETURN sin(x) / cos(x) + +Create the table ``example.default.tan(double)`` if it does not already +exist, adding a function description and explicitly listing all the supported +routine characteristics:: + + CREATE OR REPLACE FUNCTION example.default.tan(x double) + RETURNS double + COMMENT 'tangent trigonometric function' + LANGUAGE SQL + DETERMINISTIC + RETURNS NULL ON NULL INPUT + RETURN sin(x) / cos(x) + +See Also +-------- + +:doc:`alter-function`, :doc:`drop-function`, :doc:`show-functions` diff --git a/presto-docs/src/main/sphinx/sql/drop-function.rst b/presto-docs/src/main/sphinx/sql/drop-function.rst new file mode 100644 index 0000000000000..a6e9e51b9ba9f --- /dev/null +++ b/presto-docs/src/main/sphinx/sql/drop-function.rst @@ -0,0 +1,45 @@ +============= +DROP FUNCTION +============= + +Synopsis +-------- + +.. code-block:: none + + DROP FUNCTION [ IF EXISTS ] qualified_function_name [ ( parameter_type[, ...] ) ] + + +Description +----------- + +Drop an existing function. + +The optional ``IF EXISTS`` clause causes the ``NOT_FOUND`` error +to be suppressed if the function does not exists. + +Each ``DROP FUNCTION`` statement can only drop one function +at a time. If multiple functions are matched by not specifying +the parameter type list, the query would fail. + + +Examples +-------- + +Drop the function ``example.default.tan(double)``:: + + DROP FUNCTION example.default.tan(double) + +If only one function exists for ``example.default.tan``, parameter type list may be omitted:: + + DROP FUNCTION example.default.tan + +Drop the function ``example.default.tan(double)`` if it exists:: + + DROP FUNCTION IF EXISTS example.default.tan(double) + + +See Also +-------- + +:doc:`create-function`, :doc:`alter-function`, :doc:`show-functions`