|
39 | 39 | import com.facebook.presto.spi.StandardErrorCode; |
40 | 40 | import com.facebook.presto.spi.function.AggregationFunctionImplementation; |
41 | 41 | import com.facebook.presto.spi.function.AlterRoutineCharacteristics; |
42 | | -import com.facebook.presto.spi.function.CatalogSchemaFunctionName; |
43 | 42 | import com.facebook.presto.spi.function.FunctionHandle; |
44 | 43 | import com.facebook.presto.spi.function.FunctionMetadata; |
45 | 44 | import com.facebook.presto.spi.function.FunctionMetadataManager; |
|
108 | 107 | import static com.facebook.presto.metadata.FunctionSignatureMatcher.constructFunctionNotFoundErrorMessage; |
109 | 108 | import static com.facebook.presto.metadata.SessionFunctionHandle.SESSION_NAMESPACE; |
110 | 109 | import static com.facebook.presto.metadata.SignatureBinder.applyBoundVariables; |
111 | | -import static com.facebook.presto.metadata.TableFunctionRegistry.toPath; |
112 | 110 | import static com.facebook.presto.spi.StandardErrorCode.ALREADY_EXISTS; |
113 | 111 | import static com.facebook.presto.spi.StandardErrorCode.AMBIGUOUS_FUNCTION_CALL; |
114 | 112 | import static com.facebook.presto.spi.StandardErrorCode.FUNCTION_IMPLEMENTATION_MISSING; |
@@ -167,7 +165,6 @@ public class FunctionAndTypeManager |
167 | 165 | private final ConcurrentHashMap<ConnectorId, Function<ConnectorTableFunctionHandle, TableFunctionProcessorProvider>> tableFunctionProcessorProviderMap = new ConcurrentHashMap<>(); |
168 | 166 | private final Map<String, TVFProviderFactory> tvfProviderFactories = new ConcurrentHashMap<>(); |
169 | 167 | private final Map<ConnectorId, TVFProvider> tvfProviders = new ConcurrentHashMap<>(); |
170 | | - private final AtomicReference<TVFProvider> servingTableFunctionsProvider; |
171 | 168 |
|
172 | 169 | @Inject |
173 | 170 | public FunctionAndTypeManager( |
@@ -204,7 +201,6 @@ public FunctionAndTypeManager( |
204 | 201 | this.servingTypeManager = new AtomicReference<>(builtInTypeAndFunctionNamespaceManager); |
205 | 202 | this.servingTypeManagerParametricTypesSupplier = new AtomicReference<>(this::getServingTypeManagerParametricTypes); |
206 | 203 | this.builtInPluginFunctionNamespaceManager = new BuiltInPluginFunctionNamespaceManager(this); |
207 | | - this.servingTableFunctionsProvider = new AtomicReference<>(builtInTypeAndFunctionNamespaceManager); |
208 | 204 | } |
209 | 205 |
|
210 | 206 | public static FunctionAndTypeManager createTestFunctionAndTypeManager() |
@@ -373,12 +369,11 @@ public void loadTVFProvider(String tvfProviderName, NodeManager nodeManager) |
373 | 369 | requireNonNull(tvfProviderName, "tvfProviderName is null"); |
374 | 370 | TVFProviderFactory factory = tvfProviderFactories.get(tvfProviderName); |
375 | 371 | checkState(factory != null, "No factory for tvf provider %s", tvfProviderName); |
376 | | - TVFProvider tvfProvider = factory.createTVFProvider(tvfProviderName, ImmutableMap.of(), new TVFProviderContext(nodeManager, this)); |
| 372 | + TVFProvider tvfProvider = factory.createTVFProvider(ImmutableMap.of(), new TVFProviderContext(nodeManager, this)); |
377 | 373 |
|
378 | 374 | if (tvfProviders.putIfAbsent(new ConnectorId(tvfProviderName), tvfProvider) != null) { |
379 | 375 | throw new IllegalArgumentException(format("TVF provider [%s] is already registered", tvfProvider)); |
380 | 376 | } |
381 | | - servingTableFunctionsProvider.compareAndSet(servingTableFunctionsProvider.get(), tvfProvider); |
382 | 377 | } |
383 | 378 |
|
384 | 379 | public void loadTVFProviders(NodeManager nodeManager) |
@@ -498,19 +493,19 @@ public void addTypeManagerFactory(TypeManagerFactory factory) |
498 | 493 |
|
499 | 494 | public TableFunctionMetadata resolveTableFunction(Session session, QualifiedName qualifiedName) |
500 | 495 | { |
501 | | - // Fetch if it's a builtin function first |
502 | | - TableFunctionMetadata tableFunctionMetadata = |
503 | | - servingTableFunctionsProvider.get() |
504 | | - .resolveTableFunction( |
505 | | - getFunctionAndTypeResolver() |
506 | | - .qualifyObjectName(qualifiedName).getObjectName()); |
507 | | - if (tableFunctionMetadata == null) { |
508 | | - // populate the registry before trying to resolve the table functions from table functions provider |
509 | | - List<CatalogSchemaFunctionName> name = toPath(session, qualifiedName); |
510 | | - ConnectorId connectorId = new ConnectorId(name.get(0).getCatalogName()); |
511 | | - return tableFunctionRegistry.resolve(connectorId, name.get(0)); |
| 496 | + // Before resolving the table function, add all the TVF provider's table functions to the function registry. |
| 497 | + if (!tableFunctionRegistry.areTvfProviderFunctionsLoaded()) { |
| 498 | + for (ConnectorId connectorId : tvfProviders.keySet()) { |
| 499 | + // In terms of the NativeTVFProvider, you want it to act similarly to the system connector table functions, hence we replace the Java loaded system connector table functions. |
| 500 | + // This is only enforced when native execution is enabled and presto-native-tvf module is loaded. |
| 501 | + if (connectorId.getCatalogName().equals("system")) { |
| 502 | + tableFunctionRegistry.removeTableFunctions(connectorId); |
| 503 | + } |
| 504 | + tableFunctionRegistry.addTableFunctions(connectorId, tvfProviders.get(connectorId).getTableFunctions()); |
| 505 | + } |
| 506 | + tableFunctionRegistry.updateTvfProviderFunctionsLoaded(); |
512 | 507 | } |
513 | | - return tableFunctionMetadata; |
| 508 | + return tableFunctionRegistry.resolve(session, qualifiedName); |
514 | 509 | } |
515 | 510 |
|
516 | 511 | public TransactionManager getTransactionManager() |
|
0 commit comments