From 26749a8e471d968bcdf0fcee3fe42814ed4b10d0 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Sun, 13 Dec 2020 20:26:57 -0800 Subject: [PATCH] [Functions] Provide an interface for functions worker service (#8560) *Motivation* Make the pulsar functions worker serve as an interface to allow plugin different functions of worker service implementations. --- .../pulsar/broker/ServiceConfiguration.java | 6 + .../apache/pulsar/PulsarBrokerStarter.java | 47 +- .../pulsar/PulsarClusterMetadataSetup.java | 20 +- .../org/apache/pulsar/PulsarStandalone.java | 47 +- .../apache/pulsar/broker/PulsarService.java | 175 ++--- .../broker/admin/impl/FunctionsBase.java | 62 +- .../broker/admin/impl/NamespacesBase.java | 24 +- .../pulsar/broker/admin/impl/SinksBase.java | 48 +- .../pulsar/broker/admin/impl/SourcesBase.java | 48 +- .../pulsar/broker/admin/v1/Namespaces.java | 1 + .../pulsar/broker/admin/v2/Functions.java | 48 +- .../pulsar/broker/admin/v2/Namespaces.java | 1 + .../apache/pulsar/broker/admin/v2/Worker.java | 20 +- .../pulsar/broker/admin/v2/WorkerStats.java | 23 +- .../PrometheusMetricsGenerator.java | 6 +- .../pulsar/broker/web/PulsarWebResource.java | 21 +- .../worker/PulsarFunctionE2ESecurityTest.java | 12 +- .../worker/PulsarFunctionLocalRunTest.java | 10 +- .../worker/PulsarFunctionPublishTest.java | 12 +- .../worker/PulsarWorkerAssignmentTest.java | 15 +- .../pulsar/io/PulsarFunctionAdminTest.java | 13 +- .../pulsar/io/PulsarFunctionE2ETest.java | 11 +- .../pulsar/io/PulsarFunctionTlsTest.java | 44 +- .../pulsar/common/policies/data/Policies.java | 26 + .../common/policies/path/PolicyPath.java | 53 ++ .../common/policies/path/package-info.java | 22 + .../pulsar/functions/worker/WorkerConfig.java | 24 +- .../KubernetesRuntimeFactoryTest.java | 2 +- .../kubernetes/KubernetesRuntimeTest.java | 3 +- .../worker/FunctionRuntimeManager.java | 4 +- .../worker/FunctionsStatsGenerator.java | 2 +- .../functions/worker/PulsarWorkerService.java | 633 ++++++++++++++++++ .../pulsar/functions/worker/Worker.java | 104 +-- .../functions/worker/WorkerService.java | 460 ++++--------- .../pulsar/functions/worker/WorkerUtils.java | 7 +- .../functions/worker/rest/WorkerServer.java | 36 +- .../worker/rest/api/ComponentImpl.java | 28 +- .../worker/rest/api/FunctionsImpl.java | 14 +- .../worker/rest/api/FunctionsImplV2.java | 27 +- .../rest/api/FunctionsMetricsResource.java | 3 +- .../functions/worker/rest/api/SinksImpl.java | 13 +- .../worker/rest/api/SourcesImpl.java | 14 +- .../functions/worker/rest/api/WorkerImpl.java | 27 +- .../rest/api/v2/FunctionsApiV2Resource.java | 41 +- .../rest/api/v2/WorkerApiV2Resource.java | 23 +- .../rest/api/v2/WorkerStatsApiV2Resource.java | 15 +- .../rest/api/v3/FunctionsApiV3Resource.java | 55 +- .../rest/api/v3/SinksApiV3Resource.java | 47 +- .../rest/api/v3/SourcesApiV3Resource.java | 47 +- .../service/WorkerServiceDefinition.java | 44 ++ .../worker/service/WorkerServiceLoader.java | 168 +++++ .../worker/service/WorkerServiceMetadata.java | 42 ++ .../service/WorkerServiceWithClassLoader.java | 124 ++++ .../worker/service/api/Component.java | 158 +++++ .../worker/service/api/Functions.java | 81 +++ .../worker/service/api/FunctionsV2.java | 126 ++++ .../functions/worker/service/api/Sinks.java | 85 +++ .../functions/worker/service/api/Sources.java | 86 +++ .../functions/worker/service/api/Workers.java | 53 ++ .../worker/service/api/package-info.java | 22 + .../worker/service/package-info.java | 22 + .../worker/FunctionAssignmentTailerTest.java | 6 +- .../worker/FunctionRuntimeManagerTest.java | 100 ++- .../worker/MembershipManagerTest.java | 8 +- .../worker/rest/api/FunctionsImplTest.java | 5 +- .../api/v2/FunctionApiV2ResourceTest.java | 11 +- .../api/v3/FunctionApiV3ResourceTest.java | 15 +- .../rest/api/v3/SinkApiV3ResourceTest.java | 11 +- .../rest/api/v3/SourceApiV3ResourceTest.java | 4 +- 69 files changed, 2539 insertions(+), 1076 deletions(-) create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/path/PolicyPath.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/path/package-info.java create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceDefinition.java create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceLoader.java create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceMetadata.java create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceWithClassLoader.java create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Component.java create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Functions.java create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/FunctionsV2.java create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Sinks.java create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Sources.java create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Workers.java create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/package-info.java create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/package-info.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index fff7c46c9ab84..dcfe0c8742369 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -1804,6 +1804,12 @@ public class ServiceConfiguration implements PulsarConfiguration { ) private boolean functionsWorkerEnabled = false; + @FieldContext( + category = CATEGORY_FUNCTIONS, + doc = "The nar package for the function worker service" + ) + private String functionsWorkerServiceNarPackage = ""; + /**** --- Broker Web Stats --- ****/ @FieldContext( category = CATEGORY_METRICS, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarBrokerStarter.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarBrokerStarter.java index c5403bd064854..bfe14d2cf150e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarBrokerStarter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarBrokerStarter.java @@ -47,12 +47,12 @@ import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.ServiceConfigurationUtils; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.naming.NamespaceBundleSplitAlgorithm; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; +import org.apache.pulsar.functions.worker.service.WorkerServiceLoader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.bridge.SLF4JBridgeHandler; @@ -129,6 +129,7 @@ private static class BrokerStarter { private final StatsProvider bookieStatsProvider; private final ServerConfiguration bookieConfig; private final WorkerService functionsWorkerService; + private final WorkerConfig workerConfig; BrokerStarter(String[] args) throws Exception{ StarterArguments starterArguments = new StarterArguments(); @@ -170,50 +171,18 @@ private static class BrokerStarter { // init functions worker if (starterArguments.runFunctionsWorker || brokerConfig.isFunctionsWorkerEnabled()) { - WorkerConfig workerConfig; - if (isBlank(starterArguments.fnWorkerConfigFile)) { - workerConfig = new WorkerConfig(); - } else { - workerConfig = WorkerConfig.load(starterArguments.fnWorkerConfigFile); - } - // worker talks to local broker - String hostname = ServiceConfigurationUtils.getDefaultOrConfiguredAddress( - brokerConfig.getAdvertisedAddress()); - workerConfig.setWorkerHostname(hostname); - workerConfig.setWorkerPort(brokerConfig.getWebServicePort().get()); - workerConfig.setWorkerId( - "c-" + brokerConfig.getClusterName() - + "-fw-" + hostname - + "-" + workerConfig.getWorkerPort()); - // inherit broker authorization setting - workerConfig.setAuthenticationEnabled(brokerConfig.isAuthenticationEnabled()); - workerConfig.setAuthenticationProviders(brokerConfig.getAuthenticationProviders()); - - workerConfig.setAuthorizationEnabled(brokerConfig.isAuthorizationEnabled()); - workerConfig.setAuthorizationProvider(brokerConfig.getAuthorizationProvider()); - workerConfig.setConfigurationStoreServers(brokerConfig.getConfigurationStoreServers()); - workerConfig.setZooKeeperSessionTimeoutMillis(brokerConfig.getZooKeeperSessionTimeoutMillis()); - workerConfig.setZooKeeperOperationTimeoutSeconds(brokerConfig.getZooKeeperOperationTimeoutSeconds()); - - workerConfig.setTlsAllowInsecureConnection(brokerConfig.isTlsAllowInsecureConnection()); - workerConfig.setTlsEnableHostnameVerification(false); - workerConfig.setBrokerClientTrustCertsFilePath(brokerConfig.getTlsTrustCertsFilePath()); - - // client in worker will use this config to authenticate with broker - workerConfig.setBrokerClientAuthenticationPlugin(brokerConfig.getBrokerClientAuthenticationPlugin()); - workerConfig.setBrokerClientAuthenticationParameters( - brokerConfig.getBrokerClientAuthenticationParameters()); - - // inherit super users - workerConfig.setSuperUserRoles(brokerConfig.getSuperUserRoles()); - - functionsWorkerService = new WorkerService(workerConfig); + workerConfig = PulsarService.initializeWorkerConfigFromBrokerConfig( + brokerConfig, starterArguments.fnWorkerConfigFile + ); + functionsWorkerService = WorkerServiceLoader.load(workerConfig); } else { + workerConfig = null; functionsWorkerService = null; } // init pulsar service pulsarService = new PulsarService(brokerConfig, + workerConfig, Optional.ofNullable(functionsWorkerService), (exitCode) -> { log.info("Halting broker process with code {}", diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java index b533fa3c02f6a..7f0ee96e67eb4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java @@ -19,9 +19,9 @@ package org.apache.pulsar; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES_ROOT; +import static org.apache.pulsar.common.policies.data.Policies.getBundles; import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; -import com.google.common.collect.Lists; import java.io.IOException; import java.util.Collections; import java.util.List; @@ -35,7 +35,6 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; -import org.apache.pulsar.common.policies.data.BundlesData; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.TenantInfo; @@ -355,22 +354,5 @@ public static ZooKeeper initZk(String connection, int sessionTimeout) throws Exc return zkConnect; } - private static BundlesData getBundles(int numBundles) { - Long maxVal = ((long) 1) << 32; - Long segSize = maxVal / numBundles; - List partitions = Lists.newArrayList(); - partitions.add(String.format("0x%08x", 0L)); - Long curPartition = segSize; - for (int i = 0; i < numBundles; i++) { - if (i != numBundles - 1) { - partitions.add(String.format("0x%08x", curPartition)); - } else { - partitions.add(String.format("0x%08x", maxVal - 1)); - } - curPartition += segSize; - } - return new BundlesData(partitions); - } - private static final Logger log = LoggerFactory.getLogger(PulsarClusterMetadataSetup.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java index 9cea534265263..51adb8e7ab9b8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar; -import static org.apache.commons.lang3.StringUtils.isBlank; import com.beust.jcommander.Parameter; import com.google.common.collect.Sets; import java.io.File; @@ -28,7 +27,6 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.ServiceConfigurationUtils; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -37,6 +35,7 @@ import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; +import org.apache.pulsar.functions.worker.service.WorkerServiceLoader; import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.slf4j.Logger; @@ -51,6 +50,7 @@ public class PulsarStandalone implements AutoCloseable { LocalBookkeeperEnsemble bkEnsemble; ServiceConfiguration config; WorkerService fnWorkerService; + WorkerConfig workerConfig; public void setBroker(PulsarService broker) { this.broker = broker; @@ -267,12 +267,8 @@ public void start() throws Exception { // initialize the functions worker if (!this.isNoFunctionsWorker()) { - WorkerConfig workerConfig; - if (isBlank(this.getFnWorkerConfigFile())) { - workerConfig = new WorkerConfig(); - } else { - workerConfig = WorkerConfig.load(this.getFnWorkerConfigFile()); - } + workerConfig = PulsarService.initializeWorkerConfigFromBrokerConfig( + config, this.getFnWorkerConfigFile()); // worker talks to local broker if (this.isNoStreamStorage()) { // only set the state storage service url when state is enabled. @@ -280,41 +276,14 @@ public void start() throws Exception { } else if (workerConfig.getStateStorageServiceUrl() == null) { workerConfig.setStateStorageServiceUrl("bk://127.0.0.1:" + this.getStreamStoragePort()); } - - String hostname = ServiceConfigurationUtils.getDefaultOrConfiguredAddress( - config.getAdvertisedAddress()); - workerConfig.setWorkerHostname(hostname); - workerConfig.setWorkerPort(config.getWebServicePort().get()); - workerConfig.setWorkerId( - "c-" + config.getClusterName() - + "-fw-" + hostname - + "-" + workerConfig.getWorkerPort()); - // inherit broker authorization setting - workerConfig.setAuthenticationEnabled(config.isAuthenticationEnabled()); - workerConfig.setAuthenticationProviders(config.getAuthenticationProviders()); - - workerConfig.setAuthorizationEnabled(config.isAuthorizationEnabled()); - workerConfig.setAuthorizationProvider(config.getAuthorizationProvider()); - workerConfig.setConfigurationStoreServers(config.getConfigurationStoreServers()); - workerConfig.setZooKeeperSessionTimeoutMillis(config.getZooKeeperSessionTimeoutMillis()); - workerConfig.setZooKeeperOperationTimeoutSeconds(config.getZooKeeperOperationTimeoutSeconds()); - - workerConfig.setTlsAllowInsecureConnection(config.isTlsAllowInsecureConnection()); - workerConfig.setTlsEnableHostnameVerification(false); - workerConfig.setBrokerClientTrustCertsFilePath(config.getTlsTrustCertsFilePath()); - - // client in worker will use this config to authenticate with broker - workerConfig.setBrokerClientAuthenticationPlugin(config.getBrokerClientAuthenticationPlugin()); - workerConfig.setBrokerClientAuthenticationParameters(config.getBrokerClientAuthenticationParameters()); - - // inherit super users - workerConfig.setSuperUserRoles(config.getSuperUserRoles()); - - fnWorkerService = new WorkerService(workerConfig); + fnWorkerService = WorkerServiceLoader.load(workerConfig); + } else { + workerConfig = new WorkerConfig(); } // Start Broker broker = new PulsarService(config, + workerConfig, Optional.ofNullable(fnWorkerService), (exitCode) -> { log.info("Halting standalone process with code {}", exitCode); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 8d9c564160ad4..2f0bf0cb9da90 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -19,20 +19,18 @@ package org.apache.pulsar.broker; import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.commons.lang3.StringUtils.isNotBlank; -import static org.apache.pulsar.broker.admin.impl.NamespacesBase.getBundles; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; import io.netty.util.concurrent.DefaultThreadFactory; import java.io.IOException; import java.lang.reflect.Method; import java.net.InetSocketAddress; -import java.net.URI; import java.util.Collections; import java.util.List; import java.util.Map; @@ -62,7 +60,6 @@ import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader; import org.apache.bookkeeper.mledger.offload.OffloaderUtils; import org.apache.bookkeeper.mledger.offload.Offloaders; -import org.apache.bookkeeper.util.ZkUtils; import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.builder.ReflectionToStringBuilder; @@ -104,24 +101,18 @@ import org.apache.pulsar.common.conf.InternalConfigurationData; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.configuration.VipStatus; -import org.apache.pulsar.common.naming.NamedEntity; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.OffloadPolicies; -import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.policies.data.RetentionPolicies; -import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.protocol.schema.SchemaStorage; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.compaction.Compactor; import org.apache.pulsar.compaction.TwoPhaseCompactor; import org.apache.pulsar.functions.worker.ErrorNotifier; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; -import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.packages.management.core.PackagesManagement; import org.apache.pulsar.packages.management.core.PackagesStorage; import org.apache.pulsar.packages.management.core.PackagesStorageProvider; @@ -141,9 +132,6 @@ import org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher.ShutdownService; import org.apache.pulsar.zookeeper.ZookeeperBkClientFactoryImpl; import org.apache.pulsar.zookeeper.ZookeeperSessionExpiredHandler; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.ZooKeeper; import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.websocket.servlet.WebSocketServlet; @@ -200,6 +188,7 @@ public class PulsarService implements AutoCloseable { private final String brokerVersion; private SchemaStorage schemaStorage = null; private SchemaRegistryService schemaRegistryService = null; + private final WorkerConfig workerConfig; private final Optional functionWorkerService; private ProtocolHandlers protocolHandlers = null; @@ -237,6 +226,13 @@ public PulsarService(ServiceConfiguration config) { public PulsarService(ServiceConfiguration config, Optional functionWorkerService, Consumer processTerminator) { + this(config, new WorkerConfig(), functionWorkerService, processTerminator); + } + + public PulsarService(ServiceConfiguration config, + WorkerConfig workerConfig, + Optional functionWorkerService, + Consumer processTerminator) { // Validate correctness of configuration PulsarConfigurationLoader.isComplete(config); // validate `advertisedAddress`, `advertisedListeners`, `internalListenerName` @@ -261,6 +257,7 @@ public PulsarService(ServiceConfiguration config, Optional functi this.shutdownService = new MessagingServiceShutdownHook(this, processTerminator); this.loadManagerExecutor = Executors .newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-load-manager")); + this.workerConfig = workerConfig; this.functionWorkerService = functionWorkerService; this.executor = Executors.newScheduledThreadPool(config.getNumExecutorThreadPoolSize(), new DefaultThreadFactory("pulsar")); @@ -401,7 +398,7 @@ public ServiceConfiguration getConfiguration() { * @return the current function worker service configuration. */ public Optional getWorkerConfig() { - return functionWorkerService.map(service -> service.getWorkerConfig()); + return functionWorkerService.map(service -> workerConfig); } public Map getProtocolDataToAdvertise() { @@ -877,6 +874,11 @@ public NamespaceService getNamespaceService() { return this.nsService; } + + public Optional getWorkerServiceOpt() { + return functionWorkerService; + } + public WorkerService getWorkerService() { return functionWorkerService.orElse(null); } @@ -1235,11 +1237,9 @@ public TopicPoliciesService getTopicPoliciesService() { private void startWorkerService(AuthenticationService authenticationService, AuthorizationService authorizationService) - throws InterruptedException, IOException, KeeperException { + throws Exception { if (functionWorkerService.isPresent()) { LOG.info("Starting function worker service"); - - WorkerConfig workerConfig = functionWorkerService.get().getWorkerConfig(); if (workerConfig.isUseTls()) { workerConfig.setPulsarServiceUrl(brokerServiceUrlTls); workerConfig.setPulsarWebServiceUrl(webServiceAddressTls); @@ -1247,98 +1247,20 @@ private void startWorkerService(AuthenticationService authenticationService, workerConfig.setPulsarServiceUrl(brokerServiceUrl); workerConfig.setPulsarWebServiceUrl(webServiceAddress); } - String namespace = functionWorkerService.get() - .getWorkerConfig().getPulsarFunctionsNamespace(); - String[] a = functionWorkerService.get().getWorkerConfig().getPulsarFunctionsNamespace().split("/"); - String property = a[0]; - String cluster = functionWorkerService.get().getWorkerConfig().getPulsarFunctionsCluster(); - - /* - multiple brokers may be trying to create the property, cluster, and namespace - for function worker service this in parallel. The function worker service uses the namespace - to create topics for internal function - */ - - // create property for function worker service - try { - NamedEntity.checkName(property); - this.getGlobalZkCache().getZooKeeper().create( - AdminResource.path(POLICIES, property), - ObjectMapperFactory.getThreadLocal().writeValueAsBytes( - new TenantInfo( - Sets.newHashSet(config.getSuperUserRoles()), - Sets.newHashSet(cluster))), - ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - LOG.info("Created property {} for function worker", property); - } catch (KeeperException.NodeExistsException e) { - LOG.debug("Failed to create already existing property {} for function worker service", cluster, e); - } catch (IllegalArgumentException e) { - LOG.error("Failed to create property with invalid name {} for function worker service", cluster, e); - throw e; - } catch (Exception e) { - LOG.error("Failed to create property {} for function worker", cluster, e); - throw e; - } - // create cluster for function worker service - try { - NamedEntity.checkName(cluster); - ClusterData clusterData = new ClusterData(this.getSafeWebServiceAddress(), null /* serviceUrlTls */, - brokerServiceUrl, null /* brokerServiceUrlTls */); - this.getGlobalZkCache().getZooKeeper().create( - AdminResource.path("clusters", cluster), - ObjectMapperFactory.getThreadLocal().writeValueAsBytes(clusterData), - ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - LOG.info("Created cluster {} for function worker", cluster); - } catch (KeeperException.NodeExistsException e) { - LOG.debug("Failed to create already existing cluster {} for function worker service", cluster, e); - } catch (IllegalArgumentException e) { - LOG.error("Failed to create cluster with invalid name {} for function worker service", cluster, e); - throw e; - } catch (Exception e) { - LOG.error("Failed to create cluster {} for function worker service", cluster, e); - throw e; - } - - // create namespace for function worker service - try { - Policies policies = new Policies(); - policies.retention_policies = new RetentionPolicies(-1, -1); - policies.replication_clusters = Collections.singleton( - functionWorkerService.get().getWorkerConfig().getPulsarFunctionsCluster()); - int defaultNumberOfBundles = this.getConfiguration().getDefaultNumberOfNamespaceBundles(); - policies.bundles = getBundles(defaultNumberOfBundles); - - this.getConfigurationCache().policiesCache().invalidate(AdminResource.path(POLICIES, namespace)); - ZkUtils.createFullPathOptimistic(this.getGlobalZkCache().getZooKeeper(), - AdminResource.path(POLICIES, namespace), - ObjectMapperFactory.getThreadLocal().writeValueAsBytes(policies), - ZooDefs.Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT); - LOG.info("Created namespace {} for function worker service", namespace); - } catch (KeeperException.NodeExistsException e) { - LOG.debug("Failed to create already existing namespace {} for function worker service", namespace); - } catch (Exception e) { - LOG.error("Failed to create namespace {}", namespace, e); - throw e; - } - - InternalConfigurationData internalConf = this.getInternalConfigurationData(); + functionWorkerService.get().initInBroker( + config, + workerConfig, + getGlobalZkCache(), + getConfigurationCacheService(), + getInternalConfigurationData() + ); - URI dlogURI; - try { - // initializing dlog namespace for function worker - dlogURI = WorkerUtils.initializeDlogNamespace(internalConf); - } catch (IOException ioe) { - LOG.error("Failed to initialize dlog namespace with zookeeper {}" - + " at metadata service uri {} for storing function packages", - internalConf.getZookeeperServers(), internalConf.getBookkeeperMetadataServiceUri(), ioe); - throw ioe; - } - LOG.info("Function worker service setup completed"); // TODO figure out how to handle errors from function worker service - functionWorkerService.get().start(dlogURI, authenticationService, - authorizationService, ErrorNotifier.getShutdownServiceImpl(shutdownService)); + functionWorkerService.get().start( + authenticationService, + authorizationService, + ErrorNotifier.getShutdownServiceImpl(shutdownService)); LOG.info("Function worker service started"); } } @@ -1370,4 +1292,45 @@ public Optional getBrokerListenPort() { public Optional getBrokerListenPortTls() { return brokerService.getListenPortTls(); } + + public static WorkerConfig initializeWorkerConfigFromBrokerConfig(ServiceConfiguration brokerConfig, + String workerConfigFile) throws IOException { + WorkerConfig workerConfig = WorkerConfig.load(workerConfigFile); + // worker talks to local broker + String hostname = ServiceConfigurationUtils.getDefaultOrConfiguredAddress( + brokerConfig.getAdvertisedAddress()); + workerConfig.setWorkerHostname(hostname); + workerConfig.setWorkerPort(brokerConfig.getWebServicePort().get()); + workerConfig.setWorkerId( + "c-" + brokerConfig.getClusterName() + + "-fw-" + hostname + + "-" + workerConfig.getWorkerPort()); + // inherit broker authorization setting + workerConfig.setAuthenticationEnabled(brokerConfig.isAuthenticationEnabled()); + workerConfig.setAuthenticationProviders(brokerConfig.getAuthenticationProviders()); + + workerConfig.setAuthorizationEnabled(brokerConfig.isAuthorizationEnabled()); + workerConfig.setAuthorizationProvider(brokerConfig.getAuthorizationProvider()); + workerConfig.setConfigurationStoreServers(brokerConfig.getConfigurationStoreServers()); + workerConfig.setZooKeeperSessionTimeoutMillis(brokerConfig.getZooKeeperSessionTimeoutMillis()); + workerConfig.setZooKeeperOperationTimeoutSeconds(brokerConfig.getZooKeeperOperationTimeoutSeconds()); + + workerConfig.setTlsAllowInsecureConnection(brokerConfig.isTlsAllowInsecureConnection()); + workerConfig.setTlsEnableHostnameVerification(false); + workerConfig.setBrokerClientTrustCertsFilePath(brokerConfig.getTlsTrustCertsFilePath()); + + // client in worker will use this config to authenticate with broker + workerConfig.setBrokerClientAuthenticationPlugin(brokerConfig.getBrokerClientAuthenticationPlugin()); + workerConfig.setBrokerClientAuthenticationParameters(brokerConfig.getBrokerClientAuthenticationParameters()); + + // inherit super users + workerConfig.setSuperUserRoles(brokerConfig.getSuperUserRoles()); + + // inherit the nar package locations + if (isBlank(workerConfig.getFunctionsWorkerServiceNarPackage())) { + workerConfig.setFunctionsWorkerServiceNarPackage( + brokerConfig.getFunctionsWorkerServiceNarPackage()); + } + return workerConfig; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java index 2d2806b660dee..9c1575fc8ed4d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java @@ -27,7 +27,6 @@ import java.io.IOException; import java.io.InputStream; import java.util.List; -import java.util.function.Supplier; import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; import javax.ws.rs.GET; @@ -48,21 +47,14 @@ import org.apache.pulsar.common.policies.data.FunctionStats; import org.apache.pulsar.common.policies.data.FunctionStatus; import org.apache.pulsar.functions.worker.WorkerService; -import org.apache.pulsar.functions.worker.rest.api.FunctionsImpl; +import org.apache.pulsar.functions.worker.service.api.Functions; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import org.glassfish.jersey.media.multipart.FormDataParam; -public class FunctionsBase extends AdminResource implements Supplier { +public class FunctionsBase extends AdminResource { - private final FunctionsImpl functions; - - public FunctionsBase() { - this.functions = new FunctionsImpl(this); - } - - @Override - public WorkerService get() { - return pulsar().getWorkerService(); + Functions functions() { + return pulsar().getWorkerService().getFunctions(); } @POST @@ -187,7 +179,7 @@ public void registerFunction( ) final @FormDataParam("functionConfig") FunctionConfig functionConfig) { - functions.registerFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, + functions().registerFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, functionPkgUrl, functionConfig, clientAppId(), clientAuthData()); } @@ -313,7 +305,7 @@ public void updateFunction( @ApiParam(value = "The update options is for the Pulsar Function that needs to be updated.") final @FormDataParam("updateOptions") UpdateOptions updateOptions) throws IOException { - functions.updateFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, + functions().updateFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, functionPkgUrl, functionConfig, clientAppId(), clientAuthData(), updateOptions); } @@ -335,7 +327,7 @@ public void deregisterFunction( final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Function") final @PathParam("functionName") String functionName) { - functions.deregisterFunction(tenant, namespace, functionName, clientAppId(), clientAuthData()); + functions().deregisterFunction(tenant, namespace, functionName, clientAppId(), clientAuthData()); } @GET @@ -357,7 +349,7 @@ public FunctionConfig getFunctionInfo( final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Function") final @PathParam("functionName") String functionName) throws IOException { - return functions.getFunctionInfo(tenant, namespace, functionName, clientAppId(), clientAuthData()); + return functions().getFunctionInfo(tenant, namespace, functionName, clientAppId(), clientAuthData()); } @GET @@ -380,7 +372,7 @@ public FunctionStatus.FunctionInstanceStatus.FunctionInstanceStatusData getFunct @ApiParam(value = "The instanceId of a Pulsar Function (if instance-id is not provided," + " the stats of all instances is returned") final @PathParam("instanceId") String instanceId) throws IOException { - return functions.getFunctionInstanceStatus(tenant, namespace, functionName, + return functions().getFunctionInstanceStatus(tenant, namespace, functionName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -404,7 +396,7 @@ public FunctionStatus getFunctionStatus( final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Function") final @PathParam("functionName") String functionName) throws IOException { - return functions.getFunctionStatus(tenant, namespace, functionName, uri.getRequestUri(), + return functions().getFunctionStatus(tenant, namespace, functionName, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -428,7 +420,7 @@ public FunctionStats getFunctionStats( final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Function") final @PathParam("functionName") String functionName) throws IOException { - return functions.getFunctionStats(tenant, namespace, functionName, + return functions().getFunctionStats(tenant, namespace, functionName, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -452,7 +444,7 @@ public FunctionStats.FunctionInstanceStats.FunctionInstanceStatsData getFunction @ApiParam(value = "The instanceId of a Pulsar Function" + " (if instance-id is not provided, the stats of all instances is returned") final @PathParam( "instanceId") String instanceId) throws IOException { - return functions.getFunctionsInstanceStats(tenant, namespace, functionName, instanceId, + return functions().getFunctionsInstanceStats(tenant, namespace, functionName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -472,7 +464,7 @@ public List listFunctions( final @PathParam("tenant") String tenant, @ApiParam(value = "The namespace of a Pulsar Function") final @PathParam("namespace") String namespace) { - return functions.listFunctions(tenant, namespace, clientAppId(), clientAuthData()); + return functions().listFunctions(tenant, namespace, clientAppId(), clientAuthData()); } @POST @@ -500,7 +492,7 @@ public String triggerFunction( @ApiParam(value = "The specific topic name that the Pulsar Function" + " consumes from which you want to inject the data to") final @FormDataParam("topic") String topic) { - return functions.triggerFunction(tenant, namespace, functionName, triggerValue, + return functions().triggerFunction(tenant, namespace, functionName, triggerValue, triggerStream, topic, clientAppId(), clientAuthData()); } @@ -525,7 +517,7 @@ public FunctionState getFunctionState( final @PathParam("functionName") String functionName, @ApiParam(value = "The stats key") final @PathParam("key") String key) { - return functions.getFunctionState(tenant, namespace, functionName, key, clientAppId(), clientAuthData()); + return functions().getFunctionState(tenant, namespace, functionName, key, clientAppId(), clientAuthData()); } @POST @@ -545,7 +537,7 @@ public void putFunctionState(final @PathParam("tenant") String tenant, final @PathParam("functionName") String functionName, final @PathParam("key") String key, final @FormDataParam("state") FunctionState stateJson) { - functions.putFunctionState(tenant, namespace, functionName, key, stateJson, clientAppId(), clientAuthData()); + functions().putFunctionState(tenant, namespace, functionName, key, stateJson, clientAppId(), clientAuthData()); } @POST @@ -565,7 +557,7 @@ public void restartFunction( @ApiParam(value = "The instanceId of a Pulsar Function (if instance-id is not provided, all instances are restarted") final @PathParam("instanceId") String instanceId) { - functions.restartFunctionInstance(tenant, namespace, functionName, instanceId, + functions().restartFunctionInstance(tenant, namespace, functionName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -585,7 +577,7 @@ public void restartFunction( final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Function") final @PathParam("functionName") String functionName) { - functions.restartFunctionInstances(tenant, namespace, functionName, clientAppId(), clientAuthData()); + functions().restartFunctionInstances(tenant, namespace, functionName, clientAppId(), clientAuthData()); } @POST @@ -604,7 +596,7 @@ public void stopFunction( @ApiParam(value = "The instanceId of a Pulsar Function (if instance-id is not provided, all instances are stopped. ") final @PathParam("instanceId") String instanceId) { - functions.stopFunctionInstance(tenant, namespace, functionName, instanceId, + functions().stopFunctionInstance(tenant, namespace, functionName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -624,7 +616,7 @@ public void stopFunction( final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Function") final @PathParam("functionName") String functionName) { - functions.stopFunctionInstances(tenant, namespace, functionName, clientAppId(), clientAuthData()); + functions().stopFunctionInstances(tenant, namespace, functionName, clientAppId(), clientAuthData()); } @POST @@ -643,7 +635,7 @@ public void startFunction( @ApiParam(value = "The instanceId of a Pulsar Function" + " (if instance-id is not provided, all instances sre started. ") final @PathParam("instanceId") String instanceId) { - functions.startFunctionInstance(tenant, namespace, functionName, instanceId, + functions().startFunctionInstance(tenant, namespace, functionName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -663,7 +655,7 @@ public void startFunction( final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Function") final @PathParam("functionName") String functionName) { - functions.startFunctionInstances(tenant, namespace, functionName, clientAppId(), clientAuthData()); + functions().startFunctionInstances(tenant, namespace, functionName, clientAppId(), clientAuthData()); } @POST @@ -675,7 +667,7 @@ public void startFunction( @Consumes(MediaType.MULTIPART_FORM_DATA) public void uploadFunction(final @FormDataParam("data") InputStream uploadedInputStream, final @FormDataParam("path") String path) { - functions.uploadFunction(uploadedInputStream, path, clientAppId()); + functions().uploadFunction(uploadedInputStream, path, clientAppId()); } @GET @@ -685,7 +677,7 @@ public void uploadFunction(final @FormDataParam("data") InputStream uploadedInpu ) @Path("/download") public StreamingOutput downloadFunction(final @QueryParam("path") String path) { - return functions.downloadFunction(path, clientAppId(), clientAuthData()); + return functions().downloadFunction(path, clientAppId(), clientAuthData()); } @GET @@ -702,7 +694,7 @@ public StreamingOutput downloadFunction( @ApiParam(value = "The name of a Pulsar Function") final @PathParam("functionName") String functionName) { - return functions.downloadFunction(tenant, namespace, functionName, clientAppId(), clientAuthData()); + return functions().downloadFunction(tenant, namespace, functionName, clientAppId(), clientAuthData()); } @GET @@ -721,7 +713,7 @@ public StreamingOutput downloadFunction( * Deprecated in favor of moving endpoint to {@link org.apache.pulsar.broker.admin.v2.Worker} */ public List getConnectorsList() throws IOException { - return functions.getListOfConnectors(); + return functions().getListOfConnectors(); } @PUT @@ -742,7 +734,7 @@ public void updateFunctionOnWorkerLeader(final @PathParam("tenant") String tenan InputStream uploadedInputStream, final @FormDataParam("delete") boolean delete) { - functions.updateFunctionOnWorkerLeader(tenant, namespace, functionName, uploadedInputStream, + functions().updateFunctionOnWorkerLeader(tenant, namespace, functionName, uploadedInputStream, delete, uri.getRequestUri(), clientAppId()); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java index ec9a923cffd5e..da65977dd72aa 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java @@ -23,6 +23,7 @@ import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; import static org.apache.pulsar.broker.cache.LocalZooKeeperCacheService.LOCAL_POLICIES_ROOT; +import static org.apache.pulsar.common.policies.data.Policies.getBundles; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.collect.Sets.SetView; @@ -103,8 +104,6 @@ public abstract class NamespacesBase extends AdminResource { - private static final long MAX_BUNDLES = ((long) 1) << 32; - protected List internalGetTenantNamespaces(String tenant) { checkNotNull(tenant, "Tenant should not be null"); try { @@ -2523,27 +2522,6 @@ protected BundlesData validateBundlesData(BundlesData initialBundles) { return new BundlesData(bundles); } - public static BundlesData getBundles(int numBundles) { - if (numBundles <= 0 || numBundles > MAX_BUNDLES) { - throw new RestException(Status.BAD_REQUEST, - "Invalid number of bundles. Number of numbles has to be in the range of (0, 2^32]."); - } - Long maxVal = ((long) 1) << 32; - Long segSize = maxVal / numBundles; - List partitions = Lists.newArrayList(); - partitions.add(String.format("0x%08x", 0L)); - Long curPartition = segSize; - for (int i = 0; i < numBundles; i++) { - if (i != numBundles - 1) { - partitions.add(String.format("0x%08x", curPartition)); - } else { - partitions.add(String.format("0x%08x", maxVal - 1)); - } - curPartition += segSize; - } - return new BundlesData(partitions); - } - private void validatePolicies(NamespaceName ns, Policies policies) { if (ns.isV2() && policies.replication_clusters.isEmpty()) { // Default to local cluster diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java index a38ef6dc63d73..d79eeeef97249 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java @@ -27,7 +27,6 @@ import java.io.IOException; import java.io.InputStream; import java.util.List; -import java.util.function.Supplier; import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; import javax.ws.rs.GET; @@ -44,21 +43,14 @@ import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.common.policies.data.SinkStatus; import org.apache.pulsar.functions.worker.WorkerService; -import org.apache.pulsar.functions.worker.rest.api.SinksImpl; +import org.apache.pulsar.functions.worker.service.api.Sinks; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import org.glassfish.jersey.media.multipart.FormDataParam; -public class SinksBase extends AdminResource implements Supplier { +public class SinksBase extends AdminResource { - private final SinksImpl sink; - - public SinksBase() { - this.sink = new SinksImpl(this); - } - - @Override - public WorkerService get() { - return pulsar().getWorkerService(); + Sinks sinks() { + return pulsar().getWorkerService().getSinks(); } @POST @@ -159,7 +151,7 @@ public void registerSink(@ApiParam(value = "The tenant of a Pulsar Sink") final ) ) final @FormDataParam("sinkConfig") SinkConfig sinkConfig) { - sink.registerSink(tenant, namespace, sinkName, uploadedInputStream, fileDetail, + sinks().registerSink(tenant, namespace, sinkName, uploadedInputStream, fileDetail, sinkPkgUrl, sinkConfig, clientAppId(), clientAuthData()); } @@ -261,7 +253,7 @@ public void updateSink(@ApiParam(value = "The tenant of a Pulsar Sink") final @P final @FormDataParam("sinkConfig") SinkConfig sinkConfig, @ApiParam(value = "Update options for the Pulsar Sink") final @FormDataParam("updateOptions") UpdateOptions updateOptions) { - sink.updateSink(tenant, namespace, sinkName, uploadedInputStream, fileDetail, + sinks().updateSink(tenant, namespace, sinkName, uploadedInputStream, fileDetail, sinkPkgUrl, sinkConfig, clientAppId(), clientAuthData(), updateOptions); } @@ -286,7 +278,7 @@ public void deregisterSink(@ApiParam(value = "The tenant of a Pulsar Sink") final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Sink") final @PathParam("sinkName") String sinkName) { - sink.deregisterFunction(tenant, namespace, sinkName, clientAppId(), clientAuthData()); + sinks().deregisterFunction(tenant, namespace, sinkName, clientAppId(), clientAuthData()); } @GET @@ -306,7 +298,7 @@ public SinkConfig getSinkInfo(@ApiParam(value = "The tenant of a Pulsar Sink") final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Sink") final @PathParam("sinkName") String sinkName) throws IOException { - return sink.getSinkInfo(tenant, namespace, sinkName); + return sinks().getSinkInfo(tenant, namespace, sinkName); } @GET @@ -332,7 +324,7 @@ public SinkStatus.SinkInstanceStatus.SinkInstanceStatusData getSinkInstanceStatu final @PathParam("sinkName") String sinkName, @ApiParam(value = "The instanceId of a Pulsar Sink") final @PathParam("instanceId") String instanceId) throws IOException { - return sink.getSinkInstanceStatus( + return sinks().getSinkInstanceStatus( tenant, namespace, sinkName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -356,7 +348,7 @@ public SinkStatus getSinkStatus(@ApiParam(value = "The tenant of a Pulsar Sink") final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Sink") final @PathParam("sinkName") String sinkName) throws IOException { - return sink.getSinkStatus(tenant, namespace, sinkName, uri.getRequestUri(), clientAppId(), clientAuthData()); + return sinks().getSinkStatus(tenant, namespace, sinkName, uri.getRequestUri(), clientAppId(), clientAuthData()); } @GET @@ -376,7 +368,7 @@ public List listSinks(@ApiParam(value = "The tenant of a Pulsar Sink") final @PathParam("tenant") String tenant, @ApiParam(value = "The namespace of a Pulsar Sink") final @PathParam("namespace") String namespace) { - return sink.listFunctions(tenant, namespace, clientAppId(), clientAuthData()); + return sinks().listFunctions(tenant, namespace, clientAppId(), clientAuthData()); } @POST @@ -401,7 +393,7 @@ public void restartSink(@ApiParam(value = "The tenant of a Pulsar Sink") final @PathParam("sinkName") String sinkName, @ApiParam(value = "The instanceId of a Pulsar Sink") final @PathParam("instanceId") String instanceId) { - sink.restartFunctionInstance(tenant, namespace, sinkName, instanceId, + sinks().restartFunctionInstance(tenant, namespace, sinkName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -423,7 +415,7 @@ public void restartSink(@ApiParam(value = "The tenant of a Pulsar Sink") final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Sink") final @PathParam("sinkName") String sinkName) { - sink.restartFunctionInstances(tenant, namespace, sinkName, clientAppId(), clientAuthData()); + sinks().restartFunctionInstances(tenant, namespace, sinkName, clientAppId(), clientAuthData()); } @POST @@ -446,7 +438,7 @@ public void stopSink(@ApiParam(value = "The tenant of a Pulsar Sink") final @PathParam("sinkName") String sinkName, @ApiParam(value = "The instanceId of a Pulsar Sink") final @PathParam("instanceId") String instanceId) { - sink.stopFunctionInstance(tenant, namespace, + sinks().stopFunctionInstance(tenant, namespace, sinkName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -468,7 +460,7 @@ public void stopSink(@ApiParam(value = "The tenant of a Pulsar Sink") final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Sink") final @PathParam("sinkName") String sinkName) { - sink.stopFunctionInstances(tenant, namespace, sinkName, clientAppId(), clientAuthData()); + sinks().stopFunctionInstances(tenant, namespace, sinkName, clientAppId(), clientAuthData()); } @POST @@ -491,7 +483,7 @@ public void startSink(@ApiParam(value = "The tenant of a Pulsar Sink") final @PathParam("sinkName") String sinkName, @ApiParam(value = "The instanceId of a Pulsar Sink") final @PathParam("instanceId") String instanceId) { - sink.startFunctionInstance(tenant, namespace, sinkName, instanceId, + sinks().startFunctionInstance(tenant, namespace, sinkName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -513,7 +505,7 @@ public void startSink(@ApiParam(value = "The tenant of a Pulsar Sink") final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Sink") final @PathParam("sinkName") String sinkName) { - sink.startFunctionInstances(tenant, namespace, sinkName, clientAppId(), clientAuthData()); + sinks().startFunctionInstances(tenant, namespace, sinkName, clientAppId(), clientAuthData()); } @GET @@ -527,7 +519,7 @@ public void startSink(@ApiParam(value = "The tenant of a Pulsar Sink") }) @Path("/builtinsinks") public List getSinkList() { - return sink.getSinkList(); + return sinks().getSinkList(); } @GET @@ -547,7 +539,7 @@ public List getSinkList() { public List getSinkConfigDefinition( @ApiParam(value = "The name of the builtin sink") final @PathParam("name") String name) throws IOException { - return sink.getSinkConfigDefinition(name); + return sinks().getSinkConfigDefinition(name); } @POST @@ -562,6 +554,6 @@ public List getSinkConfigDefinition( }) @Path("/reloadBuiltInSinks") public void reloadSinks() { - sink.reloadConnectors(clientAppId()); + sinks().reloadConnectors(clientAppId()); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java index cdd159515754f..1d0f555cec26d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java @@ -27,7 +27,6 @@ import java.io.IOException; import java.io.InputStream; import java.util.List; -import java.util.function.Supplier; import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; import javax.ws.rs.GET; @@ -44,21 +43,14 @@ import org.apache.pulsar.common.io.SourceConfig; import org.apache.pulsar.common.policies.data.SourceStatus; import org.apache.pulsar.functions.worker.WorkerService; -import org.apache.pulsar.functions.worker.rest.api.SourcesImpl; +import org.apache.pulsar.functions.worker.service.api.Sources; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import org.glassfish.jersey.media.multipart.FormDataParam; -public class SourcesBase extends AdminResource implements Supplier { +public class SourcesBase extends AdminResource { - private final SourcesImpl source; - - public SourcesBase() { - this.source = new SourcesImpl(this); - } - - @Override - public WorkerService get() { - return pulsar().getWorkerService(); + Sources sources() { + return pulsar().getWorkerService().getSources(); } @POST @@ -137,7 +129,7 @@ public void registerSource( ) ) final @FormDataParam("sourceConfig") SourceConfig sourceConfig) { - source.registerSource(tenant, namespace, sourceName, uploadedInputStream, fileDetail, + sources().registerSource(tenant, namespace, sourceName, uploadedInputStream, fileDetail, sourcePkgUrl, sourceConfig, clientAppId(), clientAuthData()); } @@ -220,7 +212,7 @@ public void updateSource( final @FormDataParam("sourceConfig") SourceConfig sourceConfig, @ApiParam(value = "Update options for Pulsar Source") final @FormDataParam("updateOptions") UpdateOptions updateOptions) { - source.updateSource(tenant, namespace, sourceName, uploadedInputStream, fileDetail, + sources().updateSource(tenant, namespace, sourceName, uploadedInputStream, fileDetail, sourcePkgUrl, sourceConfig, clientAppId(), clientAuthData(), updateOptions); } @@ -244,7 +236,7 @@ public void deregisterSource( final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Source") final @PathParam("sourceName") String sourceName) { - source.deregisterFunction(tenant, namespace, sourceName, clientAppId(), clientAuthData()); + sources().deregisterFunction(tenant, namespace, sourceName, clientAppId(), clientAuthData()); } @GET @@ -265,7 +257,7 @@ public SourceConfig getSourceInfo( final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Source") final @PathParam("sourceName") String sourceName) throws IOException { - return source.getSourceInfo(tenant, namespace, sourceName); + return sources().getSourceInfo(tenant, namespace, sourceName); } @GET @@ -287,7 +279,7 @@ public SourceStatus.SourceInstanceStatus.SourceInstanceStatusData getSourceInsta @ApiParam(value = "The instanceId of a Pulsar Source" + " (if instance-id is not provided, the stats of all instances is returned).") final @PathParam( "instanceId") String instanceId) throws IOException { - return source.getSourceInstanceStatus( + return sources().getSourceInstanceStatus( tenant, namespace, sourceName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -310,7 +302,7 @@ public SourceStatus getSourceStatus( final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Source") final @PathParam("sourceName") String sourceName) throws IOException { - return source.getSourceStatus(tenant, namespace, sourceName, uri.getRequestUri(), clientAppId(), + return sources().getSourceStatus(tenant, namespace, sourceName, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -333,7 +325,7 @@ public List listSources( final @PathParam("tenant") String tenant, @ApiParam(value = "The namespace of a Pulsar Source") final @PathParam("namespace") String namespace) { - return source.listFunctions(tenant, namespace, clientAppId(), clientAuthData()); + return sources().listFunctions(tenant, namespace, clientAppId(), clientAuthData()); } @POST @@ -355,7 +347,7 @@ public void restartSource( @ApiParam(value = "The instanceId of a Pulsar Source" + " (if instance-id is not provided, the stats of all instances is returned).") final @PathParam( "instanceId") String instanceId) { - source.restartFunctionInstance(tenant, namespace, sourceName, instanceId, + sources().restartFunctionInstance(tenant, namespace, sourceName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -377,7 +369,7 @@ public void restartSource( final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Source") final @PathParam("sourceName") String sourceName) { - source.restartFunctionInstances(tenant, namespace, sourceName, clientAppId(), clientAuthData()); + sources().restartFunctionInstances(tenant, namespace, sourceName, clientAppId(), clientAuthData()); } @POST @@ -397,7 +389,7 @@ public void stopSource( @ApiParam(value = "The name of a Pulsar Source") final @PathParam("sourceName") String sourceName, @ApiParam(value = "The instanceId of a Pulsar Source (if instance-id is not provided," + " the stats of all instances is returned).") final @PathParam("instanceId") String instanceId) { - source.stopFunctionInstance(tenant, namespace, sourceName, instanceId, + sources().stopFunctionInstance(tenant, namespace, sourceName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -419,7 +411,7 @@ public void stopSource( final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Source") final @PathParam("sourceName") String sourceName) { - source.stopFunctionInstances(tenant, namespace, sourceName, clientAppId(), clientAuthData()); + sources().stopFunctionInstances(tenant, namespace, sourceName, clientAppId(), clientAuthData()); } @POST @@ -439,7 +431,7 @@ public void startSource( @ApiParam(value = "The name of a Pulsar Source") final @PathParam("sourceName") String sourceName, @ApiParam(value = "The instanceId of a Pulsar Source (if instance-id is not provided," + " the stats of all instances is returned).") final @PathParam("instanceId") String instanceId) { - source.startFunctionInstance(tenant, namespace, sourceName, instanceId, + sources().startFunctionInstance(tenant, namespace, sourceName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -461,7 +453,7 @@ public void startSource( final @PathParam("namespace") String namespace, @ApiParam(value = "The name of a Pulsar Source") final @PathParam("sourceName") String sourceName) { - source.startFunctionInstances(tenant, namespace, sourceName, clientAppId(), clientAuthData()); + sources().startFunctionInstances(tenant, namespace, sourceName, clientAppId(), clientAuthData()); } @GET @@ -479,7 +471,7 @@ public void startSource( @Produces(MediaType.APPLICATION_JSON) @Path("/builtinsources") public List getSourceList() { - return source.getSourceList(); + return sources().getSourceList(); } @GET @@ -499,7 +491,7 @@ public List getSourceList() { public List getSourceConfigDefinition( @ApiParam(value = "The name of the builtin source") final @PathParam("name") String name) throws IOException { - return source.getSourceConfigDefinition(name); + return sources().getSourceConfigDefinition(name); } @POST @@ -514,6 +506,6 @@ public List getSourceConfigDefinition( }) @Path("/reloadBuiltInSources") public void reloadSources() { - source.reloadConnectors(clientAppId()); + sources().reloadConnectors(clientAppId()); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java index 32288750847d7..4e55688e2c5f7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.admin.v1; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; +import static org.apache.pulsar.common.policies.data.Policies.getBundles; import com.google.common.collect.Lists; import io.swagger.annotations.Api; import io.swagger.annotations.ApiOperation; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Functions.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Functions.java index 6a8c99a01721d..a83ea8cd51473 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Functions.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Functions.java @@ -25,7 +25,6 @@ import java.io.IOException; import java.io.InputStream; import java.util.List; -import java.util.function.Supplier; import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; import javax.ws.rs.GET; @@ -43,7 +42,7 @@ import org.apache.pulsar.functions.proto.Function.FunctionMetaData; import org.apache.pulsar.functions.proto.InstanceCommunication.FunctionStatus; import org.apache.pulsar.functions.worker.WorkerService; -import org.apache.pulsar.functions.worker.rest.api.FunctionsImplV2; +import org.apache.pulsar.functions.worker.service.api.FunctionsV2; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import org.glassfish.jersey.media.multipart.FormDataParam; @@ -51,17 +50,10 @@ @Api(value = "/functions", description = "Functions admin apis", tags = "functions", hidden = true) @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON) -public class Functions extends AdminResource implements Supplier { +public class Functions extends AdminResource { - private final FunctionsImplV2 functions; - - public Functions() { - this.functions = new FunctionsImplV2(this); - } - - @Override - public WorkerService get() { - return pulsar().getWorkerService(); + FunctionsV2 functions() { + return pulsar().getWorkerService().getFunctionsV2(); } @POST @@ -82,7 +74,7 @@ public Response registerFunction(final @PathParam("tenant") String tenant, final @FormDataParam("url") String functionPkgUrl, final @FormDataParam("functionDetails") String functionDetailsJson) { - return functions.registerFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, + return functions().registerFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, functionPkgUrl, functionDetailsJson, clientAppId()); } @@ -103,7 +95,7 @@ public Response updateFunction(final @PathParam("tenant") String tenant, final @FormDataParam("url") String functionPkgUrl, final @FormDataParam("functionDetails") String functionDetailsJson) { - return functions.updateFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, + return functions().updateFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, functionPkgUrl, functionDetailsJson, clientAppId()); } @@ -121,7 +113,7 @@ public Response updateFunction(final @PathParam("tenant") String tenant, public Response deregisterFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) { - return functions.deregisterFunction(tenant, namespace, functionName, clientAppId()); + return functions().deregisterFunction(tenant, namespace, functionName, clientAppId()); } @GET @@ -140,7 +132,7 @@ public Response getFunctionInfo(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) throws IOException { - return functions.getFunctionInfo( + return functions().getFunctionInfo( tenant, namespace, functionName, clientAppId()); } @@ -161,7 +153,7 @@ public Response getFunctionInstanceStatus(final @PathParam("tenant") String tena final @PathParam("functionName") String functionName, final @PathParam("instanceId") String instanceId) throws IOException { - return functions.getFunctionInstanceStatus(tenant, namespace, functionName, instanceId, uri.getRequestUri(), + return functions().getFunctionInstanceStatus(tenant, namespace, functionName, instanceId, uri.getRequestUri(), clientAppId()); } @@ -179,7 +171,7 @@ public Response getFunctionInstanceStatus(final @PathParam("tenant") String tena public Response getFunctionStatus(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) throws IOException { - return functions.getFunctionStatusV2( + return functions().getFunctionStatusV2( tenant, namespace, functionName, uri.getRequestUri(), clientAppId()); } @@ -196,7 +188,7 @@ public Response getFunctionStatus(final @PathParam("tenant") String tenant, @Path("/{tenant}/{namespace}") public Response listFunctions(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace) { - return functions.listFunctions(tenant, namespace, clientAppId()); + return functions().listFunctions(tenant, namespace, clientAppId()); } @POST @@ -218,7 +210,7 @@ public Response triggerFunction(final @PathParam("tenant") String tenant, final @FormDataParam("data") String triggerValue, final @FormDataParam("dataStream") InputStream triggerStream, final @FormDataParam("topic") String topic) { - return functions.triggerFunction(tenant, namespace, functionName, + return functions().triggerFunction(tenant, namespace, functionName, triggerValue, triggerStream, topic, clientAppId()); } @@ -238,7 +230,7 @@ public Response getFunctionState(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName, final @PathParam("key") String key) { - return functions.getFunctionState(tenant, namespace, functionName, key, clientAppId()); + return functions().getFunctionState(tenant, namespace, functionName, key, clientAppId()); } @POST @@ -254,7 +246,7 @@ public Response restartFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName, final @PathParam("instanceId") String instanceId) { - return functions.restartFunctionInstance(tenant, namespace, functionName, + return functions().restartFunctionInstance(tenant, namespace, functionName, instanceId, uri.getRequestUri(), clientAppId()); } @@ -268,7 +260,7 @@ public Response restartFunction(final @PathParam("tenant") String tenant, public Response restartFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) { - return functions.restartFunctionInstances(tenant, namespace, functionName, clientAppId()); + return functions().restartFunctionInstances(tenant, namespace, functionName, clientAppId()); } @POST @@ -282,7 +274,7 @@ public Response stopFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName, final @PathParam("instanceId") String instanceId) { - return functions.stopFunctionInstance(tenant, namespace, functionName, + return functions().stopFunctionInstance(tenant, namespace, functionName, instanceId, uri.getRequestUri(), clientAppId()); } @@ -296,7 +288,7 @@ public Response stopFunction(final @PathParam("tenant") String tenant, public Response stopFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) { - return functions.stopFunctionInstances(tenant, namespace, functionName, clientAppId()); + return functions().stopFunctionInstances(tenant, namespace, functionName, clientAppId()); } @POST @@ -308,7 +300,7 @@ public Response stopFunction(final @PathParam("tenant") String tenant, @Consumes(MediaType.MULTIPART_FORM_DATA) public Response uploadFunction(final @FormDataParam("data") InputStream uploadedInputStream, final @FormDataParam("path") String path) { - return functions.uploadFunction(uploadedInputStream, path, clientAppId()); + return functions().uploadFunction(uploadedInputStream, path, clientAppId()); } @GET @@ -318,7 +310,7 @@ public Response uploadFunction(final @FormDataParam("data") InputStream uploaded ) @Path("/download") public Response downloadFunction(final @QueryParam("path") String path) { - return functions.downloadFunction(path, clientAppId()); + return functions().downloadFunction(path, clientAppId()); } @GET @@ -333,6 +325,6 @@ public Response downloadFunction(final @QueryParam("path") String path) { }) @Path("/connectors") public List getConnectorsList() throws IOException { - return functions.getListOfConnectors(); + return functions().getListOfConnectors(); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java index 72cb4a40b7d9c..14460fc693612 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.admin.v2; +import static org.apache.pulsar.common.policies.data.Policies.getBundles; import io.swagger.annotations.Api; import io.swagger.annotations.ApiOperation; import io.swagger.annotations.ApiParam; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java index f432d46069c2d..dac46b276242a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java @@ -36,16 +36,14 @@ import org.apache.pulsar.common.functions.WorkerInfo; import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.functions.worker.WorkerService; -import org.apache.pulsar.functions.worker.rest.api.WorkerImpl; +import org.apache.pulsar.functions.worker.service.api.Workers; @Slf4j @Path("/worker") public class Worker extends AdminResource implements Supplier { - private final WorkerImpl worker; - - public Worker() { - this.worker = new WorkerImpl(this); + Workers workers() { + return pulsar().getWorkerService().getWorkers(); } @Override @@ -66,7 +64,7 @@ public WorkerService get() { @Path("/cluster") @Produces(MediaType.APPLICATION_JSON) public List getCluster() { - return worker.getCluster(clientAppId()); + return workers().getCluster(clientAppId()); } @GET @@ -81,7 +79,7 @@ public List getCluster() { @Path("/cluster/leader") @Produces(MediaType.APPLICATION_JSON) public WorkerInfo getClusterLeader() { - return worker.getClusterLeader(clientAppId()); + return workers().getClusterLeader(clientAppId()); } @GET @@ -96,7 +94,7 @@ public WorkerInfo getClusterLeader() { @Path("/assignments") @Produces(MediaType.APPLICATION_JSON) public Map> getAssignments() { - return worker.getAssignments(clientAppId()); + return workers().getAssignments(clientAppId()); } @GET @@ -112,7 +110,7 @@ public Map> getAssignments() { @Path("/connectors") @Produces(MediaType.APPLICATION_JSON) public List getConnectorsList() throws IOException { - return worker.getListOfConnectors(clientAppId()); + return workers().getListOfConnectors(clientAppId()); } @PUT @@ -126,7 +124,7 @@ public List getConnectorsList() throws IOException { }) @Path("/rebalance") public void rebalance() { - worker.rebalance(uri.getRequestUri(), clientAppId()); + workers().rebalance(uri.getRequestUri(), clientAppId()); } @GET @@ -139,6 +137,6 @@ public void rebalance() { }) @Path("/cluster/leader/ready") public Boolean isLeaderReady() { - return worker.isLeaderReady(clientAppId()); + return workers().isLeaderReady(clientAppId()); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/WorkerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/WorkerStats.java index 43d480c7c5007..6703caa7a278b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/WorkerStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/WorkerStats.java @@ -24,7 +24,6 @@ import java.io.IOException; import java.util.Collection; import java.util.List; -import java.util.function.Supplier; import javax.ws.rs.GET; import javax.ws.rs.Path; import javax.ws.rs.Produces; @@ -32,22 +31,16 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.common.policies.data.WorkerFunctionInstanceStats; +import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.functions.worker.WorkerService; -import org.apache.pulsar.functions.worker.rest.api.WorkerImpl; +import org.apache.pulsar.functions.worker.service.api.Workers; @Slf4j @Path("/worker-stats") -public class WorkerStats extends AdminResource implements Supplier { +public class WorkerStats extends AdminResource { - private final WorkerImpl worker; - - public WorkerStats() { - this.worker = new WorkerImpl(this); - } - - @Override - public WorkerService get() { - return pulsar().getWorkerService(); + public Workers workers() { + return pulsar().getWorkerService().getWorkers(); } @GET @@ -62,8 +55,8 @@ public WorkerService get() { @ApiResponse(code = 503, message = "Worker service is not running") }) @Produces(MediaType.APPLICATION_JSON) - public Collection getMetrics() throws Exception { - return worker.getWorkerMetrics(clientAppId()); + public Collection getMetrics() throws Exception { + return workers().getWorkerMetrics(clientAppId()); } @GET @@ -79,6 +72,6 @@ public Collection getMetrics() throws Ex }) @Produces(MediaType.APPLICATION_JSON) public List getStats() throws IOException { - return worker.getFunctionsMetrics(clientAppId()); + return workers().getFunctionsMetrics(clientAppId()); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java index 7f7e217f4f498..a1d4235992ca6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java @@ -45,7 +45,6 @@ import org.apache.pulsar.broker.stats.metrics.ManagedLedgerMetrics; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.SimpleTextOutputStream; -import org.apache.pulsar.functions.worker.FunctionsStatsGenerator; /** * Generate metrics aggregated at the namespace level and optionally at a topic level and formats them out @@ -93,8 +92,9 @@ public static void generate(PulsarService pulsar, boolean includeTopicMetrics, b NamespaceStatsAggregator.generate(pulsar, includeTopicMetrics, includeConsumerMetrics, stream); - FunctionsStatsGenerator.generate(pulsar.getWorkerService(), - pulsar.getConfiguration().getClusterName(), stream); + if (pulsar.getWorkerServiceOpt().isPresent()) { + pulsar.getWorkerService().generateFunctionsStats(stream); + } generateBrokerBasicMetrics(pulsar, stream); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java index 9918e75b60959..94f181c195e3c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java @@ -22,14 +22,11 @@ import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; -import com.google.common.base.Joiner; -import com.google.common.base.Splitter; import com.google.common.collect.BoundType; import com.google.common.collect.Range; import java.net.MalformedURLException; import java.net.URI; import java.net.URL; -import java.util.Iterator; import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -65,6 +62,7 @@ import org.apache.pulsar.common.policies.data.PolicyOperation; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantOperation; +import org.apache.pulsar.common.policies.path.PolicyPath; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -102,26 +100,15 @@ protected ServiceConfiguration config() { } public static String path(String... parts) { - StringBuilder sb = new StringBuilder(); - sb.append("/admin/"); - Joiner.on('/').appendTo(sb, parts); - return sb.toString(); + return PolicyPath.path(parts); } public static String joinPath(String... parts) { - StringBuilder sb = new StringBuilder(); - Joiner.on('/').appendTo(sb, parts); - return sb.toString(); + return PolicyPath.joinPath(parts); } public static String splitPath(String source, int slice) { - Iterable parts = Splitter.on('/').limit(slice).split(source); - Iterator s = parts.iterator(); - String result = ""; - for (int i = 0; i < slice; i++) { - result = s.next(); - } - return result; + return PolicyPath.splitPath(source, slice); } /** diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java index 578de1055fff0..bbfdccf4c9533 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java @@ -92,7 +92,7 @@ public class PulsarFunctionE2ESecurityTest { PulsarAdmin superUserAdmin; PulsarClient pulsarClient; BrokerStats brokerStatsClient; - WorkerService functionsWorkerService; + PulsarWorkerService functionsWorkerService; final String TENANT = "external-repl-prop"; final String TENANT2 = "tenant2"; @@ -156,7 +156,7 @@ void setup(Method method) throws Exception { "token:" + adminToken); functionsWorkerService = createPulsarFunctionWorker(config); Optional functionWorkerService = Optional.of(functionsWorkerService); - pulsar = new PulsarService(config, functionWorkerService, (exitCode) -> {}); + pulsar = new PulsarService(config, workerConfig, functionWorkerService, (exitCode) -> {}); pulsar.start(); brokerServiceUrl = pulsar.getWebServiceAddress(); @@ -201,7 +201,7 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { superUserAdmin.tenants().createTenant(TENANT2, propAdmin); superUserAdmin.namespaces().createNamespace( TENANT2 + "/" + NAMESPACE); - while (!functionWorkerService.get().getLeaderService().isLeader()) { + while (!functionsWorkerService.getLeaderService().isLeader()) { Thread.sleep(1000); } } @@ -216,7 +216,7 @@ void shutdown() throws Exception { bkEnsemble.stop(); } - private WorkerService createPulsarFunctionWorker(ServiceConfiguration config) { + private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration config) { System.setProperty(JAVA_INSTANCE_JAR_PROPERTY, FutureUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath()); @@ -253,7 +253,9 @@ private WorkerService createPulsarFunctionWorker(ServiceConfiguration config) { workerConfig.setAuthorizationEnabled(config.isAuthorizationEnabled()); workerConfig.setAuthorizationProvider(config.getAuthorizationProvider()); - return new WorkerService(workerConfig); + PulsarWorkerService workerService = new PulsarWorkerService(); + workerService.init(workerConfig, null, false); + return workerService; } protected static FunctionConfig createFunctionConfig(String tenant, String namespace, String functionName, String sourceTopic, String sinkTopic, String subscriptionName) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionLocalRunTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionLocalRunTest.java index 99ff990514235..621d46f31c882 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionLocalRunTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionLocalRunTest.java @@ -106,7 +106,7 @@ public class PulsarFunctionLocalRunTest { PulsarAdmin admin; PulsarClient pulsarClient; BrokerStats brokerStatsClient; - WorkerService functionsWorkerService; + PulsarWorkerService functionsWorkerService; final String tenant = "external-repl-prop"; String pulsarFunctionsNamespace = tenant + "/pulsar-function-admin"; String primaryHost; @@ -180,7 +180,7 @@ void setup(Method method) throws Exception { functionsWorkerService = createPulsarFunctionWorker(config); Optional functionWorkerService = Optional.of(functionsWorkerService); - pulsar = new PulsarService(config, functionWorkerService, (exitCode) -> {}); + pulsar = new PulsarService(config, workerConfig, functionWorkerService, (exitCode) -> {}); pulsar.start(); String brokerServiceUrl = pulsar.getWebServiceAddressTls(); @@ -284,7 +284,7 @@ void shutdown() throws Exception { bkEnsemble.stop(); } - private WorkerService createPulsarFunctionWorker(ServiceConfiguration config) { + private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration config) { System.setProperty(JAVA_INSTANCE_JAR_PROPERTY, FutureUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath()); @@ -322,7 +322,9 @@ private WorkerService createPulsarFunctionWorker(ServiceConfiguration config) { workerConfig.setAuthenticationEnabled(true); workerConfig.setAuthorizationEnabled(true); - return new WorkerService(workerConfig); + PulsarWorkerService workerService = new PulsarWorkerService(); + workerService.init(workerConfig, null, false); + return workerService; } protected static FunctionConfig createFunctionConfig(String tenant, String namespace, String functionName, String sourceTopic, String sinkTopic, String subscriptionName) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java index 2eca82ed4a3ad..7ad906d03e2ba 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java @@ -95,7 +95,7 @@ public class PulsarFunctionPublishTest { PulsarAdmin admin; PulsarClient pulsarClient; BrokerStats brokerStatsClient; - WorkerService functionsWorkerService; + PulsarWorkerService functionsWorkerService; final String tenant = "external-repl-prop"; String pulsarFunctionsNamespace = tenant + "/pulsar-function-admin"; String primaryHost; @@ -164,7 +164,7 @@ void setup(Method method) throws Exception { functionsWorkerService = createPulsarFunctionWorker(config); Optional functionWorkerService = Optional.of(functionsWorkerService); - pulsar = new PulsarService(config, functionWorkerService, (exitCode) -> {}); + pulsar = new PulsarService(config, workerConfig, functionWorkerService, (exitCode) -> {}); pulsar.start(); String brokerServiceUrl = pulsar.getWebServiceAddressTls(); @@ -206,7 +206,7 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { System.setProperty(JAVA_INSTANCE_JAR_PROPERTY, FutureUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath()); - while(!functionWorkerService.get().getLeaderService().isLeader()) { + while(!functionsWorkerService.getLeaderService().isLeader()) { Thread.sleep(1000); } } @@ -221,7 +221,7 @@ void shutdown() throws Exception { bkEnsemble.stop(); } - private WorkerService createPulsarFunctionWorker(ServiceConfiguration config) { + private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration config) { workerConfig = new WorkerConfig(); workerConfig.setPulsarFunctionsNamespace(pulsarFunctionsNamespace); @@ -256,7 +256,9 @@ private WorkerService createPulsarFunctionWorker(ServiceConfiguration config) { workerConfig.setAuthenticationEnabled(true); workerConfig.setAuthorizationEnabled(true); - return new WorkerService(workerConfig); + PulsarWorkerService workerService = new PulsarWorkerService(); + workerService.init(workerConfig, null, false); + return workerService; } protected static FunctionConfig createFunctionConfig(String tenant, String namespace, String functionName, String sourceTopic, String publishTopic, String subscriptionName) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java index fad174c591276..86a9ad27747c3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java @@ -72,7 +72,7 @@ public class PulsarWorkerAssignmentTest { PulsarAdmin admin; PulsarClient pulsarClient; BrokerStats brokerStatsClient; - WorkerService functionsWorkerService; + PulsarWorkerService functionsWorkerService; final String tenant = "external-repl-prop"; final String pulsarFunctionsNamespace = tenant + "/pulsar-function-admin"; String primaryHost; @@ -99,7 +99,7 @@ void setup(Method method) throws Exception { functionsWorkerService = createPulsarFunctionWorker(config); final Optional functionWorkerService = Optional.of(functionsWorkerService); - pulsar = new PulsarService(config, functionWorkerService, (exitCode) -> {}); + pulsar = new PulsarService(config, workerConfig, functionWorkerService, (exitCode) -> {}); pulsar.start(); admin = spy(PulsarAdmin.builder().serviceHttpUrl(pulsar.getWebServiceAddress()).build()); @@ -136,7 +136,7 @@ void shutdown() { } } - private WorkerService createPulsarFunctionWorker(ServiceConfiguration config) { + private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration config) { workerConfig = new WorkerConfig(); workerConfig.setPulsarFunctionsNamespace(pulsarFunctionsNamespace); workerConfig.setSchedulerClassName( @@ -161,7 +161,9 @@ private WorkerService createPulsarFunctionWorker(ServiceConfiguration config) { workerConfig.setWorkerId(workerId); workerConfig.setTopicCompactionFrequencySec(1); - return new WorkerService(workerConfig); + PulsarWorkerService workerService = new PulsarWorkerService(); + workerService.init(workerConfig, null, false); + return workerService; } @Test(timeOut = 60000, enabled = false) @@ -286,8 +288,9 @@ public void testFunctionAssignmentsWithRestart() throws Exception { // (3) Restart worker service and check registered functions final URI dlUri = functionsWorkerService.getDlogUri(); functionsWorkerService.stop(); - functionsWorkerService = new WorkerService(workerConfig); - functionsWorkerService.start(dlUri, new AuthenticationService(PulsarConfigurationLoader.convertFrom(workerConfig)), null, ErrorNotifier.getDefaultImpl()); + functionsWorkerService = new PulsarWorkerService(); + functionsWorkerService.init(workerConfig, dlUri, false); + functionsWorkerService.start(new AuthenticationService(PulsarConfigurationLoader.convertFrom(workerConfig)), null, ErrorNotifier.getDefaultImpl()); final FunctionRuntimeManager runtimeManager2 = functionsWorkerService.getFunctionRuntimeManager(); retryStrategically((test) -> { try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java index fd31b49b714f2..ab6072c709d3d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java @@ -48,9 +48,9 @@ import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory; import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactoryConfig; +import org.apache.pulsar.functions.worker.PulsarWorkerService; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; -import org.apache.pulsar.functions.worker.rest.WorkerServer; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -71,8 +71,7 @@ public class PulsarFunctionAdminTest { PulsarAdmin admin; PulsarClient pulsarClient; BrokerStats brokerStatsClient; - WorkerServer functionsWorkerServer; - WorkerService functionsWorkerService; + PulsarWorkerService functionsWorkerService; final String tenant = "external-repl-prop"; String pulsarFunctionsNamespace = tenant + "/pulsar-function-admin"; String primaryHost; @@ -116,7 +115,7 @@ void setup(Method method) throws Exception { functionsWorkerService = createPulsarFunctionWorker(config); Optional functionWorkerService = Optional.of(functionsWorkerService); - pulsar = new PulsarService(config, functionWorkerService, (exitCode) -> {}); + pulsar = new PulsarService(config, workerConfig, functionWorkerService, (exitCode) -> {}); pulsar.start(); urlTls = new URL(pulsar.getBrokerServiceUrlTls()); @@ -168,7 +167,7 @@ void shutdown() throws Exception { bkEnsemble.stop(); } - private WorkerService createPulsarFunctionWorker(ServiceConfiguration config) { + private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration config) { workerConfig = new WorkerConfig(); workerConfig.setPulsarFunctionsNamespace(pulsarFunctionsNamespace); workerConfig.setSchedulerClassName( @@ -199,6 +198,8 @@ private WorkerService createPulsarFunctionWorker(ServiceConfiguration config) { workerConfig.setTlsAllowInsecureConnection(true); workerConfig.setTlsTrustCertsFilePath(TLS_CLIENT_CERT_FILE_PATH); - return new WorkerService(workerConfig); + PulsarWorkerService workerService = new PulsarWorkerService(); + workerService.init(workerConfig, null, false); + return workerService; } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java index 4649b3d099f22..6ccd74f428463 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java @@ -101,6 +101,7 @@ import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.worker.FunctionRuntimeManager; import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactoryConfig; +import org.apache.pulsar.functions.worker.PulsarWorkerService; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; @@ -125,7 +126,7 @@ public class PulsarFunctionE2ETest { PulsarAdmin admin; PulsarClient pulsarClient; BrokerStats brokerStatsClient; - WorkerService functionsWorkerService; + PulsarWorkerService functionsWorkerService; final String tenant = "external-repl-prop"; String pulsarFunctionsNamespace = tenant + "/pulsar-function-admin"; String primaryHost; @@ -197,7 +198,7 @@ void setup(Method method) throws Exception { functionsWorkerService = createPulsarFunctionWorker(config); Optional functionWorkerService = Optional.of(functionsWorkerService); - pulsar = new PulsarService(config, functionWorkerService, (exitCode) -> {}); + pulsar = new PulsarService(config, workerConfig, functionWorkerService, (exitCode) -> {}); pulsar.start(); Map authParams = new HashMap<>(); @@ -311,7 +312,7 @@ void shutdown() throws Exception { bkEnsemble.stop(); } - private WorkerService createPulsarFunctionWorker(ServiceConfiguration config) { + private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration config) { System.setProperty(JAVA_INSTANCE_JAR_PROPERTY, FutureUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath()); @@ -346,7 +347,9 @@ private WorkerService createPulsarFunctionWorker(ServiceConfiguration config) { workerConfig.setAuthenticationEnabled(true); workerConfig.setAuthorizationEnabled(true); - return new WorkerService(workerConfig); + PulsarWorkerService workerService = new PulsarWorkerService(); + workerService.init(workerConfig, null, false); + return workerService; } protected static FunctionConfig createFunctionConfig(String tenant, String namespace, String functionName, String sourceTopic, String sinkTopic, String subscriptionName) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java index ef0826bff6c3e..c69fae807fcc3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.io; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -49,6 +50,7 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.Tenants; import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.policies.data.TenantInfo; @@ -59,8 +61,9 @@ import org.apache.pulsar.functions.sink.PulsarSink; import org.apache.pulsar.functions.worker.FunctionMetaDataManager; import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactoryConfig; +import org.apache.pulsar.functions.worker.PulsarWorkerService; +import org.apache.pulsar.functions.worker.PulsarWorkerService.PulsarClientCreator; import org.apache.pulsar.functions.worker.WorkerConfig; -import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.rest.WorkerServer; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.slf4j.Logger; @@ -79,7 +82,7 @@ public class PulsarFunctionTlsTest { ServiceConfiguration config; WorkerConfig workerConfig; URL urlTls; - WorkerService functionsWorkerService; + PulsarWorkerService functionsWorkerService; final String tenant = "external-repl-prop"; String pulsarFunctionsNamespace = tenant + "/use/pulsar-function-admin"; String workerId; @@ -117,17 +120,10 @@ void setup(Method method) throws Exception { config.setTlsKeyFilePath(TLS_SERVER_KEY_FILE_PATH); config.setTlsAllowInsecureConnection(true); config.setAdvertisedAddress("localhost"); - functionsWorkerService = spy(createPulsarFunctionWorker(config)); - AuthenticationService authenticationService = new AuthenticationService(config); - AuthorizationService authorizationService = new AuthorizationService(config, mock(ConfigurationCacheService.class)); - when(functionsWorkerService.getAuthenticationService()).thenReturn(authenticationService); - when(functionsWorkerService.getAuthorizationService()).thenReturn(authorizationService); - when(functionsWorkerService.isInitialized()).thenReturn(true); PulsarAdmin admin = mock(PulsarAdmin.class); Tenants tenants = mock(Tenants.class); when(admin.tenants()).thenReturn(tenants); - when(functionsWorkerService.getBrokerAdmin()).thenReturn(admin); Set admins = Sets.newHashSet("superUser"); TenantInfo tenantInfo = new TenantInfo(admins, null); when(tenants.getTenantInfo(any())).thenReturn(tenantInfo); @@ -135,12 +131,23 @@ void setup(Method method) throws Exception { when(admin.namespaces()).thenReturn(namespaces); when(namespaces.getNamespaces(any())).thenReturn(namespaceList); + functionsWorkerService = spy(createPulsarFunctionWorker(config, admin)); + doNothing().when(functionsWorkerService).initAsStandalone(any(WorkerConfig.class)); + when(functionsWorkerService.getBrokerAdmin()).thenReturn(admin); + functionsWorkerService.init(workerConfig, null, false); + + AuthenticationService authenticationService = new AuthenticationService(config); + AuthorizationService authorizationService = new AuthorizationService(config, mock(ConfigurationCacheService.class)); + when(functionsWorkerService.getAuthenticationService()).thenReturn(authenticationService); + when(functionsWorkerService.getAuthorizationService()).thenReturn(authorizationService); + when(functionsWorkerService.isInitialized()).thenReturn(true); + // mock: once authentication passes, function should return response: function already exist FunctionMetaDataManager dataManager = mock(FunctionMetaDataManager.class); when(dataManager.containsFunction(any(), any(), any())).thenReturn(true); when(functionsWorkerService.getFunctionMetaDataManager()).thenReturn(dataManager); - workerServer = new WorkerServer(functionsWorkerService); + workerServer = new WorkerServer(functionsWorkerService, authenticationService); workerServer.start(); Thread.sleep(2000); String functionTlsUrl = String.format("https://%s:%s", @@ -168,7 +175,8 @@ void shutdown() throws Exception { functionsWorkerService.stop(); } - private WorkerService createPulsarFunctionWorker(ServiceConfiguration config) { + private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration config, + PulsarAdmin mockPulsarAdmin) throws Exception { workerConfig = new WorkerConfig(); workerConfig.setPulsarFunctionsNamespace(pulsarFunctionsNamespace); workerConfig.setSchedulerClassName( @@ -207,7 +215,19 @@ private WorkerService createPulsarFunctionWorker(ServiceConfiguration config) { workerConfig.setAuthenticationEnabled(true); workerConfig.setAuthorizationEnabled(true); - return new WorkerService(workerConfig); + PulsarWorkerService workerService = new PulsarWorkerService(new PulsarClientCreator() { + @Override + public PulsarAdmin newPulsarAdmin(String pulsarServiceUrl, WorkerConfig workerConfig) { + return mockPulsarAdmin; + } + + @Override + public PulsarClient newPulsarClient(String pulsarServiceUrl, WorkerConfig workerConfig) { + return null; + } + }); + + return workerService; } @Test diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java index a7ee4f764bccb..a5e218441f6a2 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java @@ -26,6 +26,8 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import javax.ws.rs.core.Response.Status; +import org.apache.pulsar.common.util.RestException; /** * Definition of Pulsar policies. @@ -237,4 +239,28 @@ public String toString() { .add("is_allow_auto_update_Schema", is_allow_auto_update_schema) .add("offload_policies", offload_policies).toString(); } + + + private static final long MAX_BUNDLES = ((long) 1) << 32; + + public static BundlesData getBundles(int numBundles) { + if (numBundles <= 0 || numBundles > MAX_BUNDLES) { + throw new RestException(Status.BAD_REQUEST, + "Invalid number of bundles. Number of numbles has to be in the range of (0, 2^32]."); + } + Long maxVal = ((long) 1) << 32; + Long segSize = maxVal / numBundles; + List partitions = Lists.newArrayList(); + partitions.add(String.format("0x%08x", 0L)); + Long curPartition = segSize; + for (int i = 0; i < numBundles; i++) { + if (i != numBundles - 1) { + partitions.add(String.format("0x%08x", curPartition)); + } else { + partitions.add(String.format("0x%08x", maxVal - 1)); + } + curPartition += segSize; + } + return new BundlesData(partitions); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/path/PolicyPath.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/path/PolicyPath.java new file mode 100644 index 0000000000000..b6e0973e1f28c --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/path/PolicyPath.java @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.pulsar.common.policies.path; + +import com.google.common.base.Joiner; +import com.google.common.base.Splitter; +import java.util.Iterator; + +/** + * Policy path utilities. + */ +public class PolicyPath { + + public static String path(String... parts) { + StringBuilder sb = new StringBuilder(); + sb.append("/admin/"); + Joiner.on('/').appendTo(sb, parts); + return sb.toString(); + } + + public static String joinPath(String... parts) { + StringBuilder sb = new StringBuilder(); + Joiner.on('/').appendTo(sb, parts); + return sb.toString(); + } + + public static String splitPath(String source, int slice) { + Iterable parts = Splitter.on('/').limit(slice).split(source); + Iterator s = parts.iterator(); + String result = ""; + for (int i = 0; i < slice; i++) { + result = s.next(); + } + return result; + } + +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/path/package-info.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/path/package-info.java new file mode 100644 index 0000000000000..8db40590b4d54 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/path/package-info.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** + * Paths for pulsar policies. + */ +package org.apache.pulsar.common.policies.path; \ No newline at end of file diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java index 18f1a966d7721..77058ac9ab76b 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.functions.worker; +import static org.apache.commons.lang3.StringUtils.isBlank; + import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; @@ -406,6 +408,15 @@ public boolean getTlsEnabled() { ) private String brokerClientTrustCertsFilePath; + public String getBrokerClientTrustCertsFilePath() { + // for compatible, if user do not define brokerClientTrustCertsFilePath, we will use tlsTrustCertsFilePath, + // otherwise we will use brokerClientTrustCertsFilePath + if (StringUtils.isNotBlank(brokerClientTrustCertsFilePath)) { + return brokerClientTrustCertsFilePath; + } else { + return tlsTrustCertsFilePath; + } + } /******** Function Runtime configurations **********/ @@ -497,20 +508,29 @@ public String getFunctionAssignmentTopic() { return String.format("persistent://%s/%s", pulsarFunctionsNamespace, functionAssignmentTopicName); } + @FieldContext( + category = CATEGORY_WORKER, + doc = "The nar package for the function worker service" + ) + private String functionsWorkerServiceNarPackage = ""; + public static WorkerConfig load(String yamlFile) throws IOException { + if (isBlank(yamlFile)) { + return new WorkerConfig(); + } ObjectMapper mapper = new ObjectMapper(new YAMLFactory()); return mapper.readValue(new File(yamlFile), WorkerConfig.class); } public String getWorkerId() { - if (StringUtils.isBlank(this.workerId)) { + if (isBlank(this.workerId)) { this.workerId = String.format("%s-%s", this.getWorkerHostname(), this.getWorkerPort()); } return this.workerId; } public String getWorkerHostname() { - if (StringUtils.isBlank(this.workerHostname)) { + if (isBlank(this.workerHostname)) { this.workerHostname = unsafeLocalhostResolve(); } return this.workerHostname; diff --git a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeFactoryTest.java b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeFactoryTest.java index 62d71c0614e1e..a281187f9e0cc 100644 --- a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeFactoryTest.java +++ b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeFactoryTest.java @@ -147,6 +147,7 @@ KubernetesRuntimeFactory createKubernetesRuntimeFactory(String extraDepsDir, Optional functionAuthProvider, Optional manifestCustomizer) throws Exception { KubernetesRuntimeFactory factory = spy(new KubernetesRuntimeFactory()); + doNothing().when(factory).setupClient(); WorkerConfig workerConfig = new WorkerConfig(); KubernetesRuntimeFactoryConfig kubernetesRuntimeFactoryConfig = new KubernetesRuntimeFactoryConfig(); @@ -182,7 +183,6 @@ KubernetesRuntimeFactory createKubernetesRuntimeFactory(String extraDepsDir, workerConfig.setAuthenticationEnabled(false); factory.initialize(workerConfig,null, new TestSecretProviderConfigurator(), functionAuthProvider, manifestCustomizer); - doNothing().when(factory).setupClient(); return factory; } diff --git a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java index cf59b26c6d76c..ee7d1c5277213 100644 --- a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java +++ b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java @@ -181,6 +181,7 @@ KubernetesRuntimeFactory createKubernetesRuntimeFactory(String extraDepsDir, int double cpuOverCommitRatio, double memoryOverCommitRatio, Optional manifestCustomizer) throws Exception { KubernetesRuntimeFactory factory = spy(new KubernetesRuntimeFactory()); + doNothing().when(factory).setupClient(); WorkerConfig workerConfig = new WorkerConfig(); KubernetesRuntimeFactoryConfig kubernetesRuntimeFactoryConfig = new KubernetesRuntimeFactoryConfig(); @@ -215,8 +216,6 @@ KubernetesRuntimeFactory createKubernetesRuntimeFactory(String extraDepsDir, int workerConfig.setAuthenticationEnabled(false); factory.initialize(workerConfig, null, new TestSecretProviderConfigurator(), Optional.empty(), manifestCustomizer); - doNothing().when(factory).setupClient(); - return factory; } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionRuntimeManager.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionRuntimeManager.java index 009aaa9a31ecc..67b72be8b6365 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionRuntimeManager.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionRuntimeManager.java @@ -130,7 +130,7 @@ public int size() { private final PulsarAdmin functionAdmin; @Getter - private WorkerService workerService; + private PulsarWorkerService workerService; boolean isInitializePhase = false; @@ -143,7 +143,7 @@ public int size() { private final ErrorNotifier errorNotifier; - public FunctionRuntimeManager(WorkerConfig workerConfig, WorkerService workerService, Namespace dlogNamespace, + public FunctionRuntimeManager(WorkerConfig workerConfig, PulsarWorkerService workerService, Namespace dlogNamespace, MembershipManager membershipManager, ConnectorsManager connectorsManager, FunctionsManager functionsManager, FunctionMetaDataManager functionMetaDataManager, WorkerStatsManager workerStatsManager, ErrorNotifier errorNotifier) throws Exception { this.workerConfig = workerConfig; diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionsStatsGenerator.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionsStatsGenerator.java index 142caf634cb64..7dd1e74b8e5bc 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionsStatsGenerator.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionsStatsGenerator.java @@ -35,7 +35,7 @@ public class FunctionsStatsGenerator { private static final Logger log = LoggerFactory.getLogger(FunctionsStatsGenerator.class); - public static void generate(WorkerService workerService, String cluster, SimpleTextOutputStream out) { + public static void generate(PulsarWorkerService workerService, SimpleTextOutputStream out) { // only when worker service is initialized, we generate the stats. otherwise we will get bunch of NPE. if (workerService != null && workerService.isInitialized()) { diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java new file mode 100644 index 0000000000000..528954a9c9643 --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java @@ -0,0 +1,633 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.pulsar.functions.worker; + +import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; +import static org.apache.pulsar.common.policies.data.Policies.getBundles; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Sets; +import io.netty.util.concurrent.DefaultThreadFactory; +import java.io.IOException; +import java.net.URI; +import java.util.Collections; +import java.util.HashSet; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import javax.ws.rs.core.Response; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.clients.StorageClientBuilder; +import org.apache.bookkeeper.clients.admin.StorageAdminClient; +import org.apache.bookkeeper.clients.config.StorageClientSettings; +import org.apache.bookkeeper.util.ZkUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.distributedlog.DistributedLogConfiguration; +import org.apache.distributedlog.api.namespace.Namespace; +import org.apache.distributedlog.api.namespace.NamespaceBuilder; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.broker.cache.ConfigurationCacheService; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.common.conf.InternalConfigurationData; +import org.apache.pulsar.common.naming.NamedEntity; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.path.PolicyPath; +import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.apache.pulsar.common.util.SimpleTextOutputStream; +import org.apache.pulsar.functions.worker.rest.api.FunctionsImpl; +import org.apache.pulsar.functions.worker.rest.api.FunctionsImplV2; +import org.apache.pulsar.functions.worker.rest.api.SinksImpl; +import org.apache.pulsar.functions.worker.rest.api.SourcesImpl; +import org.apache.pulsar.functions.worker.rest.api.WorkerImpl; +import org.apache.pulsar.functions.worker.service.api.Functions; +import org.apache.pulsar.functions.worker.service.api.FunctionsV2; +import org.apache.pulsar.functions.worker.service.api.Sinks; +import org.apache.pulsar.functions.worker.service.api.Sources; +import org.apache.pulsar.functions.worker.service.api.Workers; +import org.apache.pulsar.zookeeper.ZooKeeperCache; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A service component contains everything to run a worker except rest server. + */ +@Slf4j +@Getter +public class PulsarWorkerService implements WorkerService { + + private static final Logger LOG = LoggerFactory.getLogger(PulsarWorkerService.class); + + public interface PulsarClientCreator { + + PulsarAdmin newPulsarAdmin(String pulsarServiceUrl, WorkerConfig workerConfig); + + PulsarClient newPulsarClient(String pulsarServiceUrl, WorkerConfig workerConfig); + + } + + private WorkerConfig workerConfig; + + private PulsarClient client; + private FunctionRuntimeManager functionRuntimeManager; + private FunctionMetaDataManager functionMetaDataManager; + private ClusterServiceCoordinator clusterServiceCoordinator; + // dlog namespace for storing function jars in bookkeeper + private Namespace dlogNamespace; + // storage client for accessing state storage for functions + private StorageAdminClient stateStoreAdminClient; + private MembershipManager membershipManager; + private SchedulerManager schedulerManager; + private volatile boolean isInitialized = false; + private ScheduledExecutorService statsUpdater; + private AuthenticationService authenticationService; + private AuthorizationService authorizationService; + private ConnectorsManager connectorsManager; + private FunctionsManager functionsManager; + private PulsarAdmin brokerAdmin; + private PulsarAdmin functionAdmin; + private MetricsGenerator metricsGenerator; + @VisibleForTesting + private URI dlogUri; + private LeaderService leaderService; + private FunctionAssignmentTailer functionAssignmentTailer; + private WorkerStatsManager workerStatsManager; + private Functions functions; + private FunctionsV2 functionsV2; + private Sinks sinks; + private Sources sources; + private Workers workers; + + private final PulsarClientCreator clientCreator; + + public PulsarWorkerService() { + this.clientCreator = new PulsarClientCreator() { + @Override + public PulsarAdmin newPulsarAdmin(String pulsarServiceUrl, WorkerConfig workerConfig) { + // using isBrokerClientAuthenticationEnabled instead of isAuthenticationEnabled in function-worker + if (workerConfig.isBrokerClientAuthenticationEnabled()) { + return WorkerUtils.getPulsarAdminClient( + pulsarServiceUrl, + workerConfig.getBrokerClientAuthenticationPlugin(), + workerConfig.getBrokerClientAuthenticationParameters(), + workerConfig.getBrokerClientTrustCertsFilePath(), + workerConfig.isTlsAllowInsecureConnection(), + workerConfig.isTlsEnableHostnameVerification()); + } else { + return WorkerUtils.getPulsarAdminClient(pulsarServiceUrl); + } + } + + @Override + public PulsarClient newPulsarClient(String pulsarServiceUrl, WorkerConfig workerConfig) { + // using isBrokerClientAuthenticationEnabled instead of isAuthenticationEnabled in function-worker + if (workerConfig.isBrokerClientAuthenticationEnabled()) { + return WorkerUtils.getPulsarClient( + pulsarServiceUrl, + workerConfig.getBrokerClientAuthenticationPlugin(), + workerConfig.getBrokerClientAuthenticationParameters(), + workerConfig.isUseTls(), + workerConfig.getBrokerClientTrustCertsFilePath(), + workerConfig.isTlsAllowInsecureConnection(), + workerConfig.isTlsEnableHostnameVerification()); + } else { + return WorkerUtils.getPulsarClient(pulsarServiceUrl); + } + } + }; + } + + public PulsarWorkerService(PulsarClientCreator clientCreator) { + this.clientCreator = clientCreator; + } + + @Override + public void generateFunctionsStats(SimpleTextOutputStream out) { + FunctionsStatsGenerator.generate( + this, out + ); + } + + @VisibleForTesting + public void init(WorkerConfig workerConfig, + URI dlogUri, + boolean runAsStandalone) { + this.statsUpdater = Executors + .newSingleThreadScheduledExecutor(new DefaultThreadFactory("worker-stats-updater")); + this.metricsGenerator = new MetricsGenerator(this.statsUpdater, workerConfig); + this.workerConfig = workerConfig; + this.dlogUri = dlogUri; + this.workerStatsManager = new WorkerStatsManager(workerConfig, runAsStandalone); + this.functions = new FunctionsImpl(() -> PulsarWorkerService.this); + this.functionsV2 = new FunctionsImplV2(() -> PulsarWorkerService.this); + this.sinks = new SinksImpl(() -> PulsarWorkerService.this); + this.sources = new SourcesImpl(() -> PulsarWorkerService.this); + this.workers = new WorkerImpl(() -> PulsarWorkerService.this); + } + + @Override + public void initAsStandalone(WorkerConfig workerConfig) throws Exception { + URI dlogUri = initializeStandaloneWorkerService(clientCreator, workerConfig); + init(workerConfig, dlogUri, true); + } + + private static URI initializeStandaloneWorkerService(PulsarClientCreator clientCreator, + WorkerConfig workerConfig) throws Exception { + // initializing pulsar functions namespace + PulsarAdmin admin = clientCreator.newPulsarAdmin(workerConfig.getPulsarWebServiceUrl(), workerConfig); + InternalConfigurationData internalConf; + // make sure pulsar broker is up + log.info("Checking if pulsar service at {} is up...", workerConfig.getPulsarWebServiceUrl()); + int maxRetries = workerConfig.getInitialBrokerReconnectMaxRetries(); + int retries = 0; + while (true) { + try { + admin.clusters().getClusters(); + break; + } catch (PulsarAdminException e) { + log.warn("Failed to retrieve clusters from pulsar service", e); + log.warn("Retry to connect to Pulsar service at {}", workerConfig.getPulsarWebServiceUrl()); + if (retries >= maxRetries) { + log.error("Failed to connect to Pulsar service at {} after {} attempts", + workerConfig.getPulsarFunctionsNamespace(), maxRetries); + throw e; + } + retries ++; + Thread.sleep(1000); + } + } + + // getting namespace policy + log.info("Initializing Pulsar Functions namespace..."); + try { + try { + admin.namespaces().getPolicies(workerConfig.getPulsarFunctionsNamespace()); + } catch (PulsarAdminException e) { + if (e.getStatusCode() == Response.Status.NOT_FOUND.getStatusCode()) { + // if not found than create + try { + Policies policies = new Policies(); + policies.retention_policies = new RetentionPolicies(-1, -1); + policies.replication_clusters = new HashSet<>(); + policies.replication_clusters.add(workerConfig.getPulsarFunctionsCluster()); + admin.namespaces().createNamespace(workerConfig.getPulsarFunctionsNamespace(), + policies); + } catch (PulsarAdminException e1) { + // prevent race condition with other workers starting up + if (e1.getStatusCode() != Response.Status.CONFLICT.getStatusCode()) { + log.error("Failed to create namespace {} for pulsar functions", workerConfig + .getPulsarFunctionsNamespace(), e1); + throw e1; + } + } + } else { + log.error("Failed to get retention policy for pulsar function namespace {}", + workerConfig.getPulsarFunctionsNamespace(), e); + throw e; + } + } + try { + internalConf = admin.brokers().getInternalConfigurationData(); + } catch (PulsarAdminException e) { + log.error("Failed to retrieve broker internal configuration", e); + throw e; + } + } finally { + admin.close(); + } + + // initialize the dlog namespace + // TODO: move this as part of pulsar cluster initialization later + try { + return WorkerUtils.initializeDlogNamespace(internalConf); + } catch (IOException ioe) { + log.error("Failed to initialize dlog namespace with zookeeper {} at metadata service uri {} for storing function packages", + internalConf.getZookeeperServers(), internalConf.getBookkeeperMetadataServiceUri(), ioe); + throw ioe; + } + } + + @Override + public void initInBroker(ServiceConfiguration brokerConfig, + WorkerConfig workerConfig, + ZooKeeperCache globalZkCache, + ConfigurationCacheService configurationCacheService, + InternalConfigurationData internalConf) throws Exception { + + String namespace = workerConfig.getPulsarFunctionsNamespace(); + String[] a = workerConfig.getPulsarFunctionsNamespace().split("/"); + String property = a[0]; + String cluster = workerConfig.getPulsarFunctionsCluster(); + + /* + multiple brokers may be trying to create the property, cluster, and namespace + for function worker service this in parallel. The function worker service uses the namespace + to create topics for internal function + */ + + // create property for function worker service + try { + NamedEntity.checkName(property); + globalZkCache.getZooKeeper().create( + PolicyPath.path(POLICIES, property), + ObjectMapperFactory.getThreadLocal().writeValueAsBytes( + new TenantInfo( + Sets.newHashSet(workerConfig.getSuperUserRoles()), + Sets.newHashSet(cluster))), + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + LOG.info("Created property {} for function worker", property); + } catch (KeeperException.NodeExistsException e) { + LOG.debug("Failed to create already existing property {} for function worker service", cluster, e); + } catch (IllegalArgumentException e) { + LOG.error("Failed to create property with invalid name {} for function worker service", cluster, e); + throw e; + } catch (Exception e) { + LOG.error("Failed to create property {} for function worker", cluster, e); + throw e; + } + + // create cluster for function worker service + try { + NamedEntity.checkName(cluster); + ClusterData clusterData = new ClusterData( + workerConfig.getPulsarWebServiceUrl(), + null /* serviceUrlTls */, + workerConfig.getPulsarServiceUrl(), + null /* brokerServiceUrlTls */); + globalZkCache.getZooKeeper().create( + PolicyPath.path("clusters", cluster), + ObjectMapperFactory.getThreadLocal().writeValueAsBytes(clusterData), + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + LOG.info("Created cluster {} for function worker", cluster); + } catch (KeeperException.NodeExistsException e) { + LOG.debug("Failed to create already existing cluster {} for function worker service", cluster, e); + } catch (IllegalArgumentException e) { + LOG.error("Failed to create cluster with invalid name {} for function worker service", cluster, e); + throw e; + } catch (Exception e) { + LOG.error("Failed to create cluster {} for function worker service", cluster, e); + throw e; + } + + // create namespace for function worker service + try { + Policies policies = new Policies(); + policies.retention_policies = new RetentionPolicies(-1, -1); + policies.replication_clusters = Collections.singleton(workerConfig.getPulsarFunctionsCluster()); + int defaultNumberOfBundles = brokerConfig.getDefaultNumberOfNamespaceBundles(); + policies.bundles = getBundles(defaultNumberOfBundles); + + configurationCacheService.policiesCache().invalidate(PolicyPath.path(POLICIES, namespace)); + ZkUtils.createFullPathOptimistic(globalZkCache.getZooKeeper(), + PolicyPath.path(POLICIES, namespace), + ObjectMapperFactory.getThreadLocal().writeValueAsBytes(policies), + ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + LOG.info("Created namespace {} for function worker service", namespace); + } catch (KeeperException.NodeExistsException e) { + LOG.debug("Failed to create already existing namespace {} for function worker service", namespace); + } catch (Exception e) { + LOG.error("Failed to create namespace {}", namespace, e); + throw e; + } + + URI dlogURI; + try { + // initializing dlog namespace for function worker + dlogURI = WorkerUtils.initializeDlogNamespace(internalConf); + } catch (IOException ioe) { + LOG.error("Failed to initialize dlog namespace with zookeeper {} at at metadata service uri {} for storing function packages", + internalConf.getZookeeperServers(), internalConf.getBookkeeperMetadataServiceUri(), ioe); + throw ioe; + } + + init(workerConfig, dlogURI, false); + + LOG.info("Function worker service setup completed"); + } + + @Override + public void start(AuthenticationService authenticationService, + AuthorizationService authorizationService, + ErrorNotifier errorNotifier) throws Exception { + + workerStatsManager.startupTimeStart(); + log.info("/** Starting worker id={} **/", workerConfig.getWorkerId()); + + try { + log.info("Worker Configs: {}", new ObjectMapper().writerWithDefaultPrettyPrinter() + .writeValueAsString(workerConfig)); + } catch (JsonProcessingException e) { + log.warn("Failed to print worker configs with error {}", e.getMessage(), e); + } + + try { + DistributedLogConfiguration dlogConf = WorkerUtils.getDlogConf(workerConfig); + try { + this.dlogNamespace = NamespaceBuilder.newBuilder() + .conf(dlogConf) + .clientId("function-worker-" + workerConfig.getWorkerId()) + .uri(dlogUri) + .build(); + } catch (Exception e) { + log.error("Failed to initialize dlog namespace {} for storing function packages", + dlogUri, e); + throw new RuntimeException(e); + } + + // create the state storage client for accessing function state + if (workerConfig.getStateStorageServiceUrl() != null) { + StorageClientSettings clientSettings = StorageClientSettings.newBuilder() + .serviceUri(workerConfig.getStateStorageServiceUrl()) + .build(); + this.stateStoreAdminClient = StorageClientBuilder.newBuilder() + .withSettings(clientSettings) + .buildAdmin(); + } + + final String functionWebServiceUrl = StringUtils.isNotBlank(workerConfig.getFunctionWebServiceUrl()) + ? workerConfig.getFunctionWebServiceUrl() + : workerConfig.getWorkerWebAddress(); + + this.brokerAdmin = clientCreator.newPulsarAdmin(workerConfig.getPulsarWebServiceUrl(), workerConfig); + this.functionAdmin = clientCreator.newPulsarAdmin(functionWebServiceUrl, workerConfig); + this.client = clientCreator.newPulsarClient(workerConfig.getPulsarServiceUrl(), workerConfig); + + getBrokerAdmin().topics().createNonPartitionedTopic(workerConfig.getFunctionAssignmentTopic()); + getBrokerAdmin().topics().createNonPartitionedTopic(workerConfig.getClusterCoordinationTopic()); + getBrokerAdmin().topics().createNonPartitionedTopic(workerConfig.getFunctionMetadataTopic()); + //create scheduler manager + this.schedulerManager = new SchedulerManager(workerConfig, client, getBrokerAdmin(), workerStatsManager, errorNotifier); + + //create function meta data manager + this.functionMetaDataManager = new FunctionMetaDataManager( + this.workerConfig, this.schedulerManager, this.client, errorNotifier); + + this.connectorsManager = new ConnectorsManager(workerConfig); + this.functionsManager = new FunctionsManager(workerConfig); + + //create membership manager + String coordinationTopic = workerConfig.getClusterCoordinationTopic(); + if (!getBrokerAdmin().topics().getSubscriptions(coordinationTopic).contains(MembershipManager.COORDINATION_TOPIC_SUBSCRIPTION)) { + getBrokerAdmin().topics() + .createSubscription(coordinationTopic, MembershipManager.COORDINATION_TOPIC_SUBSCRIPTION, + MessageId.earliest); + } + this.membershipManager = new MembershipManager(this, client, getBrokerAdmin()); + + // create function runtime manager + this.functionRuntimeManager = new FunctionRuntimeManager( + workerConfig, + this, + dlogNamespace, + membershipManager, + connectorsManager, + functionsManager, + functionMetaDataManager, + workerStatsManager, + errorNotifier); + + + // initialize function assignment tailer that reads from the assignment topic + this.functionAssignmentTailer = new FunctionAssignmentTailer( + functionRuntimeManager, + client.newReader(), + workerConfig, + errorNotifier); + + // Start worker early in the worker service init process so that functions don't get re-assigned because + // initialize operations of FunctionRuntimeManager and FunctionMetadataManger might take a while + this.leaderService = new LeaderService(this, + client, + functionAssignmentTailer, + schedulerManager, + functionRuntimeManager, + functionMetaDataManager, + errorNotifier); + + log.info("/** Start Leader Service **/"); + leaderService.start(); + + // initialize function metadata manager + log.info("/** Initializing Metdata Manager **/"); + functionMetaDataManager.initialize(); + + // initialize function runtime manager + log.info("/** Initializing Runtime Manager **/"); + + MessageId lastAssignmentMessageId = functionRuntimeManager.initialize(); + + // Setting references to managers in scheduler + schedulerManager.setFunctionMetaDataManager(functionMetaDataManager); + schedulerManager.setFunctionRuntimeManager(functionRuntimeManager); + schedulerManager.setMembershipManager(membershipManager); + schedulerManager.setLeaderService(leaderService); + + this.authenticationService = authenticationService; + + this.authorizationService = authorizationService; + + // Start function assignment tailer + log.info("/** Starting Function Assignment Tailer **/"); + functionAssignmentTailer.startFromMessage(lastAssignmentMessageId); + + // start function metadata manager + log.info("/** Starting Metdata Manager **/"); + functionMetaDataManager.start(); + + // Starting cluster services + this.clusterServiceCoordinator = new ClusterServiceCoordinator( + workerConfig.getWorkerId(), + leaderService); + + clusterServiceCoordinator.addTask("membership-monitor", + workerConfig.getFailureCheckFreqMs(), + () -> { + // computing a new schedule and checking for failures cannot happen concurrently + // both paths of code modify internally cached assignments map in function runtime manager + try { + schedulerManager.getSchedulerLock().lock(); + membershipManager.checkFailures( + functionMetaDataManager, functionRuntimeManager, schedulerManager); + } finally { + schedulerManager.getSchedulerLock().unlock(); + } + }); + + if (workerConfig.getRebalanceCheckFreqSec() > 0) { + clusterServiceCoordinator.addTask("rebalance-periodic-check", + workerConfig.getRebalanceCheckFreqSec() * 1000, + () -> { + try { + schedulerManager.rebalanceIfNotInprogress().get(); + } catch (SchedulerManager.RebalanceInProgressException e) { + log.info("Scheduled for rebalance but rebalance is already in progress. Ignoring."); + } catch (Exception e) { + log.warn("Encountered error when running scheduled rebalance", e); + } + }); + } + + log.info("/** Starting Cluster Service Coordinator **/"); + clusterServiceCoordinator.start(); + + // indicate function worker service is done initializing + this.isInitialized = true; + + log.info("/** Started worker id={} **/", workerConfig.getWorkerId()); + + workerStatsManager.setFunctionRuntimeManager(functionRuntimeManager); + workerStatsManager.setFunctionMetaDataManager(functionMetaDataManager); + workerStatsManager.setLeaderService(leaderService); + workerStatsManager.startupTimeEnd(); + } catch (Throwable t) { + log.error("Error Starting up in worker", t); + throw new RuntimeException(t); + } + } + + @Override + public void stop() { + if (null != functionMetaDataManager) { + try { + functionMetaDataManager.close(); + } catch (Exception e) { + log.warn("Failed to close function metadata manager", e); + } + } + + if (null != functionAssignmentTailer) { + try { + functionAssignmentTailer.close(); + } catch (Exception e) { + log.warn("Failed to close function assignment tailer", e); + } + } + + if (null != functionRuntimeManager) { + try { + functionRuntimeManager.close(); + } catch (Exception e) { + log.warn("Failed to close function runtime manager", e); + } + } + + if (null != clusterServiceCoordinator) { + clusterServiceCoordinator.close(); + } + + if (null != membershipManager) { + membershipManager.close(); + } + + if (null != schedulerManager) { + schedulerManager.close(); + } + + if (null != leaderService) { + try { + leaderService.close(); + } catch (PulsarClientException e) { + log.warn("Failed to close leader service", e); + } + } + + if (null != client) { + try { + client.close(); + } catch (PulsarClientException e) { + log.warn("Failed to close pulsar client", e); + } + } + + if (null != getBrokerAdmin()) { + getBrokerAdmin().close(); + } + + if (null != functionAdmin) { + functionAdmin.close(); + } + + if (null != stateStoreAdminClient) { + stateStoreAdminClient.close(); + } + + if (null != dlogNamespace) { + dlogNamespace.close(); + } + + if (statsUpdater != null) { + statsUpdater.shutdownNow(); + } + } + +} diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/Worker.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/Worker.java index 239fb27b86698..b3ca4dbb136f5 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/Worker.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/Worker.java @@ -19,6 +19,10 @@ package org.apache.pulsar.functions.worker; import io.netty.util.concurrent.DefaultThreadFactory; +import java.io.IOException; +import java.util.Optional; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.pulsar.broker.PulsarServerException; @@ -26,25 +30,13 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.authorization.AuthorizationService; import org.apache.pulsar.broker.cache.ConfigurationCacheService; -import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.common.conf.InternalConfigurationData; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; -import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.functions.worker.rest.WorkerServer; +import org.apache.pulsar.functions.worker.service.WorkerServiceLoader; import org.apache.pulsar.zookeeper.GlobalZooKeeperCache; import org.apache.pulsar.zookeeper.ZooKeeperClientFactory; import org.apache.pulsar.zookeeper.ZookeeperBkClientFactoryImpl; -import javax.ws.rs.core.Response; -import java.io.IOException; -import java.net.URI; -import java.util.HashSet; -import java.util.Optional; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; - @Slf4j public class Worker { @@ -62,15 +54,14 @@ public class Worker { public Worker(WorkerConfig workerConfig) { this.workerConfig = workerConfig; - this.workerService = new WorkerService(workerConfig, true); + this.workerService = WorkerServiceLoader.load(workerConfig); this.errorNotifier = ErrorNotifier.getDefaultImpl(); } protected void start() throws Exception { - URI dlogUri = initialize(workerConfig); - - workerService.start(dlogUri, getAuthenticationService(), getAuthorizationService(), errorNotifier); - server = new WorkerServer(workerService); + workerService.initAsStandalone(workerConfig); + workerService.start(getAuthenticationService(), getAuthorizationService(), errorNotifier); + server = new WorkerServer(workerService, getAuthenticationService()); server.start(); log.info("/** Started worker server on port={} **/", this.workerConfig.getWorkerPort()); @@ -82,84 +73,7 @@ protected void start() throws Exception { } } - private static URI initialize(WorkerConfig workerConfig) - throws InterruptedException, PulsarAdminException, IOException { - // initializing pulsar functions namespace - PulsarAdmin admin = WorkerUtils.getPulsarAdminClient(workerConfig.getPulsarWebServiceUrl(), - workerConfig.getBrokerClientAuthenticationPlugin(), workerConfig.getBrokerClientAuthenticationParameters(), - workerConfig.getTlsTrustCertsFilePath(), workerConfig.isTlsAllowInsecureConnection(), - workerConfig.isTlsEnableHostnameVerification()); - InternalConfigurationData internalConf; - // make sure pulsar broker is up - log.info("Checking if pulsar service at {} is up...", workerConfig.getPulsarWebServiceUrl()); - int maxRetries = workerConfig.getInitialBrokerReconnectMaxRetries(); - int retries = 0; - while (true) { - try { - admin.clusters().getClusters(); - break; - } catch (PulsarAdminException e) { - log.warn("Failed to retrieve clusters from pulsar service", e); - log.warn("Retry to connect to Pulsar service at {}", workerConfig.getPulsarWebServiceUrl()); - if (retries >= maxRetries) { - log.error("Failed to connect to Pulsar service at {} after {} attempts", - workerConfig.getPulsarFunctionsNamespace(), maxRetries); - throw e; - } - retries ++; - Thread.sleep(1000); - } - } - // getting namespace policy - log.info("Initializing Pulsar Functions namespace..."); - try { - try { - admin.namespaces().getPolicies(workerConfig.getPulsarFunctionsNamespace()); - } catch (PulsarAdminException e) { - if (e.getStatusCode() == Response.Status.NOT_FOUND.getStatusCode()) { - // if not found than create - try { - Policies policies = new Policies(); - policies.retention_policies = new RetentionPolicies(-1, -1); - policies.replication_clusters = new HashSet<>(); - policies.replication_clusters.add(workerConfig.getPulsarFunctionsCluster()); - admin.namespaces().createNamespace(workerConfig.getPulsarFunctionsNamespace(), - policies); - } catch (PulsarAdminException e1) { - // prevent race condition with other workers starting up - if (e1.getStatusCode() != Response.Status.CONFLICT.getStatusCode()) { - log.error("Failed to create namespace {} for pulsar functions", workerConfig - .getPulsarFunctionsNamespace(), e1); - throw e1; - } - } - } else { - log.error("Failed to get retention policy for pulsar function namespace {}", - workerConfig.getPulsarFunctionsNamespace(), e); - throw e; - } - } - try { - internalConf = admin.brokers().getInternalConfigurationData(); - } catch (PulsarAdminException e) { - log.error("Failed to retrieve broker internal configuration", e); - throw e; - } - } finally { - admin.close(); - } - - // initialize the dlog namespace - // TODO: move this as part of pulsar cluster initialization later - try { - return WorkerUtils.initializeDlogNamespace(internalConf); - } catch (IOException ioe) { - log.error("Failed to initialize dlog namespace with zookeeper {} at metadata service uri {} for storing function packages", - internalConf.getZookeeperServers(), internalConf.getBookkeeperMetadataServiceUri(), ioe); - throw ioe; - } - } private AuthorizationService getAuthorizationService() throws PulsarServerException { diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerService.java index bf4c24cf4a911..d95bbb32ef37c 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerService.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerService.java @@ -18,358 +18,122 @@ */ package org.apache.pulsar.functions.worker; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; -import io.netty.util.concurrent.DefaultThreadFactory; -import java.net.URI; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import lombok.Getter; -import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.clients.StorageClientBuilder; -import org.apache.bookkeeper.clients.admin.StorageAdminClient; -import org.apache.bookkeeper.clients.config.StorageClientSettings; -import org.apache.commons.lang3.StringUtils; -import org.apache.distributedlog.DistributedLogConfiguration; -import org.apache.distributedlog.api.namespace.Namespace; -import org.apache.distributedlog.api.namespace.NamespaceBuilder; +import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.authorization.AuthorizationService; -import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.broker.cache.ConfigurationCacheService; +import org.apache.pulsar.common.conf.InternalConfigurationData; +import org.apache.pulsar.common.util.SimpleTextOutputStream; +import org.apache.pulsar.functions.worker.service.api.Functions; +import org.apache.pulsar.functions.worker.service.api.FunctionsV2; +import org.apache.pulsar.functions.worker.service.api.Sinks; +import org.apache.pulsar.functions.worker.service.api.Sources; +import org.apache.pulsar.functions.worker.service.api.Workers; +import org.apache.pulsar.zookeeper.ZooKeeperCache; /** - * A service component contains everything to run a worker except rest server. + * API service provides the ability to manage functions. */ -@Slf4j -@Getter -public class WorkerService { - - private final WorkerConfig workerConfig; - - private PulsarClient client; - private FunctionRuntimeManager functionRuntimeManager; - private FunctionMetaDataManager functionMetaDataManager; - private ClusterServiceCoordinator clusterServiceCoordinator; - // dlog namespace for storing function jars in bookkeeper - private Namespace dlogNamespace; - // storage client for accessing state storage for functions - private StorageAdminClient stateStoreAdminClient; - private MembershipManager membershipManager; - private SchedulerManager schedulerManager; - private volatile boolean isInitialized = false; - private final ScheduledExecutorService statsUpdater; - private AuthenticationService authenticationService; - private AuthorizationService authorizationService; - private ConnectorsManager connectorsManager; - private FunctionsManager functionsManager; - private PulsarAdmin brokerAdmin; - private PulsarAdmin functionAdmin; - private final MetricsGenerator metricsGenerator; - @VisibleForTesting - private URI dlogUri; - private LeaderService leaderService; - private FunctionAssignmentTailer functionAssignmentTailer; - private final WorkerStatsManager workerStatsManager; - - public WorkerService(WorkerConfig workerConfig, boolean runAsStandalone) { - this.workerConfig = workerConfig; - this.statsUpdater = Executors - .newSingleThreadScheduledExecutor(new DefaultThreadFactory("worker-stats-updater")); - this.metricsGenerator = new MetricsGenerator(this.statsUpdater, workerConfig); - this.workerStatsManager = new WorkerStatsManager(workerConfig, runAsStandalone); - } - - public WorkerService(WorkerConfig workerConfig) { - this(workerConfig, false); - } - - public void start(URI dlogUri, - AuthenticationService authenticationService, - AuthorizationService authorizationService, - ErrorNotifier errorNotifier) throws InterruptedException { - - workerStatsManager.startupTimeStart(); - log.info("/** Starting worker id={} **/", workerConfig.getWorkerId()); - - try { - log.info("Worker Configs: {}", new ObjectMapper().writerWithDefaultPrettyPrinter() - .writeValueAsString(workerConfig)); - } catch (JsonProcessingException e) { - log.warn("Failed to print worker configs with error {}", e.getMessage(), e); - } - - try { - // create the dlog namespace for storing function packages - dlogUri = dlogUri; - DistributedLogConfiguration dlogConf = WorkerUtils.getDlogConf(workerConfig); - try { - this.dlogNamespace = NamespaceBuilder.newBuilder() - .conf(dlogConf) - .clientId("function-worker-" + workerConfig.getWorkerId()) - .uri(dlogUri) - .build(); - } catch (Exception e) { - log.error("Failed to initialize dlog namespace {} for storing function packages", - dlogUri, e); - throw new RuntimeException(e); - } - - // create the state storage client for accessing function state - if (workerConfig.getStateStorageServiceUrl() != null) { - StorageClientSettings clientSettings = StorageClientSettings.newBuilder() - .serviceUri(workerConfig.getStateStorageServiceUrl()) - .build(); - this.stateStoreAdminClient = StorageClientBuilder.newBuilder() - .withSettings(clientSettings) - .buildAdmin(); - } - - final String functionWebServiceUrl = StringUtils.isNotBlank(workerConfig.getFunctionWebServiceUrl()) - ? workerConfig.getFunctionWebServiceUrl() - : workerConfig.getWorkerWebAddress(); - - // using isBrokerClientAuthenticationEnabled instead of isAuthenticationEnabled in function-worker - if (workerConfig.isBrokerClientAuthenticationEnabled()) { - // for compatible, if user do not define brokerClientTrustCertsFilePath, we will use tlsTrustCertsFilePath, - // otherwise we will use brokerClientTrustCertsFilePath - final String pulsarClientTlsTrustCertsFilePath; - if (StringUtils.isNotBlank(workerConfig.getBrokerClientTrustCertsFilePath())) { - pulsarClientTlsTrustCertsFilePath = workerConfig.getBrokerClientTrustCertsFilePath(); - } else { - pulsarClientTlsTrustCertsFilePath = workerConfig.getTlsTrustCertsFilePath(); - } - this.brokerAdmin = WorkerUtils.getPulsarAdminClient(workerConfig.getPulsarWebServiceUrl(), - workerConfig.getBrokerClientAuthenticationPlugin(), workerConfig.getBrokerClientAuthenticationParameters(), - pulsarClientTlsTrustCertsFilePath, workerConfig.isTlsAllowInsecureConnection(), - workerConfig.isTlsEnableHostnameVerification()); - - this.functionAdmin = WorkerUtils.getPulsarAdminClient(functionWebServiceUrl, - workerConfig.getBrokerClientAuthenticationPlugin(), workerConfig.getBrokerClientAuthenticationParameters(), - workerConfig.getTlsTrustCertsFilePath(), workerConfig.isTlsAllowInsecureConnection(), - workerConfig.isTlsEnableHostnameVerification()); - - this.client = WorkerUtils.getPulsarClient(workerConfig.getPulsarServiceUrl(), - workerConfig.getBrokerClientAuthenticationPlugin(), - workerConfig.getBrokerClientAuthenticationParameters(), - workerConfig.isUseTls(), pulsarClientTlsTrustCertsFilePath, - workerConfig.isTlsAllowInsecureConnection(), workerConfig.isTlsEnableHostnameVerification()); - } else { - this.brokerAdmin = WorkerUtils.getPulsarAdminClient(workerConfig.getPulsarWebServiceUrl()); - - this.functionAdmin = WorkerUtils.getPulsarAdminClient(functionWebServiceUrl); - - this.client = WorkerUtils.getPulsarClient(workerConfig.getPulsarServiceUrl()); - } - - brokerAdmin.topics().createNonPartitionedTopic(workerConfig.getFunctionAssignmentTopic()); - brokerAdmin.topics().createNonPartitionedTopic(workerConfig.getClusterCoordinationTopic()); - brokerAdmin.topics().createNonPartitionedTopic(workerConfig.getFunctionMetadataTopic()); - //create scheduler manager - this.schedulerManager = new SchedulerManager(workerConfig, client, brokerAdmin, workerStatsManager, errorNotifier); - - //create function meta data manager - this.functionMetaDataManager = new FunctionMetaDataManager( - this.workerConfig, this.schedulerManager, this.client, errorNotifier); - - this.connectorsManager = new ConnectorsManager(workerConfig); - this.functionsManager = new FunctionsManager(workerConfig); - - //create membership manager - String coordinationTopic = workerConfig.getClusterCoordinationTopic(); - if (!brokerAdmin.topics().getSubscriptions(coordinationTopic).contains(MembershipManager.COORDINATION_TOPIC_SUBSCRIPTION)) { - brokerAdmin.topics() - .createSubscription(coordinationTopic, MembershipManager.COORDINATION_TOPIC_SUBSCRIPTION, - MessageId.earliest); - } - this.membershipManager = new MembershipManager(this, client, brokerAdmin); - - // create function runtime manager - this.functionRuntimeManager = new FunctionRuntimeManager( - workerConfig, - this, - dlogNamespace, - membershipManager, - connectorsManager, - functionsManager, - functionMetaDataManager, - workerStatsManager, - errorNotifier); - - - // initialize function assignment tailer that reads from the assignment topic - this.functionAssignmentTailer = new FunctionAssignmentTailer( - functionRuntimeManager, - client.newReader(), - workerConfig, - errorNotifier); - - // Start worker early in the worker service init process so that functions don't get re-assigned because - // initialize operations of FunctionRuntimeManager and FunctionMetadataManger might take a while - this.leaderService = new LeaderService(this, - client, - functionAssignmentTailer, - schedulerManager, - functionRuntimeManager, - functionMetaDataManager, - errorNotifier); - - log.info("/** Start Leader Service **/"); - leaderService.start(); - - // initialize function metadata manager - log.info("/** Initializing Metdata Manager **/"); - functionMetaDataManager.initialize(); - - // initialize function runtime manager - log.info("/** Initializing Runtime Manager **/"); - - MessageId lastAssignmentMessageId = functionRuntimeManager.initialize(); - - // Setting references to managers in scheduler - schedulerManager.setFunctionMetaDataManager(functionMetaDataManager); - schedulerManager.setFunctionRuntimeManager(functionRuntimeManager); - schedulerManager.setMembershipManager(membershipManager); - schedulerManager.setLeaderService(leaderService); - - this.authenticationService = authenticationService; - - this.authorizationService = authorizationService; - - // Start function assignment tailer - log.info("/** Starting Function Assignment Tailer **/"); - functionAssignmentTailer.startFromMessage(lastAssignmentMessageId); - - // start function metadata manager - log.info("/** Starting Metdata Manager **/"); - functionMetaDataManager.start(); - - // Starting cluster services - this.clusterServiceCoordinator = new ClusterServiceCoordinator( - workerConfig.getWorkerId(), - leaderService); - - clusterServiceCoordinator.addTask("membership-monitor", - workerConfig.getFailureCheckFreqMs(), - () -> { - // computing a new schedule and checking for failures cannot happen concurrently - // both paths of code modify internally cached assignments map in function runtime manager - try { - schedulerManager.getSchedulerLock().lock(); - membershipManager.checkFailures( - functionMetaDataManager, functionRuntimeManager, schedulerManager); - } finally { - schedulerManager.getSchedulerLock().unlock(); - } - }); - - if (workerConfig.getRebalanceCheckFreqSec() > 0) { - clusterServiceCoordinator.addTask("rebalance-periodic-check", - workerConfig.getRebalanceCheckFreqSec() * 1000, - () -> { - try { - schedulerManager.rebalanceIfNotInprogress().get(); - } catch (SchedulerManager.RebalanceInProgressException e) { - log.info("Scheduled for rebalance but rebalance is already in progress. Ignoring."); - } catch (Exception e) { - log.warn("Encountered error when running scheduled rebalance", e); - } - }); - } - - log.info("/** Starting Cluster Service Coordinator **/"); - clusterServiceCoordinator.start(); - - // indicate function worker service is done initializing - this.isInitialized = true; - - log.info("/** Started worker id={} **/", workerConfig.getWorkerId()); - - workerStatsManager.setFunctionRuntimeManager(functionRuntimeManager); - workerStatsManager.setFunctionMetaDataManager(functionMetaDataManager); - workerStatsManager.setLeaderService(leaderService); - workerStatsManager.startupTimeEnd(); - } catch (Throwable t) { - log.error("Error Starting up in worker", t); - throw new RuntimeException(t); - } - } - - public void stop() { - if (null != functionMetaDataManager) { - try { - functionMetaDataManager.close(); - } catch (Exception e) { - log.warn("Failed to close function metadata manager", e); - } - } - - if (null != functionAssignmentTailer) { - try { - functionAssignmentTailer.close(); - } catch (Exception e) { - log.warn("Failed to close function assignment tailer", e); - } - } - - if (null != functionRuntimeManager) { - try { - functionRuntimeManager.close(); - } catch (Exception e) { - log.warn("Failed to close function runtime manager", e); - } - } - - if (null != clusterServiceCoordinator) { - clusterServiceCoordinator.close(); - } - - if (null != membershipManager) { - membershipManager.close(); - } - - if (null != schedulerManager) { - schedulerManager.close(); - } - - if (null != leaderService) { - try { - leaderService.close(); - } catch (PulsarClientException e) { - log.warn("Failed to close leader service", e); - } - } - - if (null != client) { - try { - client.close(); - } catch (PulsarClientException e) { - log.warn("Failed to close pulsar client", e); - } - } - - if (null != brokerAdmin) { - brokerAdmin.close(); - } - - if (null != functionAdmin) { - functionAdmin.close(); - } - - if (null != stateStoreAdminClient) { - stateStoreAdminClient.close(); - } - - if (null != dlogNamespace) { - dlogNamespace.close(); - } - - if (statsUpdater != null) { - statsUpdater.shutdownNow(); - } - } +public interface WorkerService { + + /** + * Return the worker config. + * + * @return worker config + */ + WorkerConfig getWorkerConfig(); + + /** + * Initialize the worker API service using the provided config. + * + * @param workerConfig the worker config + * @throws Exception when fail to initialize the worker API service. + */ + void initAsStandalone(WorkerConfig workerConfig) + throws Exception; + + /** + * Initialize the worker service in broker. + * + * @param brokerConfig broker config + * @param workerConfig worker config + * @param globalZkCache global zookeeper cache + * @param configurationCacheService configuration cache + * @param internalConf pulsar internal configuration data + * @throws Exception when failed to initialize the worker service in broker. + */ + void initInBroker(ServiceConfiguration brokerConfig, + WorkerConfig workerConfig, + ZooKeeperCache globalZkCache, + ConfigurationCacheService configurationCacheService, + InternalConfigurationData internalConf) throws Exception; + + /** + * Start the worker API service. + * + * @param authenticationService the authentication service. + * @param authorizationService the authorization service. + * @param errorNotifier error notifier. + * @throws Exception when fail to start the worker API service. + */ + void start(AuthenticationService authenticationService, + AuthorizationService authorizationService, + ErrorNotifier errorNotifier) throws Exception; + + /** + * Stop the worker API service. + */ + void stop(); + + /** + * Check if the worker service is initialized or not. + * + * @return true if the worker service is initialized otherwise false. + */ + boolean isInitialized(); + + /** + * Get the functions service. + * + * @return the functions service. + */ + Functions getFunctions(); + + /** + * Get the functions service (v2). + * + *

This is a legacy API service for supporting v2. + * + * @return the functions service (v2). + */ + FunctionsV2 getFunctionsV2(); + + /** + * Get the sinks service. + * + * @return the sinks service. + */ + Sinks getSinks(); + + /** + * Get the sources service. + * + * @return the sources service. + */ + Sources getSources(); + + /** + * Get the worker service. + * + * @return the worker service. + */ + Workers getWorkers(); + + /** + * Generate functions stats. + * + * @param out output stream + */ + void generateFunctionsStats(SimpleTextOutputStream out); } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerUtils.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerUtils.java index bbf101a5358f4..93612826508e5 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerUtils.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerUtils.java @@ -155,6 +155,11 @@ public static DistributedLogConfiguration getDlogConf(WorkerConfig workerConfig) return conf; } + public static URI newDlogNamespaceURI(InternalConfigurationData internalConf) { + String zookeeperServers = internalConf.getZookeeperServers(); + return URI.create(String.format("distributedlog://%s/pulsar/functions", zookeeperServers)); + } + public static URI initializeDlogNamespace(InternalConfigurationData internalConf) throws IOException { String zookeeperServers = internalConf.getZookeeperServers(); String ledgersRootPath; @@ -170,8 +175,8 @@ public static URI initializeDlogNamespace(InternalConfigurationData internalConf } BKDLConfig dlConfig = new BKDLConfig(ledgersStoreServers, ledgersRootPath); DLMetadata dlMetadata = DLMetadata.create(dlConfig); - URI dlogUri = URI.create(String.format("distributedlog://%s/pulsar/functions", zookeeperServers)); + URI dlogUri = newDlogNamespaceURI(internalConf); try { dlMetadata.create(dlogUri); } catch (ZKException e) { diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java index 51ddafc7ab747..60d871c136f83 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java @@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import io.prometheus.client.jetty.JettyStatisticsCollector; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.web.AuthenticationFilter; import org.apache.pulsar.broker.web.RateLimitingFilter; import org.apache.pulsar.broker.web.WebExecutorThreadPool; @@ -60,6 +61,7 @@ public class WorkerServer { private final WorkerConfig workerConfig; private final WorkerService workerService; + private final AuthenticationService authenticationService; private static final String MATCH_ALL = "/*"; private static final int MAX_CONCURRENT_REQUESTS = 1024; private final WebExecutorThreadPool webServerExecutor; @@ -77,9 +79,10 @@ private static String getErrorMessage(Server server, int port, Exception ex) { return ex.getMessage(); } - public WorkerServer(WorkerService workerService) { + public WorkerServer(WorkerService workerService, AuthenticationService authenticationService) { this.workerConfig = workerService.getWorkerConfig(); this.workerService = workerService; + this.authenticationService = authenticationService; this.webServerExecutor = new WebExecutorThreadPool(this.workerConfig.getNumHttpServerThreads(), "function-web"); init(); } @@ -98,14 +101,16 @@ private void init() { connectors.add(httpConnector); List handlers = new ArrayList<>(4); - handlers.add( - newServletContextHandler("/admin", new ResourceConfig(Resources.getApiV2Resources()), workerService)); - handlers.add( - newServletContextHandler("/admin/v2", new ResourceConfig(Resources.getApiV2Resources()), workerService)); - handlers.add( - newServletContextHandler("/admin/v3", new ResourceConfig(Resources.getApiV3Resources()), workerService)); + handlers.add(newServletContextHandler("/admin", + new ResourceConfig(Resources.getApiV2Resources()), workerService, authenticationService)); + handlers.add(newServletContextHandler("/admin/v2", + new ResourceConfig(Resources.getApiV2Resources()), workerService, authenticationService)); + handlers.add(newServletContextHandler("/admin/v3", + new ResourceConfig(Resources.getApiV3Resources()), workerService, authenticationService)); // don't require auth for metrics or config routes - handlers.add(newServletContextHandler("/", new ResourceConfig(Resources.getRootResources()), workerService, workerConfig.isAuthenticateMetricsEndpoint())); + handlers.add(newServletContextHandler("/", + new ResourceConfig(Resources.getRootResources()), workerService, + workerConfig.isAuthenticateMetricsEndpoint(), authenticationService)); RequestLogHandler requestLogHandler = new RequestLogHandler(); Slf4jRequestLog requestLog = new Slf4jRequestLog(); @@ -153,11 +158,18 @@ private void init() { server.setConnectors(connectors.toArray(new ServerConnector[connectors.size()])); } - public static ServletContextHandler newServletContextHandler(String contextPath, ResourceConfig config, WorkerService workerService) { - return newServletContextHandler(contextPath, config, workerService, true); + public static ServletContextHandler newServletContextHandler(String contextPath, + ResourceConfig config, + WorkerService workerService, + AuthenticationService authenticationService) { + return newServletContextHandler(contextPath, config, workerService, true, authenticationService); } - public static ServletContextHandler newServletContextHandler(String contextPath, ResourceConfig config, WorkerService workerService, boolean requireAuthentication) { + public static ServletContextHandler newServletContextHandler(String contextPath, + ResourceConfig config, + WorkerService workerService, + boolean requireAuthentication, + AuthenticationService authenticationService) { final ServletContextHandler contextHandler = new ServletContextHandler(ServletContextHandler.NO_SESSIONS); @@ -170,7 +182,7 @@ public static ServletContextHandler newServletContextHandler(String contextPath, new ServletHolder(new ServletContainer(config)); contextHandler.addServlet(apiServlet, "/*"); if (workerService.getWorkerConfig().isAuthenticationEnabled() && requireAuthentication) { - FilterHolder filter = new FilterHolder(new AuthenticationFilter(workerService.getAuthenticationService())); + FilterHolder filter = new FilterHolder(new AuthenticationFilter(authenticationService)); contextHandler.addFilter(filter, MATCH_ALL, EnumSet.allOf(DispatcherType.class)); } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java index 29766e6b99962..fdae616ded773 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java @@ -79,8 +79,10 @@ import org.apache.pulsar.functions.worker.FunctionMetaDataManager; import org.apache.pulsar.functions.worker.FunctionRuntimeInfo; import org.apache.pulsar.functions.worker.FunctionRuntimeManager; +import org.apache.pulsar.functions.worker.PulsarWorkerService; import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; +import org.apache.pulsar.functions.worker.service.api.Component; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import java.io.File; @@ -106,13 +108,13 @@ import javax.ws.rs.core.UriBuilder; @Slf4j -public abstract class ComponentImpl { +public abstract class ComponentImpl implements Component { private final AtomicReference storageClient = new AtomicReference<>(); - protected final Supplier workerServiceSupplier; + protected final Supplier workerServiceSupplier; protected final Function.FunctionDetails.ComponentType componentType; - public ComponentImpl(Supplier workerServiceSupplier, Function.FunctionDetails.ComponentType componentType) { + public ComponentImpl(Supplier workerServiceSupplier, Function.FunctionDetails.ComponentType componentType) { this.workerServiceSupplier = workerServiceSupplier; this.componentType = componentType; } @@ -250,7 +252,8 @@ public S getComponentStatus(final String tenant, } } - protected WorkerService worker() { + @Override + public PulsarWorkerService worker() { try { return checkNotNull(workerServiceSupplier.get()); } catch (Throwable t) { @@ -337,6 +340,7 @@ PackageLocationMetaData.Builder getFunctionPackageLocation(final FunctionMetaDat return packageLocationMetaDataBuilder; } + @Override public void deregisterFunction(final String tenant, final String namespace, final String componentName, @@ -410,6 +414,7 @@ public void deregisterFunction(final String tenant, } } + @Override public FunctionConfig getFunctionInfo(final String tenant, final String namespace, final String componentName, @@ -453,6 +458,7 @@ public FunctionConfig getFunctionInfo(final String tenant, return config; } + @Override public void stopFunctionInstance(final String tenant, final String namespace, final String componentName, @@ -463,6 +469,7 @@ public void stopFunctionInstance(final String tenant, changeFunctionInstanceStatus(tenant, namespace, componentName, instanceId, false, uri, clientRole, clientAuthenticationDataHttps); } + @Override public void startFunctionInstance(final String tenant, final String namespace, final String componentName, @@ -528,6 +535,7 @@ public void changeFunctionInstanceStatus(final String tenant, tenant, namespace, componentName, instanceId)); } + @Override public void restartFunctionInstance(final String tenant, final String namespace, final String componentName, @@ -582,6 +590,7 @@ public void restartFunctionInstance(final String tenant, } } + @Override public void stopFunctionInstances(final String tenant, final String namespace, final String componentName, @@ -590,6 +599,7 @@ public void stopFunctionInstances(final String tenant, changeFunctionStatusAllInstances(tenant, namespace, componentName, false, clientRole, clientAuthenticationDataHttps); } + @Override public void startFunctionInstances(final String tenant, final String namespace, final String componentName, @@ -756,6 +766,7 @@ public FunctionStats getFunctionStats(final String tenant, return functionStats; } + @Override public FunctionStats.FunctionInstanceStats.FunctionInstanceStatsData getFunctionsInstanceStats(final String tenant, final String namespace, final String componentName, @@ -819,6 +830,7 @@ public FunctionStats.FunctionInstanceStats.FunctionInstanceStatsData getFunction return functionInstanceStatsData; } + @Override public List listFunctions(final String tenant, final String namespace, final String clientRole, @@ -866,6 +878,7 @@ void updateRequest(FunctionMetaData existingFunctionMetaData, final FunctionMeta updatedVersionMetaData, false, "Update Failed"); } + @Override public List getListOfConnectors() { if (!isWorkerServiceAvailable()) { throwUnavailableException(); @@ -874,6 +887,7 @@ public List getListOfConnectors() { return this.worker().getConnectorsManager().getConnectors(); } + @Override public void reloadConnectors(String clientRole) { if (!isWorkerServiceAvailable()) { throwUnavailableException(); @@ -891,6 +905,7 @@ public void reloadConnectors(String clientRole) { } } + @Override public String triggerFunction(final String tenant, final String namespace, final String functionName, @@ -1015,6 +1030,7 @@ public String triggerFunction(final String tenant, } } + @Override public FunctionState getFunctionState(final String tenant, final String namespace, final String functionName, @@ -1096,6 +1112,7 @@ public FunctionState getFunctionState(final String tenant, return value; } + @Override public void putFunctionState(final String tenant, final String namespace, final String functionName, @@ -1174,6 +1191,7 @@ public void putFunctionState(final String tenant, } } + @Override public void uploadFunction(final InputStream uploadedInputStream, final String path, String clientRole) { if (!isWorkerServiceAvailable()) { @@ -1204,6 +1222,7 @@ public void uploadFunction(final InputStream uploadedInputStream, final String p } } + @Override public StreamingOutput downloadFunction(String tenant, String namespace, String componentName, String clientRole, AuthenticationDataHttps clientAuthenticationDataHttps) { if (!isWorkerServiceAvailable()) { @@ -1251,6 +1270,7 @@ public StreamingOutput downloadFunction(String tenant, String namespace, String return streamingOutput; } + @Override public StreamingOutput downloadFunction(final String path, String clientRole, AuthenticationDataHttps clientAuthenticationDataHttps) { if (!isWorkerServiceAvailable()) { diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java index a753c74db87b8..08f2b69460f83 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java @@ -20,6 +20,7 @@ import com.google.protobuf.ByteString; import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.authentication.AuthenticationDataHttps; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -38,9 +39,9 @@ import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.FunctionConfigUtils; import org.apache.pulsar.functions.worker.FunctionMetaDataManager; -import org.apache.pulsar.functions.worker.WorkerService; +import org.apache.pulsar.functions.worker.PulsarWorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; -import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.functions.worker.service.api.Functions; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import javax.ws.rs.WebApplicationException; @@ -64,12 +65,13 @@ import static org.apache.pulsar.functions.worker.rest.RestUtils.throwUnavailableException; @Slf4j -public class FunctionsImpl extends ComponentImpl { +public class FunctionsImpl extends ComponentImpl implements Functions { - public FunctionsImpl(Supplier workerServiceSupplier) { + public FunctionsImpl(Supplier workerServiceSupplier) { super(workerServiceSupplier, Function.FunctionDetails.ComponentType.FUNCTION); } + @Override public void registerFunction(final String tenant, final String namespace, final String functionName, @@ -236,6 +238,7 @@ public void registerFunction(final String tenant, } } + @Override public void updateFunction(final String tenant, final String namespace, final String functionName, @@ -591,6 +594,7 @@ private ExceptionInformation getExceptionInformation(InstanceCommunication.Funct * @param instanceId the function instance id * @return the function status */ + @Override public FunctionStatus.FunctionInstanceStatus.FunctionInstanceStatusData getFunctionInstanceStatus(final String tenant, final String namespace, final String componentName, @@ -624,6 +628,7 @@ public FunctionStatus.FunctionInstanceStatus.FunctionInstanceStatusData getFunct * @return a list of function statuses * @throws PulsarAdminException */ + @Override public FunctionStatus getFunctionStatus(final String tenant, final String namespace, final String componentName, @@ -647,6 +652,7 @@ public FunctionStatus getFunctionStatus(final String tenant, return functionStatus; } + @Override public void updateFunctionOnWorkerLeader(final String tenant, final String namespace, final String functionName, diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplV2.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplV2.java index 86118cea3b208..9ed6afb96943c 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplV2.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplV2.java @@ -30,7 +30,9 @@ import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.FunctionConfigUtils; import org.apache.pulsar.functions.worker.FunctionMetaDataManager; -import org.apache.pulsar.functions.worker.WorkerService; +import org.apache.pulsar.functions.worker.PulsarWorkerService; +import org.apache.pulsar.functions.worker.service.api.Functions; +import org.apache.pulsar.functions.worker.service.api.FunctionsV2; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import javax.ws.rs.core.Response; @@ -43,10 +45,11 @@ import java.util.stream.Collectors; @Slf4j -public class FunctionsImplV2 { +public class FunctionsImplV2 implements FunctionsV2 { - private FunctionsImpl delegate; - public FunctionsImplV2(Supplier workerServiceSupplier) { + private final Functions delegate; + + public FunctionsImplV2(Supplier workerServiceSupplier) { this.delegate = new FunctionsImpl(workerServiceSupplier); } @@ -55,6 +58,7 @@ public FunctionsImplV2(FunctionsImpl delegate) { this.delegate = delegate; } + @Override public Response getFunctionInfo(final String tenant, final String namespace, final String functionName, String clientRole) throws IOException { @@ -69,6 +73,7 @@ public Response getFunctionInfo(final String tenant, final String namespace, fin return Response.status(Response.Status.OK).entity(functionDetailsJson).build(); } + @Override public Response getFunctionInstanceStatus(final String tenant, final String namespace, final String functionName, final String instanceId, URI uri, String clientRole) throws IOException { @@ -79,6 +84,7 @@ public Response getFunctionInstanceStatus(final String tenant, final String name return Response.status(Response.Status.OK).entity(jsonResponse).build(); } + @Override public Response getFunctionStatusV2(String tenant, String namespace, String functionName, URI requestUri, String clientRole) throws IOException { FunctionStatus functionStatus = delegate.getFunctionStatus(tenant, namespace, functionName, requestUri, clientRole, null); @@ -90,6 +96,7 @@ public Response getFunctionStatusV2(String tenant, String namespace, String func return Response.status(Response.Status.OK).entity(jsonResponse).build(); } + @Override public Response registerFunction(String tenant, String namespace, String functionName, InputStream uploadedInputStream, FormDataContentDisposition fileDetail, String functionPkgUrl, String functionDetailsJson, String clientRole) { @@ -107,6 +114,7 @@ public Response registerFunction(String tenant, String namespace, String functio return Response.ok().build(); } + @Override public Response updateFunction(String tenant, String namespace, String functionName, InputStream uploadedInputStream, FormDataContentDisposition fileDetail, String functionPkgUrl, String functionDetailsJson, String clientRole) { @@ -124,22 +132,26 @@ public Response updateFunction(String tenant, String namespace, String functionN return Response.ok().build(); } + @Override public Response deregisterFunction(String tenant, String namespace, String functionName, String clientAppId) { delegate.deregisterFunction(tenant, namespace, functionName, clientAppId, null); return Response.ok().build(); } + @Override public Response listFunctions(String tenant, String namespace, String clientRole) { Collection functionStateList = delegate.listFunctions( tenant, namespace, clientRole, null); return Response.status(Response.Status.OK).entity(new Gson().toJson(functionStateList.toArray())).build(); } + @Override public Response triggerFunction(String tenant, String namespace, String functionName, String triggerValue, InputStream triggerStream, String topic, String clientRole) { String result = delegate.triggerFunction(tenant, namespace, functionName, triggerValue, triggerStream, topic, clientRole, null); return Response.status(Response.Status.OK).entity(result).build(); } + @Override public Response getFunctionState(String tenant, String namespace, String functionName, String key, String clientRole) { FunctionState functionState = delegate.getFunctionState( tenant, namespace, functionName, key, clientRole, null); @@ -155,37 +167,44 @@ public Response getFunctionState(String tenant, String namespace, String functio .build(); } + @Override public Response restartFunctionInstance(String tenant, String namespace, String functionName, String instanceId, URI uri, String clientRole) { delegate.restartFunctionInstance(tenant, namespace, functionName, instanceId, uri, clientRole, null); return Response.ok().build(); } + @Override public Response restartFunctionInstances(String tenant, String namespace, String functionName, String clientRole) { delegate.restartFunctionInstances(tenant, namespace, functionName, clientRole, null); return Response.ok().build(); } + @Override public Response stopFunctionInstance(String tenant, String namespace, String functionName, String instanceId, URI uri, String clientRole) { delegate.stopFunctionInstance(tenant, namespace, functionName, instanceId, uri, clientRole ,null); return Response.ok().build(); } + @Override public Response stopFunctionInstances(String tenant, String namespace, String functionName, String clientRole) { delegate.stopFunctionInstances(tenant, namespace, functionName, clientRole, null); return Response.ok().build(); } + @Override public Response uploadFunction(InputStream uploadedInputStream, String path, String clientRole) { delegate.uploadFunction(uploadedInputStream, path, clientRole); return Response.ok().build(); } + @Override public Response downloadFunction(String path, String clientRole) { return Response.status(Response.Status.OK).entity(delegate.downloadFunction(path, clientRole, null)).build(); } + @Override public List getListOfConnectors() { return delegate.getListOfConnectors(); } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsMetricsResource.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsMetricsResource.java index e9cfbfa77aa0c..1d341acbf6692 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsMetricsResource.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsMetricsResource.java @@ -23,7 +23,6 @@ import io.prometheus.client.CollectorRegistry; import io.prometheus.client.exporter.common.TextFormat; import org.apache.pulsar.common.util.SimpleTextOutputStream; -import org.apache.pulsar.functions.worker.FunctionsStatsGenerator; import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.rest.FunctionApiResource; @@ -55,7 +54,7 @@ public Response getMetrics() throws IOException { try { SimpleTextOutputStream stream = new SimpleTextOutputStream(buf); - FunctionsStatsGenerator.generate(workerService,"default", stream); + workerService.generateFunctionsStats(stream); byte[] payload = buf.array(); int arrayOffset = buf.arrayOffset(); int readableBytes = buf.readableBytes(); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java index d1b9895866afb..8f99da9d0f9a9 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java @@ -40,8 +40,10 @@ import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.SinkConfigUtils; import org.apache.pulsar.functions.worker.FunctionMetaDataManager; +import org.apache.pulsar.functions.worker.PulsarWorkerService; import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; +import org.apache.pulsar.functions.worker.service.api.Sinks; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import javax.ws.rs.WebApplicationException; @@ -61,12 +63,13 @@ import static org.apache.pulsar.functions.worker.rest.RestUtils.throwUnavailableException; @Slf4j -public class SinksImpl extends ComponentImpl { +public class SinksImpl extends ComponentImpl implements Sinks { - public SinksImpl(Supplier workerServiceSupplier) { + public SinksImpl(Supplier workerServiceSupplier) { super(workerServiceSupplier, Function.FunctionDetails.ComponentType.SINK); } + @Override public void registerSink(final String tenant, final String namespace, final String sinkName, @@ -233,6 +236,7 @@ public void registerSink(final String tenant, } } + @Override public void updateSink(final String tenant, final String namespace, final String sinkName, @@ -587,6 +591,7 @@ private ExceptionInformation getExceptionInformation(InstanceCommunication.Funct return exceptionInformation; } + @Override public SinkStatus.SinkInstanceStatus.SinkInstanceStatusData getSinkInstanceStatus(final String tenant, final String namespace, final String sinkName, @@ -612,6 +617,7 @@ public SinkStatus.SinkInstanceStatus.SinkInstanceStatusData getSinkInstanceStatu return sinkInstanceStatusData; } + @Override public SinkStatus getSinkStatus(final String tenant, final String namespace, final String componentName, @@ -635,6 +641,7 @@ public SinkStatus getSinkStatus(final String tenant, return sinkStatus; } + @Override public SinkConfig getSinkInfo(final String tenant, final String namespace, final String componentName) { @@ -665,6 +672,7 @@ public SinkConfig getSinkInfo(final String tenant, return config; } + @Override public List getSinkList() { List connectorDefinitions = getListOfConnectors(); List retval = new ArrayList<>(); @@ -676,6 +684,7 @@ public List getSinkList() { return retval; } + @Override public List getSinkConfigDefinition(String name) { if (!isWorkerServiceAvailable()) { throwUnavailableException(); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java index f312370c350e6..a4d3ff024f4a3 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java @@ -40,8 +40,9 @@ import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.SourceConfigUtils; import org.apache.pulsar.functions.worker.FunctionMetaDataManager; -import org.apache.pulsar.functions.worker.WorkerService; +import org.apache.pulsar.functions.worker.PulsarWorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; +import org.apache.pulsar.functions.worker.service.api.Sources; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import javax.ws.rs.WebApplicationException; @@ -61,12 +62,13 @@ import static org.apache.pulsar.functions.worker.rest.RestUtils.throwUnavailableException; @Slf4j -public class SourcesImpl extends ComponentImpl { +public class SourcesImpl extends ComponentImpl implements Sources { - public SourcesImpl(Supplier workerServiceSupplier) { + public SourcesImpl(Supplier workerServiceSupplier) { super(workerServiceSupplier, Function.FunctionDetails.ComponentType.SOURCE); } + @Override public void registerSource(final String tenant, final String namespace, final String sourceName, @@ -233,6 +235,7 @@ public void registerSource(final String tenant, } } + @Override public void updateSource(final String tenant, final String namespace, final String sourceName, @@ -587,6 +590,7 @@ public SourceStatus emptyStatus(final int parallelism) { } } + @Override public SourceStatus getSourceStatus(final String tenant, final String namespace, final String componentName, @@ -608,6 +612,7 @@ public SourceStatus getSourceStatus(final String tenant, return sourceStatus; } + @Override public SourceStatus.SourceInstanceStatus.SourceInstanceStatusData getSourceInstanceStatus(final String tenant, final String namespace, final String sourceName, @@ -631,6 +636,7 @@ public SourceStatus.SourceInstanceStatus.SourceInstanceStatusData getSourceInsta return sourceInstanceStatusData; } + @Override public SourceConfig getSourceInfo(final String tenant, final String namespace, final String componentName) { @@ -661,6 +667,7 @@ public SourceConfig getSourceInfo(final String tenant, return config; } + @Override public List getSourceList() { List connectorDefinitions = getListOfConnectors(); List retval = new ArrayList<>(); @@ -672,6 +679,7 @@ public List getSourceList() { return retval; } + @Override public List getSourceConfigDefinition(String name) { if (!isWorkerServiceAvailable()) { throwUnavailableException(); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/WorkerImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/WorkerImpl.java index e2fefab699c33..a67d179234a92 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/WorkerImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/WorkerImpl.java @@ -29,6 +29,7 @@ import org.apache.pulsar.functions.worker.FunctionRuntimeInfo; import org.apache.pulsar.functions.worker.FunctionRuntimeManager; import org.apache.pulsar.functions.worker.MembershipManager; +import org.apache.pulsar.functions.worker.PulsarWorkerService; import org.apache.pulsar.functions.worker.SchedulerManager; import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; @@ -44,24 +45,22 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.Future; import java.util.function.Supplier; +import org.apache.pulsar.functions.worker.service.api.Workers; import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.pulsar.functions.worker.rest.RestUtils.throwUnavailableException; @Slf4j -public class WorkerImpl { +public class WorkerImpl implements Workers { - private final Supplier workerServiceSupplier; - private Future currentRebalanceFuture; + private final Supplier workerServiceSupplier; - - public WorkerImpl(Supplier workerServiceSupplier) { + public WorkerImpl(Supplier workerServiceSupplier) { this.workerServiceSupplier = workerServiceSupplier; } - private WorkerService worker() { + private PulsarWorkerService worker() { try { return checkNotNull(workerServiceSupplier.get()); } catch (Throwable t) { @@ -81,6 +80,7 @@ private boolean isWorkerServiceAvailable() { return true; } + @Override public List getCluster(String clientRole) { if (!isWorkerServiceAvailable()) { throwUnavailableException(); @@ -94,6 +94,7 @@ public List getCluster(String clientRole) { return workers; } + @Override public WorkerInfo getClusterLeader(String clientRole) { if (!isWorkerServiceAvailable()) { throwUnavailableException(); @@ -114,6 +115,7 @@ public WorkerInfo getClusterLeader(String clientRole) { return leader; } + @Override public Map> getAssignments(String clientRole) { if (!isWorkerServiceAvailable()) { throwUnavailableException(); @@ -137,8 +139,9 @@ private boolean isSuperUser(final String clientRole) { return clientRole != null && worker().getWorkerConfig().getSuperUserRoles().contains(clientRole); } + @Override public List getWorkerMetrics(final String clientRole) { - if (!isWorkerServiceAvailable()) { + if (!isWorkerServiceAvailable() || worker().getMetricsGenerator() == null) { throwUnavailableException(); } @@ -149,6 +152,7 @@ public List getWorkerMetrics(final Strin return worker().getMetricsGenerator().generate(); } + @Override public List getFunctionsMetrics(String clientRole) throws IOException { if (!isWorkerServiceAvailable()) { throwUnavailableException(); @@ -159,8 +163,7 @@ public List getFunctionsMetrics(String clientRole) throw new RestException(Status.UNAUTHORIZED, "client is not authorize to perform operation"); } - WorkerService workerService = worker(); - Map functionRuntimes = workerService.getFunctionRuntimeManager() + Map functionRuntimes = worker().getFunctionRuntimeManager() .getFunctionRuntimeInfos(); List metricsList = new ArrayList<>(functionRuntimes.size()); @@ -169,7 +172,7 @@ public List getFunctionsMetrics(String clientRole) String fullyQualifiedInstanceName = entry.getKey(); FunctionRuntimeInfo functionRuntimeInfo = entry.getValue(); - if (workerService.getFunctionRuntimeManager().getRuntimeFactory().externallyManaged()) { + if (worker().getFunctionRuntimeManager().getRuntimeFactory().externallyManaged()) { Function.FunctionDetails functionDetails = functionRuntimeInfo.getFunctionInstance().getFunctionMetaData().getFunctionDetails(); int parallelism = functionDetails.getParallelism(); for (int i = 0; i < parallelism; ++i) { @@ -195,6 +198,7 @@ public List getFunctionsMetrics(String clientRole) return metricsList; } + @Override public List getListOfConnectors(String clientRole) { if (!isWorkerServiceAvailable()) { throwUnavailableException(); @@ -230,6 +234,7 @@ public void rebalance(final URI uri, final String clientRole) { } } + @Override public Boolean isLeaderReady(final String clientRole) { if (!isWorkerServiceAvailable()) { throwUnavailableException(); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionsApiV2Resource.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionsApiV2Resource.java index 43e7c5cc0582b..a53c2ac16d937 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionsApiV2Resource.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionsApiV2Resource.java @@ -26,8 +26,9 @@ import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.InstanceCommunication; +import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.rest.FunctionApiResource; -import org.apache.pulsar.functions.worker.rest.api.FunctionsImplV2; +import org.apache.pulsar.functions.worker.service.api.FunctionsV2; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import org.glassfish.jersey.media.multipart.FormDataParam; @@ -49,10 +50,8 @@ @Path("/functions") public class FunctionsApiV2Resource extends FunctionApiResource { - protected final FunctionsImplV2 functions; - - public FunctionsApiV2Resource() { - this.functions = new FunctionsImplV2(this); + FunctionsV2 functions() { + return get().getFunctionsV2(); } @POST @@ -73,7 +72,7 @@ public Response registerFunction(final @PathParam("tenant") String tenant, final @FormDataParam("url") String functionPkgUrl, final @FormDataParam("functionDetails") String functionDetailsJson) { - return functions.registerFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, + return functions().registerFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, functionPkgUrl, functionDetailsJson, clientAppId()); } @@ -94,7 +93,7 @@ public Response updateFunction(final @PathParam("tenant") String tenant, final @FormDataParam("url") String functionPkgUrl, final @FormDataParam("functionDetails") String functionDetailsJson) { - return functions.updateFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, + return functions().updateFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, functionPkgUrl, functionDetailsJson, clientAppId()); } @@ -112,7 +111,7 @@ public Response updateFunction(final @PathParam("tenant") String tenant, public Response deregisterFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) { - return functions.deregisterFunction(tenant, namespace, functionName, clientAppId()); + return functions().deregisterFunction(tenant, namespace, functionName, clientAppId()); } @GET @@ -131,7 +130,7 @@ public Response getFunctionInfo(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) throws IOException { - return functions.getFunctionInfo( + return functions().getFunctionInfo( tenant, namespace, functionName, clientAppId()); } @@ -152,7 +151,7 @@ public Response getFunctionInstanceStatus(final @PathParam("tenant") String tena final @PathParam("functionName") String functionName, final @PathParam("instanceId") String instanceId) throws IOException { - return functions.getFunctionInstanceStatus(tenant, namespace, functionName, instanceId, uri.getRequestUri(), + return functions().getFunctionInstanceStatus(tenant, namespace, functionName, instanceId, uri.getRequestUri(), clientAppId()); } @@ -170,7 +169,7 @@ public Response getFunctionInstanceStatus(final @PathParam("tenant") String tena public Response getFunctionStatus(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) throws IOException { - return functions.getFunctionStatusV2(tenant, namespace, functionName, uri.getRequestUri(), clientAppId()); + return functions().getFunctionStatusV2(tenant, namespace, functionName, uri.getRequestUri(), clientAppId()); } @GET @@ -186,7 +185,7 @@ public Response getFunctionStatus(final @PathParam("tenant") String tenant, @Path("/{tenant}/{namespace}") public Response listFunctions(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace) { - return functions.listFunctions(tenant, namespace, clientAppId()); + return functions().listFunctions(tenant, namespace, clientAppId()); } @POST @@ -208,7 +207,7 @@ public Response triggerFunction(final @PathParam("tenant") String tenant, final @FormDataParam("data") String triggerValue, final @FormDataParam("dataStream") InputStream triggerStream, final @FormDataParam("topic") String topic) { - return functions.triggerFunction(tenant, namespace, functionName, triggerValue, triggerStream, topic, + return functions().triggerFunction(tenant, namespace, functionName, triggerValue, triggerStream, topic, clientAppId()); } @@ -228,7 +227,7 @@ public Response getFunctionState(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName, final @PathParam("key") String key) { - return functions.getFunctionState(tenant, namespace, functionName, key, clientAppId()); + return functions().getFunctionState(tenant, namespace, functionName, key, clientAppId()); } @POST @@ -243,7 +242,7 @@ public Response getFunctionState(final @PathParam("tenant") String tenant, public Response restartFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName, final @PathParam("instanceId") String instanceId) { - return functions.restartFunctionInstance(tenant, namespace, functionName, instanceId, uri.getRequestUri(), + return functions().restartFunctionInstance(tenant, namespace, functionName, instanceId, uri.getRequestUri(), clientAppId()); } @@ -256,7 +255,7 @@ public Response restartFunction(final @PathParam("tenant") String tenant, @Consumes(MediaType.APPLICATION_JSON) public Response restartFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) { - return functions.restartFunctionInstances(tenant, namespace, functionName, clientAppId()); + return functions().restartFunctionInstances(tenant, namespace, functionName, clientAppId()); } @POST @@ -269,7 +268,7 @@ public Response restartFunction(final @PathParam("tenant") String tenant, public Response stopFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName, final @PathParam("instanceId") String instanceId) { - return functions.stopFunctionInstance(tenant, namespace, functionName, instanceId, uri.getRequestUri(), + return functions().stopFunctionInstance(tenant, namespace, functionName, instanceId, uri.getRequestUri(), clientAppId()); } @@ -282,7 +281,7 @@ public Response stopFunction(final @PathParam("tenant") String tenant, @Consumes(MediaType.APPLICATION_JSON) public Response stopFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) { - return functions.stopFunctionInstances(tenant, namespace, functionName, clientAppId()); + return functions().stopFunctionInstances(tenant, namespace, functionName, clientAppId()); } @POST @@ -294,7 +293,7 @@ public Response stopFunction(final @PathParam("tenant") String tenant, @Consumes(MediaType.MULTIPART_FORM_DATA) public Response uploadFunction(final @FormDataParam("data") InputStream uploadedInputStream, final @FormDataParam("path") String path) { - return functions.uploadFunction(uploadedInputStream, path, clientAppId()); + return functions().uploadFunction(uploadedInputStream, path, clientAppId()); } @GET @@ -304,7 +303,7 @@ public Response uploadFunction(final @FormDataParam("data") InputStream uploaded ) @Path("/download") public Response downloadFunction(final @QueryParam("path") String path) { - return functions.downloadFunction(path, clientAppId()); + return functions().downloadFunction(path, clientAppId()); } @GET @@ -323,6 +322,6 @@ public Response downloadFunction(final @QueryParam("path") String path) { */ @Deprecated public List getConnectorsList() throws IOException { - return functions.getListOfConnectors(); + return functions().getListOfConnectors(); } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v2/WorkerApiV2Resource.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v2/WorkerApiV2Resource.java index 0746f6b28648e..26d96b29fba29 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v2/WorkerApiV2Resource.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v2/WorkerApiV2Resource.java @@ -27,7 +27,6 @@ import org.apache.pulsar.common.functions.WorkerInfo; import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.functions.worker.WorkerService; -import org.apache.pulsar.functions.worker.rest.api.WorkerImpl; import java.io.IOException; import java.util.Collection; @@ -45,6 +44,7 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.UriInfo; +import org.apache.pulsar.functions.worker.service.api.Workers; @Slf4j @Path("/worker") @@ -55,7 +55,6 @@ public class WorkerApiV2Resource implements Supplier { public static final String ATTRIBUTE_WORKER_SERVICE = "worker"; - protected final WorkerImpl worker; private WorkerService workerService; @Context protected ServletContext servletContext; @@ -64,10 +63,6 @@ public class WorkerApiV2Resource implements Supplier { @Context protected UriInfo uri; - public WorkerApiV2Resource() { - this.worker = new WorkerImpl(this); - } - @Override public synchronized WorkerService get() { if (this.workerService == null) { @@ -76,6 +71,10 @@ public synchronized WorkerService get() { return this.workerService; } + Workers workers() { + return get().getWorkers(); + } + public String clientAppId() { return httpRequest != null ? (String) httpRequest.getAttribute(AuthenticationFilter.AuthenticatedRoleAttributeName) @@ -95,7 +94,7 @@ public String clientAppId() { @Path("/cluster") @Produces(MediaType.APPLICATION_JSON) public List getCluster() { - return worker.getCluster(clientAppId()); + return workers().getCluster(clientAppId()); } @GET @@ -110,7 +109,7 @@ public List getCluster() { @Path("/cluster/leader") @Produces(MediaType.APPLICATION_JSON) public WorkerInfo getClusterLeader() { - return worker.getClusterLeader(clientAppId()); + return workers().getClusterLeader(clientAppId()); } @GET @@ -125,7 +124,7 @@ public WorkerInfo getClusterLeader() { @Path("/assignments") @Produces(MediaType.APPLICATION_JSON) public Map> getAssignments() { - return worker.getAssignments(clientAppId()); + return workers().getAssignments(clientAppId()); } @GET @@ -140,7 +139,7 @@ public Map> getAssignments() { }) @Path("/connectors") public List getConnectorsList() throws IOException { - return worker.getListOfConnectors(clientAppId()); + return workers().getListOfConnectors(clientAppId()); } @PUT @@ -154,7 +153,7 @@ public List getConnectorsList() throws IOException { }) @Path("/rebalance") public void rebalance() { - worker.rebalance(uri.getRequestUri(), clientAppId()); + workers().rebalance(uri.getRequestUri(), clientAppId()); } @GET @@ -167,6 +166,6 @@ public void rebalance() { }) @Path("/cluster/leader/ready") public Boolean isLeaderReady() { - return worker.isLeaderReady(clientAppId()); + return workers().isLeaderReady(clientAppId()); } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v2/WorkerStatsApiV2Resource.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v2/WorkerStatsApiV2Resource.java index bd404d6ffee56..19c5d8ef3ec11 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v2/WorkerStatsApiV2Resource.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v2/WorkerStatsApiV2Resource.java @@ -26,7 +26,6 @@ import org.apache.pulsar.broker.web.AuthenticationFilter; import org.apache.pulsar.common.policies.data.WorkerFunctionInstanceStats; import org.apache.pulsar.functions.worker.WorkerService; -import org.apache.pulsar.functions.worker.rest.api.WorkerImpl; import javax.servlet.ServletContext; import javax.servlet.http.HttpServletRequest; @@ -39,6 +38,7 @@ import java.io.IOException; import java.util.List; import java.util.function.Supplier; +import org.apache.pulsar.functions.worker.service.api.Workers; @Slf4j @Path("/worker-stats") @@ -49,17 +49,12 @@ public class WorkerStatsApiV2Resource implements Supplier { public static final String ATTRIBUTE_WORKERSTATS_SERVICE = "worker-stats"; - protected final WorkerImpl worker; private WorkerService workerService; @Context protected ServletContext servletContext; @Context protected HttpServletRequest httpRequest; - public WorkerStatsApiV2Resource() { - this.worker = new WorkerImpl(this); - } - @Override public synchronized WorkerService get() { if (this.workerService == null) { @@ -68,6 +63,10 @@ public synchronized WorkerService get() { return this.workerService; } + Workers workers() { + return get().getWorkers(); + } + public String clientAppId() { return httpRequest != null ? (String) httpRequest.getAttribute(AuthenticationFilter.AuthenticatedRoleAttributeName) @@ -87,7 +86,7 @@ public String clientAppId() { }) @Produces(MediaType.APPLICATION_JSON) public List getMetrics() throws Exception { - return worker.getWorkerMetrics(clientAppId()); + return workers().getWorkerMetrics(clientAppId()); } @GET @@ -103,6 +102,6 @@ public List getMetrics() throws Exceptio }) @Produces(MediaType.APPLICATION_JSON) public List getStats() throws IOException { - return worker.getFunctionsMetrics(clientAppId()); + return workers().getFunctionsMetrics(clientAppId()); } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionsApiV3Resource.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionsApiV3Resource.java index 07eee4aa35154..e38a734242f2a 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionsApiV3Resource.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionsApiV3Resource.java @@ -29,8 +29,9 @@ import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.policies.data.FunctionStats; import org.apache.pulsar.common.policies.data.FunctionStatus; +import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.rest.FunctionApiResource; -import org.apache.pulsar.functions.worker.rest.api.FunctionsImpl; +import org.apache.pulsar.functions.worker.service.api.Functions; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import org.glassfish.jersey.media.multipart.FormDataParam; @@ -53,10 +54,8 @@ @Path("/functions") public class FunctionsApiV3Resource extends FunctionApiResource { - protected final FunctionsImpl functions; - - public FunctionsApiV3Resource() { - this.functions = new FunctionsImpl(this); + Functions functions() { + return get().getFunctions(); } @POST @@ -70,7 +69,7 @@ public void registerFunction(final @PathParam("tenant") String tenant, final @FormDataParam("url") String functionPkgUrl, final @FormDataParam("functionConfig") FunctionConfig functionConfig) { - functions.registerFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, + functions().registerFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, functionPkgUrl, functionConfig, clientAppId(), clientAuthData()); } @@ -87,7 +86,7 @@ public void updateFunction(final @PathParam("tenant") String tenant, final @FormDataParam("functionConfig") FunctionConfig functionConfig, final @FormDataParam("updateOptions") UpdateOptions updateOptions) { - functions.updateFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, + functions().updateFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, functionPkgUrl, functionConfig, clientAppId(), clientAuthData(), updateOptions); } @@ -97,7 +96,7 @@ public void updateFunction(final @PathParam("tenant") String tenant, public void deregisterFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) { - functions.deregisterFunction(tenant, namespace, functionName, clientAppId(), clientAuthData()); + functions().deregisterFunction(tenant, namespace, functionName, clientAppId(), clientAuthData()); } @GET @@ -105,7 +104,7 @@ public void deregisterFunction(final @PathParam("tenant") String tenant, public FunctionConfig getFunctionInfo(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) { - return functions.getFunctionInfo(tenant, namespace, functionName, clientAppId(), clientAuthData()); + return functions().getFunctionInfo(tenant, namespace, functionName, clientAppId(), clientAuthData()); } @GET @@ -113,7 +112,7 @@ public FunctionConfig getFunctionInfo(final @PathParam("tenant") String tenant, @Path("/{tenant}/{namespace}") public List listSources(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace) { - return functions.listFunctions(tenant, namespace, clientAppId(), clientAuthData()); + return functions().listFunctions(tenant, namespace, clientAppId(), clientAuthData()); } @GET @@ -134,7 +133,7 @@ public FunctionStatus.FunctionInstanceStatus.FunctionInstanceStatusData getFunct final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName, final @PathParam("instanceId") String instanceId) throws IOException { - return functions.getFunctionInstanceStatus( + return functions().getFunctionInstanceStatus( tenant, namespace, functionName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -155,7 +154,7 @@ public FunctionStatus getFunctionStatus( final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) throws IOException { - return functions.getFunctionStatus( + return functions().getFunctionStatus( tenant, namespace, functionName, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -175,7 +174,7 @@ public FunctionStatus getFunctionStatus( public FunctionStats getFunctionStats(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) throws IOException { - return functions.getFunctionStats(tenant, namespace, functionName, uri.getRequestUri(), clientAppId(), clientAuthData()); + return functions().getFunctionStats(tenant, namespace, functionName, uri.getRequestUri(), clientAppId(), clientAuthData()); } @GET @@ -196,7 +195,7 @@ public FunctionStats.FunctionInstanceStats.FunctionInstanceStatsData getFunction final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName, final @PathParam("instanceId") String instanceId) throws IOException { - return functions.getFunctionsInstanceStats( + return functions().getFunctionsInstanceStats( tenant, namespace, functionName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -209,7 +208,7 @@ public String triggerFunction(final @PathParam("tenant") String tenant, final @FormDataParam("data") String input, final @FormDataParam("dataStream") InputStream uploadedInputStream, final @FormDataParam("topic") String topic) { - return functions.triggerFunction(tenant, namespace, functionName, input, uploadedInputStream, topic, clientAppId(), clientAuthData()); + return functions().triggerFunction(tenant, namespace, functionName, input, uploadedInputStream, topic, clientAppId(), clientAuthData()); } @POST @@ -226,7 +225,7 @@ public void restartFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName, final @PathParam("instanceId") String instanceId) { - functions.restartFunctionInstance(tenant, namespace, functionName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); + functions().restartFunctionInstance(tenant, namespace, functionName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); } @POST @@ -241,7 +240,7 @@ public void restartFunction(final @PathParam("tenant") String tenant, public void restartFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) { - functions.restartFunctionInstances(tenant, namespace, functionName, clientAppId(), clientAuthData()); + functions().restartFunctionInstances(tenant, namespace, functionName, clientAppId(), clientAuthData()); } @POST @@ -257,7 +256,7 @@ public void stopFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName, final @PathParam("instanceId") String instanceId) { - functions.stopFunctionInstance(tenant, namespace, functionName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); + functions().stopFunctionInstance(tenant, namespace, functionName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); } @POST @@ -272,7 +271,7 @@ public void stopFunction(final @PathParam("tenant") String tenant, public void stopFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) { - functions.stopFunctionInstances(tenant, namespace, functionName, clientAppId(), clientAuthData()); + functions().stopFunctionInstances(tenant, namespace, functionName, clientAppId(), clientAuthData()); } @POST @@ -288,7 +287,7 @@ public void startFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName, final @PathParam("instanceId") String instanceId) { - functions.startFunctionInstance(tenant, namespace, functionName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); + functions().startFunctionInstance(tenant, namespace, functionName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); } @POST @@ -303,7 +302,7 @@ public void startFunction(final @PathParam("tenant") String tenant, public void startFunction(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName) { - functions.startFunctionInstances(tenant, namespace, functionName, clientAppId(), clientAuthData()); + functions().startFunctionInstances(tenant, namespace, functionName, clientAppId(), clientAuthData()); } @POST @@ -311,13 +310,13 @@ public void startFunction(final @PathParam("tenant") String tenant, @Consumes(MediaType.MULTIPART_FORM_DATA) public void uploadFunction(final @FormDataParam("data") InputStream uploadedInputStream, final @FormDataParam("path") String path) { - functions.uploadFunction(uploadedInputStream, path, clientAppId()); + functions().uploadFunction(uploadedInputStream, path, clientAppId()); } @GET @Path("/download") public StreamingOutput downloadFunction(final @QueryParam("path") String path) { - return functions.downloadFunction(path, clientAppId(), clientAuthData()); + return functions().downloadFunction(path, clientAppId(), clientAuthData()); } @GET @@ -334,7 +333,7 @@ public StreamingOutput downloadFunction( @ApiParam(value = "The name of functions") final @PathParam("functionName") String functionName) { - return functions.downloadFunction(tenant, namespace, functionName, clientAppId(), clientAuthData()); + return functions().downloadFunction(tenant, namespace, functionName, clientAppId(), clientAuthData()); } @GET @@ -344,7 +343,7 @@ public StreamingOutput downloadFunction( */ @Deprecated public List getConnectorsList() throws IOException { - return functions.getListOfConnectors(); + return functions().getListOfConnectors(); } @GET @@ -353,7 +352,7 @@ public FunctionState getFunctionState(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("functionName") String functionName, final @PathParam("key") String key) throws IOException { - return functions.getFunctionState(tenant, namespace, functionName, key, clientAppId(), clientAuthData()); + return functions().getFunctionState(tenant, namespace, functionName, key, clientAppId(), clientAuthData()); } @POST @@ -364,7 +363,7 @@ public void putFunctionState(final @PathParam("tenant") String tenant, final @PathParam("functionName") String functionName, final @PathParam("key") String key, final @FormDataParam("state") FunctionState stateJson) throws IOException { - functions.putFunctionState(tenant, namespace, functionName, key, stateJson, clientAppId(), clientAuthData()); + functions().putFunctionState(tenant, namespace, functionName, key, stateJson, clientAppId(), clientAuthData()); } @PUT @@ -384,7 +383,7 @@ public void updateFunctionOnWorkerLeader(final @PathParam("tenant") String tenan final @FormDataParam("functionMetaData") InputStream uploadedInputStream, final @FormDataParam("delete") boolean delete) { - functions.updateFunctionOnWorkerLeader(tenant, namespace, functionName, uploadedInputStream, + functions().updateFunctionOnWorkerLeader(tenant, namespace, functionName, uploadedInputStream, delete, uri.getRequestUri(), clientAppId()); } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v3/SinksApiV3Resource.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v3/SinksApiV3Resource.java index 9186594d4c0cc..af56d247966aa 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v3/SinksApiV3Resource.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v3/SinksApiV3Resource.java @@ -18,15 +18,20 @@ */ package org.apache.pulsar.functions.worker.rest.api.v3; -import io.swagger.annotations.*; +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiParam; +import io.swagger.annotations.ApiResponse; +import io.swagger.annotations.ApiResponses; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.functions.UpdateOptions; import org.apache.pulsar.common.io.ConfigFieldDefinition; import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.common.policies.data.SinkStatus; +import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.rest.FunctionApiResource; -import org.apache.pulsar.functions.worker.rest.api.SinksImpl; +import org.apache.pulsar.functions.worker.service.api.Sinks; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import org.glassfish.jersey.media.multipart.FormDataParam; @@ -50,10 +55,8 @@ @Path("/sinks") public class SinksApiV3Resource extends FunctionApiResource { - protected final SinksImpl sink; - - public SinksApiV3Resource() { - this.sink = new SinksImpl(this); + Sinks sinks() { + return get().getSinks(); } @POST @@ -67,7 +70,7 @@ public void registerSink(final @PathParam("tenant") String tenant, final @FormDataParam("url") String functionPkgUrl, final @FormDataParam("sinkConfig") SinkConfig sinkConfig) { - sink.registerSink(tenant, namespace, sinkName, uploadedInputStream, fileDetail, + sinks().registerSink(tenant, namespace, sinkName, uploadedInputStream, fileDetail, functionPkgUrl, sinkConfig, clientAppId(), clientAuthData()); } @@ -83,7 +86,7 @@ public void updateSink(final @PathParam("tenant") String tenant, final @FormDataParam("sinkConfig") SinkConfig sinkConfig, final @FormDataParam("updateOptions") UpdateOptions updateOptions) { - sink.updateSink(tenant, namespace, sinkName, uploadedInputStream, fileDetail, + sinks().updateSink(tenant, namespace, sinkName, uploadedInputStream, fileDetail, functionPkgUrl, sinkConfig, clientAppId(), clientAuthData(), updateOptions); } @@ -92,7 +95,7 @@ public void updateSink(final @PathParam("tenant") String tenant, public void deregisterSink(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sinkName") String sinkName) { - sink.deregisterFunction(tenant, namespace, sinkName, clientAppId(), clientAuthData()); + sinks().deregisterFunction(tenant, namespace, sinkName, clientAppId(), clientAuthData()); } @GET @@ -101,7 +104,7 @@ public SinkConfig getSinkInfo(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sinkName") String sinkName) throws IOException { - return sink.getSinkInfo(tenant, namespace, sinkName); + return sinks().getSinkInfo(tenant, namespace, sinkName); } @GET @@ -122,7 +125,7 @@ public SinkStatus.SinkInstanceStatus.SinkInstanceStatusData getSinkInstanceStatu final @PathParam("namespace") String namespace, final @PathParam("sinkName") String sinkName, final @PathParam("instanceId") String instanceId) throws IOException { - return sink.getSinkInstanceStatus(tenant, namespace, sinkName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); + return sinks().getSinkInstanceStatus(tenant, namespace, sinkName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @GET @@ -141,14 +144,14 @@ public SinkStatus.SinkInstanceStatus.SinkInstanceStatusData getSinkInstanceStatu public SinkStatus getSinkStatus(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sinkName") String sinkName) throws IOException { - return sink.getSinkStatus(tenant, namespace, sinkName, uri.getRequestUri(), clientAppId(), clientAuthData()); + return sinks().getSinkStatus(tenant, namespace, sinkName, uri.getRequestUri(), clientAppId(), clientAuthData()); } @GET @Path("/{tenant}/{namespace}") public List listSink(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace) { - return sink.listFunctions(tenant, namespace, clientAppId(), clientAuthData()); + return sinks().listFunctions(tenant, namespace, clientAppId(), clientAuthData()); } @POST @@ -164,7 +167,7 @@ public void restartSink(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sinkName") String sinkName, final @PathParam("instanceId") String instanceId) { - sink.restartFunctionInstance(tenant, namespace, sinkName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); + sinks().restartFunctionInstance(tenant, namespace, sinkName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); } @POST @@ -176,7 +179,7 @@ public void restartSink(final @PathParam("tenant") String tenant, public void restartSink(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sinkName") String sinkName) { - sink.restartFunctionInstances(tenant, namespace, sinkName, clientAppId(), clientAuthData()); + sinks().restartFunctionInstances(tenant, namespace, sinkName, clientAppId(), clientAuthData()); } @POST @@ -190,7 +193,7 @@ public void stopSink(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sinkName") String sinkName, final @PathParam("instanceId") String instanceId) { - sink.stopFunctionInstance(tenant, namespace, sinkName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); + sinks().stopFunctionInstance(tenant, namespace, sinkName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); } @POST @@ -203,7 +206,7 @@ public void stopSink(final @PathParam("tenant") String tenant, public void stopSink(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sinkName") String sinkName) { - sink.stopFunctionInstances(tenant, namespace, sinkName, clientAppId(), clientAuthData()); + sinks().stopFunctionInstances(tenant, namespace, sinkName, clientAppId(), clientAuthData()); } @POST @@ -217,7 +220,7 @@ public void startSink(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sinkName") String sinkName, final @PathParam("instanceId") String instanceId) { - sink.startFunctionInstance(tenant, namespace, sinkName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); + sinks().startFunctionInstance(tenant, namespace, sinkName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); } @POST @@ -230,13 +233,13 @@ public void startSink(final @PathParam("tenant") String tenant, public void startSink(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sinkName") String sinkName) { - sink.startFunctionInstances(tenant, namespace, sinkName, clientAppId(), clientAuthData()); + sinks().startFunctionInstances(tenant, namespace, sinkName, clientAppId(), clientAuthData()); } @GET @Path("/builtinsinks") public List getSinkList() { - return sink.getSinkList(); + return sinks().getSinkList(); } @GET @@ -256,7 +259,7 @@ public List getSinkList() { public List getSinkConfigDefinition( @ApiParam(value = "The name of the builtin sink") final @PathParam("name") String name) throws IOException { - return sink.getSinkConfigDefinition(name); + return sinks().getSinkConfigDefinition(name); } @POST @@ -271,6 +274,6 @@ public List getSinkConfigDefinition( }) @Path("/reloadBuiltInSinks") public void reloadSinks() { - sink.reloadConnectors(clientAppId()); + sinks().reloadConnectors(clientAppId()); } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v3/SourcesApiV3Resource.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v3/SourcesApiV3Resource.java index 4eece39d97ae0..db8eb2460c3e3 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v3/SourcesApiV3Resource.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/v3/SourcesApiV3Resource.java @@ -18,15 +18,20 @@ */ package org.apache.pulsar.functions.worker.rest.api.v3; -import io.swagger.annotations.*; +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiParam; +import io.swagger.annotations.ApiResponse; +import io.swagger.annotations.ApiResponses; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.functions.UpdateOptions; import org.apache.pulsar.common.io.ConfigFieldDefinition; import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.io.SourceConfig; import org.apache.pulsar.common.policies.data.SourceStatus; +import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.rest.FunctionApiResource; -import org.apache.pulsar.functions.worker.rest.api.SourcesImpl; +import org.apache.pulsar.functions.worker.service.api.Sources; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import org.glassfish.jersey.media.multipart.FormDataParam; @@ -50,10 +55,8 @@ @Path("/sources") public class SourcesApiV3Resource extends FunctionApiResource { - protected final SourcesImpl source; - - public SourcesApiV3Resource() { - this.source = new SourcesImpl(this); + Sources sources() { + return get().getSources(); } @POST @@ -67,7 +70,7 @@ public void registerSource(final @PathParam("tenant") String tenant, final @FormDataParam("url") String functionPkgUrl, final @FormDataParam("sourceConfig") SourceConfig sourceConfig) { - source.registerSource(tenant, namespace, sourceName, uploadedInputStream, fileDetail, + sources().registerSource(tenant, namespace, sourceName, uploadedInputStream, fileDetail, functionPkgUrl, sourceConfig, clientAppId(), clientAuthData()); } @@ -84,7 +87,7 @@ public void updateSource(final @PathParam("tenant") String tenant, final @FormDataParam("sourceConfig") SourceConfig sourceConfig, final @FormDataParam("updateOptions") UpdateOptions updateOptions) { - source.updateSource(tenant, namespace, sourceName, uploadedInputStream, fileDetail, + sources().updateSource(tenant, namespace, sourceName, uploadedInputStream, fileDetail, functionPkgUrl, sourceConfig, clientAppId(), clientAuthData(), updateOptions); } @@ -94,7 +97,7 @@ public void updateSource(final @PathParam("tenant") String tenant, public void deregisterSource(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sourceName") String sourceName) { - source.deregisterFunction(tenant, namespace, sourceName, clientAppId(), clientAuthData()); + sources().deregisterFunction(tenant, namespace, sourceName, clientAppId(), clientAuthData()); } @GET @@ -104,7 +107,7 @@ public SourceConfig getSourceInfo(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sourceName") String sourceName) throws IOException { - return source.getSourceInfo(tenant, namespace, sourceName); + return sources().getSourceInfo(tenant, namespace, sourceName); } @GET @@ -125,7 +128,7 @@ public SourceStatus.SourceInstanceStatus.SourceInstanceStatusData getSourceInsta final @PathParam("namespace") String namespace, final @PathParam("sourceName") String sourceName, final @PathParam("instanceId") String instanceId) throws IOException { - return source.getSourceInstanceStatus( + return sources().getSourceInstanceStatus( tenant, namespace, sourceName, instanceId, uri.getRequestUri(), clientAppId(), clientAuthData()); } @@ -145,7 +148,7 @@ public SourceStatus.SourceInstanceStatus.SourceInstanceStatusData getSourceInsta public SourceStatus getSourceStatus(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sourceName") String sourceName) throws IOException { - return source.getSourceStatus(tenant, namespace, sourceName, uri.getRequestUri(), clientAppId(), clientAuthData()); + return sources().getSourceStatus(tenant, namespace, sourceName, uri.getRequestUri(), clientAppId(), clientAuthData()); } @GET @@ -153,7 +156,7 @@ public SourceStatus getSourceStatus(final @PathParam("tenant") String tenant, @Path("/{tenant}/{namespace}") public List listSources(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace) { - return source.listFunctions(tenant, namespace, clientAppId(), clientAuthData()); + return sources().listFunctions(tenant, namespace, clientAppId(), clientAuthData()); } @POST @@ -169,7 +172,7 @@ public void restartSource(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sourceName") String sourceName, final @PathParam("instanceId") String instanceId) { - source.restartFunctionInstance(tenant, namespace, sourceName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); + sources().restartFunctionInstance(tenant, namespace, sourceName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); } @POST @@ -182,7 +185,7 @@ public void restartSource(final @PathParam("tenant") String tenant, public void restartSource(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sourceName") String sourceName) { - source.restartFunctionInstances(tenant, namespace, sourceName, clientAppId(), clientAuthData()); + sources().restartFunctionInstances(tenant, namespace, sourceName, clientAppId(), clientAuthData()); } @POST @@ -196,7 +199,7 @@ public void stopSource(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sourceName") String sourceName, final @PathParam("instanceId") String instanceId) { - source.stopFunctionInstance(tenant, namespace, sourceName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); + sources().stopFunctionInstance(tenant, namespace, sourceName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); } @POST @@ -209,7 +212,7 @@ public void stopSource(final @PathParam("tenant") String tenant, public void stopSource(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sourceName") String sourceName) { - source.stopFunctionInstances(tenant, namespace, sourceName, clientAppId(), clientAuthData()); + sources().stopFunctionInstances(tenant, namespace, sourceName, clientAppId(), clientAuthData()); } @POST @@ -223,7 +226,7 @@ public void startSource(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sourceName") String sourceName, final @PathParam("instanceId") String instanceId) { - source.startFunctionInstance(tenant, namespace, sourceName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); + sources().startFunctionInstance(tenant, namespace, sourceName, instanceId, this.uri.getRequestUri(), clientAppId(), clientAuthData()); } @POST @@ -236,7 +239,7 @@ public void startSource(final @PathParam("tenant") String tenant, public void startSource(final @PathParam("tenant") String tenant, final @PathParam("namespace") String namespace, final @PathParam("sourceName") String sourceName) { - source.startFunctionInstances(tenant, namespace, sourceName, clientAppId(), clientAuthData()); + sources().startFunctionInstances(tenant, namespace, sourceName, clientAppId(), clientAuthData()); } @GET @@ -252,7 +255,7 @@ public void startSource(final @PathParam("tenant") String tenant, @Produces(MediaType.APPLICATION_JSON) @Path("/builtinsources") public List getSourceList() { - return source.getSourceList(); + return sources().getSourceList(); } @GET @@ -272,7 +275,7 @@ public List getSourceList() { public List getSourceConfigDefinition( @ApiParam(value = "The name of the builtin source") final @PathParam("name") String name) throws IOException { - return source.getSourceConfigDefinition(name); + return sources().getSourceConfigDefinition(name); } @POST @@ -287,6 +290,6 @@ public List getSourceConfigDefinition( }) @Path("/reloadBuiltInSources") public void reloadSources() { - source.reloadConnectors(clientAppId()); + sources().reloadConnectors(clientAppId()); } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceDefinition.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceDefinition.java new file mode 100644 index 0000000000000..497f7bdd8709d --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceDefinition.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.pulsar.functions.worker.service; + +import lombok.Data; +import lombok.NoArgsConstructor; + +/** + * Metadata information about a function worker service. + */ +@Data +@NoArgsConstructor +public class WorkerServiceDefinition { + /** + * The name of the plugin. + */ + private String name; + + /** + * The description of the plugin to be used for user help. + */ + private String description; + + /** + * The class name for the plugin. + */ + private String handlerClass; +} diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceLoader.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceLoader.java new file mode 100644 index 0000000000000..3f49c515aacfd --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceLoader.java @@ -0,0 +1,168 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.pulsar.functions.worker.service; + +import static org.apache.commons.lang3.StringUtils.isEmpty; + +import com.sun.corba.se.pept.protocol.ProtocolHandler; +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collections; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.common.nar.NarClassLoader; +import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.apache.pulsar.functions.worker.PulsarWorkerService; +import org.apache.pulsar.functions.worker.WorkerConfig; +import org.apache.pulsar.functions.worker.WorkerService; + +/** + * Loader to load worker service. + */ +@Slf4j +public class WorkerServiceLoader { + + static final String PULSAR_FN_WORKER_DEFINITION_FILE = "pulsar-functions-worker-service.yml"; + + /** + * Retrieve the functions worker service definition from the provided worker service nar package. + * + * @param narPath the path to the worker service NAR package + * @return the worker service definition + * @throws IOException when fail to load the worker service or get the definition + */ + public static WorkerServiceDefinition getWorkerServiceDefinition(String narPath, String narExtractionDirectory) + throws IOException { + try (NarClassLoader ncl = NarClassLoader.getFromArchive( + new File(narPath), Collections.emptySet(), narExtractionDirectory)) { + return getWorkerServiceDefinition(ncl); + } + } + + private static WorkerServiceDefinition getWorkerServiceDefinition(NarClassLoader ncl) throws IOException { + String configStr = ncl.getServiceDefinition(PULSAR_FN_WORKER_DEFINITION_FILE); + + return ObjectMapperFactory.getThreadLocalYaml().readValue( + configStr, WorkerServiceDefinition.class + ); + } + + /** + * Load the worker service according to the handler definition. + * + * @param metadata the worker service definition. + * @return + */ + static WorkerServiceWithClassLoader load(WorkerServiceMetadata metadata, + String narExtractionDirectory) throws IOException { + NarClassLoader ncl = NarClassLoader.getFromArchive( + metadata.getArchivePath().toAbsolutePath().toFile(), + Collections.emptySet(), + ProtocolHandler.class.getClassLoader(), narExtractionDirectory); + + WorkerServiceDefinition phDef = getWorkerServiceDefinition(ncl); + if (StringUtils.isBlank(phDef.getHandlerClass())) { + throw new IOException("Functions Worker Service Nar Package `" + phDef.getName() + + "` does NOT provide a functions worker service implementation"); + } + + try { + Class handlerClass = ncl.loadClass(phDef.getHandlerClass()); + Object handler = handlerClass.newInstance(); + if (!(handler instanceof WorkerService)) { + throw new IOException("Class " + phDef.getHandlerClass() + + " does not implement worker service interface"); + } + WorkerService ph = (WorkerService) handler; + return new WorkerServiceWithClassLoader(ph, ncl); + } catch (Throwable t) { + rethrowIOException(t); + return null; + } + } + + private static void rethrowIOException(Throwable cause) + throws IOException { + if (cause instanceof IOException) { + throw (IOException) cause; + } else if (cause instanceof RuntimeException) { + throw (RuntimeException) cause; + } else if (cause instanceof Error) { + throw (Error) cause; + } else { + throw new IOException(cause.getMessage(), cause); + } + } + + /** + * Load the worker services for the given protocol list. + * + * @param workerConfig the functions worker config + * @return the worker service + */ + public static WorkerService load(WorkerConfig workerConfig) { + return load( + workerConfig.getFunctionsWorkerServiceNarPackage(), + workerConfig.getNarExtractionDirectory()); + } + + /** + * Load the worker services for the given protocol list. + * + * @param wsNarPackage worker service nar package + * @param narExtractionDirectory the directory to extract nar directory + * @return the worker service + */ + static WorkerService load(String wsNarPackage, String narExtractionDirectory) { + if (isEmpty(wsNarPackage)) { + return new PulsarWorkerService(); + } + + WorkerServiceDefinition definition; + try { + definition = getWorkerServiceDefinition( + wsNarPackage, + narExtractionDirectory + ); + } catch (IOException ioe) { + log.error("Failed to get the worker service definition from {}", + wsNarPackage, ioe); + throw new RuntimeException("Failed to get the worker service definition from " + + wsNarPackage, ioe); + } + + WorkerServiceMetadata metadata = new WorkerServiceMetadata(); + Path narPath = Paths.get(wsNarPackage); + metadata.setArchivePath(narPath); + metadata.setDefinition(definition); + + WorkerServiceWithClassLoader service; + try { + service = load(metadata, narExtractionDirectory); + } catch (IOException e) { + log.error("Failed to load the worker service {}", metadata, e); + throw new RuntimeException("Failed to load the worker service " + metadata, e); + } + + log.info("Successfully loaded worker service {}", metadata); + return service; + } +} diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceMetadata.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceMetadata.java new file mode 100644 index 0000000000000..6a44cb4dd273c --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceMetadata.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.pulsar.functions.worker.service; + +import java.nio.file.Path; +import lombok.Data; +import lombok.NoArgsConstructor; + +/** + * The metadata of worker service. + */ +@Data +@NoArgsConstructor +class WorkerServiceMetadata { + + /** + * The definition of the worker service package. + */ + private WorkerServiceDefinition definition; + + /** + * The path to the worker service package. + */ + private Path archivePath; + +} diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceWithClassLoader.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceWithClassLoader.java new file mode 100644 index 0000000000000..a7f6ac59bb1b5 --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/WorkerServiceWithClassLoader.java @@ -0,0 +1,124 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.pulsar.functions.worker.service; + +import java.io.IOException; +import lombok.Data; +import lombok.RequiredArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.broker.cache.ConfigurationCacheService; +import org.apache.pulsar.common.conf.InternalConfigurationData; +import org.apache.pulsar.common.nar.NarClassLoader; +import org.apache.pulsar.common.util.SimpleTextOutputStream; +import org.apache.pulsar.functions.worker.ErrorNotifier; +import org.apache.pulsar.functions.worker.WorkerConfig; +import org.apache.pulsar.functions.worker.WorkerService; +import org.apache.pulsar.functions.worker.service.api.Functions; +import org.apache.pulsar.functions.worker.service.api.FunctionsV2; +import org.apache.pulsar.functions.worker.service.api.Sinks; +import org.apache.pulsar.functions.worker.service.api.Sources; +import org.apache.pulsar.functions.worker.service.api.Workers; +import org.apache.pulsar.zookeeper.ZooKeeperCache; + +/** + * A worker service with its classloader. + */ +@Slf4j +@Data +@RequiredArgsConstructor +public class WorkerServiceWithClassLoader implements WorkerService { + + private final WorkerService service; + private final NarClassLoader classLoader; + + @Override + public WorkerConfig getWorkerConfig() { + return service.getWorkerConfig(); + } + + @Override + public void initAsStandalone(WorkerConfig workerConfig) throws Exception { + service.initAsStandalone(workerConfig); + } + + @Override + public void initInBroker(ServiceConfiguration brokerConfig, + WorkerConfig workerConfig, + ZooKeeperCache globalZkCache, + ConfigurationCacheService configurationCacheService, + InternalConfigurationData internalConf) throws Exception { + service.initInBroker(brokerConfig, workerConfig, globalZkCache, configurationCacheService, internalConf); + } + + @Override + public void start(AuthenticationService authenticationService, + AuthorizationService authorizationService, + ErrorNotifier errorNotifier) throws Exception { + service.start(authenticationService, authorizationService, errorNotifier); + } + + @Override + public void stop() { + service.stop(); + try { + classLoader.close(); + } catch (IOException e) { + log.warn("Failed to close the worker service class loader", e); + } + } + + @Override + public boolean isInitialized() { + return service.isInitialized(); + } + + @Override + public Functions getFunctions() { + return service.getFunctions(); + } + + @Override + public FunctionsV2 getFunctionsV2() { + return service.getFunctionsV2(); + } + + @Override + public Sinks getSinks() { + return service.getSinks(); + } + + @Override + public Sources getSources() { + return service.getSources(); + } + + @Override + public Workers getWorkers() { + return service.getWorkers(); + } + + @Override + public void generateFunctionsStats(SimpleTextOutputStream out) { + service.generateFunctionsStats(out); + } + +} diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Component.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Component.java new file mode 100644 index 0000000000000..6139a5c6e1067 --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Component.java @@ -0,0 +1,158 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.pulsar.functions.worker.service.api; + +import java.io.InputStream; +import java.net.URI; +import java.util.List; +import javax.ws.rs.core.StreamingOutput; +import org.apache.pulsar.broker.authentication.AuthenticationDataHttps; +import org.apache.pulsar.broker.authentication.AuthenticationDataSource; +import org.apache.pulsar.common.functions.FunctionConfig; +import org.apache.pulsar.common.functions.FunctionState; +import org.apache.pulsar.common.io.ConnectorDefinition; +import org.apache.pulsar.common.policies.data.FunctionStats; +import org.apache.pulsar.common.policies.data.FunctionStats.FunctionInstanceStats.FunctionInstanceStatsData; +import org.apache.pulsar.functions.worker.WorkerService; + +/** + * Provide service API to access components. + * @param worker service type + */ +public interface Component { + + W worker(); + + void deregisterFunction(final String tenant, + final String namespace, + final String componentName, + final String clientRole, + AuthenticationDataHttps clientAuthenticationDataHttps); + + FunctionConfig getFunctionInfo(final String tenant, + final String namespace, + final String componentName, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + void stopFunctionInstance(final String tenant, + final String namespace, + final String componentName, + final String instanceId, + final URI uri, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + void startFunctionInstance(final String tenant, + final String namespace, + final String componentName, + final String instanceId, + final URI uri, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + void restartFunctionInstance(final String tenant, + final String namespace, + final String componentName, + final String instanceId, + final URI uri, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + void startFunctionInstances(final String tenant, + final String namespace, + final String componentName, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + void stopFunctionInstances(final String tenant, + final String namespace, + final String componentName, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + void restartFunctionInstances(final String tenant, + final String namespace, + final String componentName, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + FunctionStats getFunctionStats(final String tenant, + final String namespace, + final String componentName, + final URI uri, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + FunctionInstanceStatsData getFunctionsInstanceStats(final String tenant, + final String namespace, + final String componentName, + final String instanceId, + final URI uri, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + String triggerFunction(final String tenant, + final String namespace, + final String functionName, + final String input, + final InputStream uploadedInputStream, + final String topic, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + List listFunctions(final String tenant, + final String namespace, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + FunctionState getFunctionState(final String tenant, + final String namespace, + final String functionName, + final String key, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + void putFunctionState(final String tenant, + final String namespace, + final String functionName, + final String key, + final FunctionState state, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + void uploadFunction(final InputStream uploadedInputStream, + final String path, + String clientRole); + + StreamingOutput downloadFunction(String path, + String clientRole, + AuthenticationDataHttps clientAuthenticationDataHttps); + + StreamingOutput downloadFunction(String tenant, + String namespace, + String componentName, + String clientRole, + AuthenticationDataHttps clientAuthenticationDataHttps); + + List getListOfConnectors(); + + + void reloadConnectors(String clientRole); +} diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Functions.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Functions.java new file mode 100644 index 0000000000000..9f68439919035 --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Functions.java @@ -0,0 +1,81 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.pulsar.functions.worker.service.api; + +import java.io.InputStream; +import java.net.URI; +import org.apache.pulsar.broker.authentication.AuthenticationDataHttps; +import org.apache.pulsar.broker.authentication.AuthenticationDataSource; +import org.apache.pulsar.common.functions.FunctionConfig; +import org.apache.pulsar.common.functions.UpdateOptions; +import org.apache.pulsar.common.policies.data.FunctionStatus; +import org.apache.pulsar.common.policies.data.FunctionStatus.FunctionInstanceStatus.FunctionInstanceStatusData; +import org.apache.pulsar.functions.worker.WorkerService; +import org.glassfish.jersey.media.multipart.FormDataContentDisposition; + +/** + * The service to manage functions. + */ +public interface Functions extends Component { + + void registerFunction(final String tenant, + final String namespace, + final String functionName, + final InputStream uploadedInputStream, + final FormDataContentDisposition fileDetail, + final String functionPkgUrl, + final FunctionConfig functionConfig, + final String clientRole, + AuthenticationDataHttps clientAuthenticationDataHttps); + + void updateFunction(final String tenant, + final String namespace, + final String functionName, + final InputStream uploadedInputStream, + final FormDataContentDisposition fileDetail, + final String functionPkgUrl, + final FunctionConfig functionConfig, + final String clientRole, + AuthenticationDataHttps clientAuthenticationDataHttps, + UpdateOptions updateOptions); + + void updateFunctionOnWorkerLeader(final String tenant, + final String namespace, + final String functionName, + final InputStream uploadedInputStream, + final boolean delete, + URI uri, + final String clientRole); + + FunctionStatus getFunctionStatus(final String tenant, + final String namespace, + final String componentName, + final URI uri, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + FunctionInstanceStatusData getFunctionInstanceStatus(final String tenant, + final String namespace, + final String componentName, + final String instanceId, + final URI uri, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + +} diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/FunctionsV2.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/FunctionsV2.java new file mode 100644 index 0000000000000..6b21657586a28 --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/FunctionsV2.java @@ -0,0 +1,126 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.pulsar.functions.worker.service.api; + +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; +import java.util.List; +import javax.ws.rs.core.Response; +import org.apache.pulsar.common.io.ConnectorDefinition; +import org.apache.pulsar.functions.worker.WorkerService; +import org.glassfish.jersey.media.multipart.FormDataContentDisposition; + +/** + * The v2 functions API to manage functions. + */ +public interface FunctionsV2 { + + Response getFunctionInfo(final String tenant, + final String namespace, + final String functionName, + String clientRole) throws IOException; + + Response getFunctionInstanceStatus(final String tenant, + final String namespace, + final String functionName, + final String instanceId, + URI uri, + String clientRole) throws IOException; + + Response getFunctionStatusV2(String tenant, + String namespace, + String functionName, + URI requestUri, + String clientRole) throws IOException; + + Response registerFunction(String tenant, + String namespace, + String functionName, + InputStream uploadedInputStream, + FormDataContentDisposition fileDetail, + String functionPkgUrl, + String functionDetailsJson, + String clientRole); + + Response updateFunction(String tenant, + String namespace, + String functionName, + InputStream uploadedInputStream, + FormDataContentDisposition fileDetail, + String functionPkgUrl, + String functionDetailsJson, + String clientRole); + + Response deregisterFunction(String tenant, + String namespace, + String functionName, + String clientAppId); + + Response listFunctions(String tenant, String namespace, String clientRole); + + Response triggerFunction(String tenant, + String namespace, + String functionName, + String triggerValue, + InputStream triggerStream, + String topic, + String clientRole); + + Response getFunctionState(String tenant, + String namespace, + String functionName, + String key, + String clientRole); + + + Response restartFunctionInstance(String tenant, + String namespace, + String functionName, + String instanceId, + URI uri, + String clientRole); + + Response restartFunctionInstances(String tenant, + String namespace, + String functionName, + String clientRole); + + Response stopFunctionInstance(String tenant, + String namespace, + String functionName, + String instanceId, + URI uri, + String clientRole); + + Response stopFunctionInstances(String tenant, + String namespace, + String functionName, + String clientRole); + + Response uploadFunction(InputStream uploadedInputStream, + String path, + String clientRole); + + Response downloadFunction(String path, String clientRole); + + + List getListOfConnectors(); + +} diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Sinks.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Sinks.java new file mode 100644 index 0000000000000..ae501299406ce --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Sinks.java @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.pulsar.functions.worker.service.api; + +import java.io.InputStream; +import java.net.URI; +import java.util.List; +import org.apache.pulsar.broker.authentication.AuthenticationDataHttps; +import org.apache.pulsar.broker.authentication.AuthenticationDataSource; +import org.apache.pulsar.common.functions.UpdateOptions; +import org.apache.pulsar.common.io.ConfigFieldDefinition; +import org.apache.pulsar.common.io.ConnectorDefinition; +import org.apache.pulsar.common.io.SinkConfig; +import org.apache.pulsar.common.policies.data.SinkStatus; +import org.apache.pulsar.common.policies.data.SinkStatus.SinkInstanceStatus.SinkInstanceStatusData; +import org.apache.pulsar.functions.worker.WorkerService; +import org.glassfish.jersey.media.multipart.FormDataContentDisposition; + +/** + * The service to manage sinks. + */ +public interface Sinks extends Component { + + void registerSink(final String tenant, + final String namespace, + final String sinkName, + final InputStream uploadedInputStream, + final FormDataContentDisposition fileDetail, + final String sinkPkgUrl, + final SinkConfig sinkConfig, + final String clientRole, + AuthenticationDataHttps clientAuthenticationDataHttps); + + void updateSink(final String tenant, + final String namespace, + final String sinkName, + final InputStream uploadedInputStream, + final FormDataContentDisposition fileDetail, + final String sinkPkgUrl, + final SinkConfig sinkConfig, + final String clientRole, + AuthenticationDataHttps clientAuthenticationDataHttps, + UpdateOptions updateOptions); + + SinkInstanceStatusData getSinkInstanceStatus(final String tenant, + final String namespace, + final String sinkName, + final String instanceId, + final URI uri, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + SinkStatus getSinkStatus(final String tenant, + final String namespace, + final String componentName, + final URI uri, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + SinkConfig getSinkInfo(final String tenant, + final String namespace, + final String componentName); + + List getSinkList(); + + + List getSinkConfigDefinition(String name); + +} diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Sources.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Sources.java new file mode 100644 index 0000000000000..d66f2540e99e9 --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Sources.java @@ -0,0 +1,86 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.pulsar.functions.worker.service.api; + +import java.io.InputStream; +import java.net.URI; +import java.util.List; +import org.apache.pulsar.broker.authentication.AuthenticationDataHttps; +import org.apache.pulsar.broker.authentication.AuthenticationDataSource; +import org.apache.pulsar.common.functions.UpdateOptions; +import org.apache.pulsar.common.io.ConfigFieldDefinition; +import org.apache.pulsar.common.io.ConnectorDefinition; +import org.apache.pulsar.common.io.SourceConfig; +import org.apache.pulsar.common.policies.data.SourceStatus; +import org.apache.pulsar.common.policies.data.SourceStatus.SourceInstanceStatus.SourceInstanceStatusData; +import org.apache.pulsar.functions.worker.WorkerService; +import org.glassfish.jersey.media.multipart.FormDataContentDisposition; + +/** + * The service to manage sources. + */ +public interface Sources extends Component { + + void registerSource(final String tenant, + final String namespace, + final String sourceName, + final InputStream uploadedInputStream, + final FormDataContentDisposition fileDetail, + final String sourcePkgUrl, + final SourceConfig sourceConfig, + final String clientRole, + AuthenticationDataHttps clientAuthenticationDataHttps); + + void updateSource(final String tenant, + final String namespace, + final String sourceName, + final InputStream uploadedInputStream, + final FormDataContentDisposition fileDetail, + final String sourcePkgUrl, + final SourceConfig sourceConfig, + final String clientRole, + AuthenticationDataHttps clientAuthenticationDataHttps, + UpdateOptions updateOptions); + + + SourceStatus getSourceStatus(final String tenant, + final String namespace, + final String componentName, + final URI uri, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + + SourceInstanceStatusData getSourceInstanceStatus(final String tenant, + final String namespace, + final String sourceName, + final String instanceId, + final URI uri, + final String clientRole, + final AuthenticationDataSource clientAuthenticationDataHttps); + + SourceConfig getSourceInfo(final String tenant, + final String namespace, + final String componentName); + + List getSourceList(); + + + List getSourceConfigDefinition(String name); +} diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Workers.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Workers.java new file mode 100644 index 0000000000000..970de9dc970f6 --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/Workers.java @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.pulsar.functions.worker.service.api; + +import java.io.IOException; +import java.net.URI; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import org.apache.pulsar.common.functions.WorkerInfo; +import org.apache.pulsar.common.io.ConnectorDefinition; +import org.apache.pulsar.common.policies.data.WorkerFunctionInstanceStats; +import org.apache.pulsar.common.stats.Metrics; +import org.apache.pulsar.functions.worker.WorkerService; + +/** + * The service to manage worker. + */ +public interface Workers { + + List getCluster(String clientRole); + + WorkerInfo getClusterLeader(String clientRole); + + Map> getAssignments(String clientRole); + + List getWorkerMetrics(final String clientRole); + + List getFunctionsMetrics(String clientRole) throws IOException; + + List getListOfConnectors(String clientRole); + + void rebalance(final URI uri, final String clientRole); + + Boolean isLeaderReady(final String clientRole); + +} diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/package-info.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/package-info.java new file mode 100644 index 0000000000000..1e48c886f044a --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/api/package-info.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** + * Pulsar Worker Services API. + */ +package org.apache.pulsar.functions.worker.service.api; \ No newline at end of file diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/package-info.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/package-info.java new file mode 100644 index 0000000000000..26b81aebd1dac --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/service/package-info.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** + * Pulsar Functions Worker Service. + */ +package org.apache.pulsar.functions.worker.service; \ No newline at end of file diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionAssignmentTailerTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionAssignmentTailerTest.java index e0850d42b4268..8adeec5e9c708 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionAssignmentTailerTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionAssignmentTailerTest.java @@ -138,7 +138,7 @@ public Boolean answer(InvocationOnMock invocationOnMock) throws Throwable { doReturn(readerBuilder).when(readerBuilder).readCompacted(anyBoolean()); doReturn(reader).when(readerBuilder).create(); - WorkerService workerService = mock(WorkerService.class); + PulsarWorkerService workerService = mock(PulsarWorkerService.class); doReturn(pulsarClient).when(workerService).getClient(); doReturn(mock(PulsarAdmin.class)).when(workerService).getFunctionAdmin(); @@ -253,7 +253,7 @@ public Boolean answer(InvocationOnMock invocationOnMock) throws Throwable { doReturn(readerBuilder).when(readerBuilder).readCompacted(anyBoolean()); doReturn(reader).when(readerBuilder).create(); - WorkerService workerService = mock(WorkerService.class); + PulsarWorkerService workerService = mock(PulsarWorkerService.class); doReturn(pulsarClient).when(workerService).getClient(); doReturn(mock(PulsarAdmin.class)).when(workerService).getFunctionAdmin(); @@ -376,7 +376,7 @@ public Boolean answer(InvocationOnMock invocationOnMock) throws Throwable { doReturn(readerBuilder).when(readerBuilder).readCompacted(anyBoolean()); doReturn(reader).when(readerBuilder).create(); - WorkerService workerService = mock(WorkerService.class); + PulsarWorkerService workerService = mock(PulsarWorkerService.class); doReturn(pulsarClient).when(workerService).getClient(); doReturn(mock(PulsarAdmin.class)).when(workerService).getFunctionAdmin(); diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionRuntimeManagerTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionRuntimeManagerTest.java index 1d6dd21db436a..9e9ac22b1f212 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionRuntimeManagerTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionRuntimeManagerTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.functions.worker; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyBoolean; import static org.mockito.Mockito.anyString; @@ -30,6 +31,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.powermock.api.mockito.PowerMockito.mockStatic; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; @@ -48,17 +50,25 @@ import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.common.api.proto.PulsarApi; +import org.apache.pulsar.common.functions.AuthenticationConfig; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.functions.proto.Function; +import org.apache.pulsar.functions.runtime.RuntimeFactory; import org.apache.pulsar.functions.runtime.kubernetes.KubernetesRuntime; import org.apache.pulsar.functions.runtime.kubernetes.KubernetesRuntimeFactory; import org.apache.pulsar.functions.runtime.kubernetes.KubernetesRuntimeFactoryConfig; import org.apache.pulsar.functions.runtime.process.ProcessRuntimeFactory; import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory; import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactoryConfig; +import org.apache.pulsar.functions.secretsproviderconfigurator.SecretsProviderConfigurator; import org.apache.pulsar.functions.utils.FunctionCommon; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.testng.IObjectFactory; +import org.testng.annotations.ObjectFactory; import org.testng.annotations.Test; import java.util.HashMap; @@ -68,9 +78,21 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; +@PrepareForTest({FunctionRuntimeManager.class, RuntimeFactory.class}) @Slf4j +@PowerMockIgnore({ + "javax.management.*", + "javax.ws.*", + "org.apache.logging.log4j.*", + "org.apache.pulsar.functions.runtime.thread" +}) public class FunctionRuntimeManagerTest { + @ObjectFactory + public IObjectFactory getObjectFactory() { + return new org.powermock.modules.testng.PowerMockObjectFactory(); + } + @Test public void testProcessAssignmentUpdateAddFunctions() throws Exception { @@ -92,9 +114,14 @@ public void testProcessAssignmentUpdateAddFunctions() throws Exception { doReturn(readerBuilder).when(readerBuilder).startMessageId(any()); doReturn(readerBuilder).when(readerBuilder).readCompacted(anyBoolean()); doReturn(mock(Reader.class)).when(readerBuilder).create(); - WorkerService workerService = mock(WorkerService.class); + PulsarWorkerService workerService = mock(PulsarWorkerService.class); doReturn(pulsarClient).when(workerService).getClient(); doReturn(mock(PulsarAdmin.class)).when(workerService).getFunctionAdmin(); + + mockStatic(RuntimeFactory.class); + PowerMockito.when(RuntimeFactory.getFuntionRuntimeFactory(eq(ThreadRuntimeFactory.class.getName()))) + .thenReturn(new ThreadRuntimeFactory()); + // test new assignment add functions FunctionRuntimeManager functionRuntimeManager = spy(new FunctionRuntimeManager( workerConfig, @@ -176,10 +203,14 @@ public void testProcessAssignmentUpdateDeleteFunctions() throws Exception { doReturn(readerBuilder).when(readerBuilder).startMessageId(any()); doReturn(readerBuilder).when(readerBuilder).readCompacted(anyBoolean()); doReturn(mock(Reader.class)).when(readerBuilder).create(); - WorkerService workerService = mock(WorkerService.class); + PulsarWorkerService workerService = mock(PulsarWorkerService.class); doReturn(pulsarClient).when(workerService).getClient(); doReturn(mock(PulsarAdmin.class)).when(workerService).getFunctionAdmin(); + mockStatic(RuntimeFactory.class); + PowerMockito.when(RuntimeFactory.getFuntionRuntimeFactory(eq(ThreadRuntimeFactory.class.getName()))) + .thenReturn(new ThreadRuntimeFactory()); + // test new assignment delete functions FunctionRuntimeManager functionRuntimeManager = spy(new FunctionRuntimeManager( workerConfig, @@ -264,10 +295,14 @@ public void testProcessAssignmentUpdateModifyFunctions() throws Exception { doReturn(readerBuilder).when(readerBuilder).startMessageId(any()); doReturn(readerBuilder).when(readerBuilder).readCompacted(anyBoolean()); doReturn(mock(Reader.class)).when(readerBuilder).create(); - WorkerService workerService = mock(WorkerService.class); + PulsarWorkerService workerService = mock(PulsarWorkerService.class); doReturn(pulsarClient).when(workerService).getClient(); doReturn(mock(PulsarAdmin.class)).when(workerService).getFunctionAdmin(); + mockStatic(RuntimeFactory.class); + PowerMockito.when(RuntimeFactory.getFuntionRuntimeFactory(eq(ThreadRuntimeFactory.class.getName()))) + .thenReturn(new ThreadRuntimeFactory()); + // test new assignment update functions FunctionRuntimeManager functionRuntimeManager = new FunctionRuntimeManager( workerConfig, @@ -396,10 +431,14 @@ public void testReassignment() throws Exception { doReturn(readerBuilder).when(readerBuilder).startMessageId(any()); doReturn(readerBuilder).when(readerBuilder).readCompacted(anyBoolean()); doReturn(mock(Reader.class)).when(readerBuilder).create(); - WorkerService workerService = mock(WorkerService.class); + PulsarWorkerService workerService = mock(PulsarWorkerService.class); doReturn(pulsarClient).when(workerService).getClient(); doReturn(mock(PulsarAdmin.class)).when(workerService).getFunctionAdmin(); + mockStatic(RuntimeFactory.class); + PowerMockito.when(RuntimeFactory.getFuntionRuntimeFactory(eq(ThreadRuntimeFactory.class.getName()))) + .thenReturn(new ThreadRuntimeFactory()); + // test new assignment update functions FunctionRuntimeManager functionRuntimeManager = new FunctionRuntimeManager( workerConfig, @@ -582,12 +621,16 @@ public Boolean answer(InvocationOnMock invocationOnMock) throws Throwable { doReturn(readerBuilder).when(readerBuilder).readCompacted(anyBoolean()); doReturn(reader).when(readerBuilder).create(); - WorkerService workerService = mock(WorkerService.class); + PulsarWorkerService workerService = mock(PulsarWorkerService.class); doReturn(pulsarClient).when(workerService).getClient(); doReturn(mock(PulsarAdmin.class)).when(workerService).getFunctionAdmin(); ErrorNotifier errorNotifier = mock(ErrorNotifier.class); + mockStatic(RuntimeFactory.class); + PowerMockito.when(RuntimeFactory.getFuntionRuntimeFactory(eq(ThreadRuntimeFactory.class.getName()))) + .thenReturn(new ThreadRuntimeFactory()); + // test new assignment add functions FunctionRuntimeManager functionRuntimeManager = new FunctionRuntimeManager( workerConfig, @@ -646,11 +689,18 @@ public void testExternallyManagedRuntimeUpdate() throws Exception { doReturn(readerBuilder).when(readerBuilder).startMessageId(any()); doReturn(readerBuilder).when(readerBuilder).readCompacted(anyBoolean()); doReturn(mock(Reader.class)).when(readerBuilder).create(); - WorkerService workerService = mock(WorkerService.class); + PulsarWorkerService workerService = mock(PulsarWorkerService.class); doReturn(pulsarClient).when(workerService).getClient(); doReturn(mock(PulsarAdmin.class)).when(workerService).getFunctionAdmin(); KubernetesRuntimeFactory kubernetesRuntimeFactory = mock(KubernetesRuntimeFactory.class); + doNothing().when(kubernetesRuntimeFactory).initialize( + any(WorkerConfig.class), + any(AuthenticationConfig.class), + any(SecretsProviderConfigurator.class), + any(), + any() + ); doNothing().when(kubernetesRuntimeFactory).setupClient(); doReturn(true).when(kubernetesRuntimeFactory).externallyManaged(); @@ -661,6 +711,10 @@ public void testExternallyManagedRuntimeUpdate() throws Exception { workerConfig, kubernetesRuntimeFactory, null, null, null, null)); + mockStatic(RuntimeFactory.class); + PowerMockito.when(RuntimeFactory.getFuntionRuntimeFactory(anyString())) + .thenReturn(kubernetesRuntimeFactory); + // test new assignment update functions FunctionRuntimeManager functionRuntimeManager = new FunctionRuntimeManager( workerConfig, @@ -770,7 +824,7 @@ public void testFunctionRuntimeSetCorrectly() { workerConfig.setFunctionAssignmentTopicName("assignments"); new FunctionRuntimeManager( workerConfig, - mock(WorkerService.class), + mock(PulsarWorkerService.class), mock(Namespace.class), mock(MembershipManager.class), mock(ConnectorsManager.class), @@ -796,7 +850,7 @@ public void testFunctionRuntimeSetCorrectly() { workerConfig.setFunctionAssignmentTopicName("assignments"); new FunctionRuntimeManager( workerConfig, - mock(WorkerService.class), + mock(PulsarWorkerService.class), mock(Namespace.class), mock(MembershipManager.class), mock(ConnectorsManager.class), @@ -822,7 +876,7 @@ public void testFunctionRuntimeSetCorrectly() { workerConfig.setFunctionAssignmentTopicName("assignments"); new FunctionRuntimeManager( workerConfig, - mock(WorkerService.class), + mock(PulsarWorkerService.class), mock(Namespace.class), mock(MembershipManager.class), mock(ConnectorsManager.class), @@ -846,9 +900,14 @@ public void testFunctionRuntimeSetCorrectly() { workerConfig.setPulsarServiceUrl("pulsar://localhost:6650"); workerConfig.setStateStorageServiceUrl("foo"); workerConfig.setFunctionAssignmentTopicName("assignments"); + + mockStatic(RuntimeFactory.class); + PowerMockito.when(RuntimeFactory.getFuntionRuntimeFactory(eq(ThreadRuntimeFactory.class.getName()))) + .thenReturn(new ThreadRuntimeFactory()); + FunctionRuntimeManager functionRuntimeManager = new FunctionRuntimeManager( workerConfig, - mock(WorkerService.class), + mock(PulsarWorkerService.class), mock(Namespace.class), mock(MembershipManager.class), mock(ConnectorsManager.class), @@ -859,6 +918,7 @@ public void testFunctionRuntimeSetCorrectly() { assertEquals(functionRuntimeManager.getRuntimeFactory().getClass(), ThreadRuntimeFactory.class); } catch (Exception e) { + log.error("Failed to initialize the runtime manager : ", e); fail(); } } @@ -878,9 +938,22 @@ public void testFunctionRuntimeFactoryConfigsBackwardsCompatibility() throws Exc WorkerConfig workerConfig = new WorkerConfig(); workerConfig.setKubernetesContainerFactory(kubernetesContainerFactory); + KubernetesRuntimeFactory mockedKubernetesRuntimeFactory = spy(new KubernetesRuntimeFactory()); + doNothing().when(mockedKubernetesRuntimeFactory).initialize( + any(WorkerConfig.class), + any(AuthenticationConfig.class), + any(SecretsProviderConfigurator.class), + any(), + any() + ); + doNothing().when(mockedKubernetesRuntimeFactory).setupClient(); + doReturn(true).when(mockedKubernetesRuntimeFactory).externallyManaged(); + PowerMockito.whenNew(KubernetesRuntimeFactory.class) + .withNoArguments().thenReturn(mockedKubernetesRuntimeFactory); + FunctionRuntimeManager functionRuntimeManager = new FunctionRuntimeManager( workerConfig, - mock(WorkerService.class), + mock(PulsarWorkerService.class), mock(Namespace.class), mock(MembershipManager.class), mock(ConnectorsManager.class), @@ -889,7 +962,6 @@ public void testFunctionRuntimeFactoryConfigsBackwardsCompatibility() throws Exc mock(WorkerStatsManager.class), mock(ErrorNotifier.class)); - assertEquals(functionRuntimeManager.getRuntimeFactory().getClass(), KubernetesRuntimeFactory.class); KubernetesRuntimeFactory kubernetesRuntimeFactory = (KubernetesRuntimeFactory) functionRuntimeManager.getRuntimeFactory(); assertEquals(kubernetesRuntimeFactory.getK8Uri(), "k8Uri"); assertEquals(kubernetesRuntimeFactory.getJobNamespace(), "jobNamespace"); @@ -910,7 +982,7 @@ public void testFunctionRuntimeFactoryConfigsBackwardsCompatibility() throws Exc functionRuntimeManager = new FunctionRuntimeManager( workerConfig, - mock(WorkerService.class), + mock(PulsarWorkerService.class), mock(Namespace.class), mock(MembershipManager.class), mock(ConnectorsManager.class), @@ -936,7 +1008,7 @@ public void testFunctionRuntimeFactoryConfigsBackwardsCompatibility() throws Exc functionRuntimeManager = new FunctionRuntimeManager( workerConfig, - mock(WorkerService.class), + mock(PulsarWorkerService.class), mock(Namespace.class), mock(MembershipManager.class), mock(ConnectorsManager.class), diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/MembershipManagerTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/MembershipManagerTest.java index 50012be31daac..84832e74df83e 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/MembershipManagerTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/MembershipManagerTest.java @@ -98,7 +98,7 @@ public void testCheckFailuresNoFailures() throws Exception { doReturn(readerBuilder).when(readerBuilder).readCompacted(true); doReturn(readerBuilder).when(readerBuilder).startMessageId(any()); doReturn(mock(Reader.class)).when(readerBuilder).create(); - WorkerService workerService = mock(WorkerService.class); + PulsarWorkerService workerService = mock(PulsarWorkerService.class); doReturn(pulsarClient).when(workerService).getClient(); doReturn(workerConfig).when(workerService).getWorkerConfig(); PulsarAdmin pulsarAdmin = mock(PulsarAdmin.class); @@ -173,7 +173,7 @@ public void testCheckFailuresSomeFailures() throws Exception { doReturn(readerBuilder).when(readerBuilder).startMessageId(any()); doReturn(readerBuilder).when(readerBuilder).readCompacted(true); doReturn(mock(Reader.class)).when(readerBuilder).create(); - WorkerService workerService = mock(WorkerService.class); + PulsarWorkerService workerService = mock(PulsarWorkerService.class); doReturn(pulsarClient).when(workerService).getClient(); doReturn(workerConfig).when(workerService).getWorkerConfig(); PulsarAdmin pulsarAdmin = mock(PulsarAdmin.class); @@ -263,7 +263,7 @@ public void testCheckFailuresSomeUnassigned() throws Exception { doReturn(readerBuilder).when(readerBuilder).readCompacted(true); doReturn(readerBuilder).when(readerBuilder).startMessageId(any()); doReturn(mock(Reader.class)).when(readerBuilder).create(); - WorkerService workerService = mock(WorkerService.class); + PulsarWorkerService workerService = mock(PulsarWorkerService.class); doReturn(pulsarClient).when(workerService).getClient(); doReturn(workerConfig).when(workerService).getWorkerConfig(); PulsarAdmin pulsarAdmin = mock(PulsarAdmin.class); @@ -345,7 +345,7 @@ public void testHeartBeatFunctionWorkerDown() throws Exception { doReturn(readerBuilder).when(readerBuilder).readCompacted(true); doReturn(readerBuilder).when(readerBuilder).startMessageId(any()); doReturn(mock(Reader.class)).when(readerBuilder).create(); - WorkerService workerService = mock(WorkerService.class); + PulsarWorkerService workerService = mock(PulsarWorkerService.class); doReturn(pulsarClient).when(workerService).getClient(); doReturn(workerConfig).when(workerService).getWorkerConfig(); PulsarAdmin pulsarAdmin = mock(PulsarAdmin.class); diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java index bd65a0b2bfdcc..4d2867f48d6c2 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java @@ -42,6 +42,7 @@ import org.apache.pulsar.functions.worker.FunctionMetaDataManager; import org.apache.pulsar.functions.worker.FunctionRuntimeInfo; import org.apache.pulsar.functions.worker.FunctionRuntimeManager; +import org.apache.pulsar.functions.worker.PulsarWorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; @@ -107,7 +108,7 @@ public String process(String input, Context context) { private static final String workerId = "worker-0"; private static final String superUser = "superUser"; - private WorkerService mockedWorkerService; + private PulsarWorkerService mockedWorkerService; private PulsarAdmin mockedPulsarAdmin; private Tenants mockedTenants; private Namespaces mockedNamespaces; @@ -138,7 +139,7 @@ public void setup() throws Exception { this.mockedFunctionMetadata = Function.FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); namespaceList.add(tenant + "/" + namespace); - this.mockedWorkerService = mock(WorkerService.class); + this.mockedWorkerService = mock(PulsarWorkerService.class); when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager); when(mockedFunctionRunTimeManager.getRuntimeFactory()).thenReturn(mockedRuntimeFactory); diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java index 2a50623c8612e..e0403340c9d9f 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java @@ -77,7 +77,12 @@ import org.apache.pulsar.functions.runtime.RuntimeFactory; import org.apache.pulsar.functions.source.TopicSchema; import org.apache.pulsar.functions.utils.FunctionConfigUtils; -import org.apache.pulsar.functions.worker.*; +import org.apache.pulsar.functions.worker.FunctionMetaDataManager; +import org.apache.pulsar.functions.worker.FunctionRuntimeManager; +import org.apache.pulsar.functions.worker.LeaderService; +import org.apache.pulsar.functions.worker.PulsarWorkerService; +import org.apache.pulsar.functions.worker.WorkerConfig; +import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.rest.api.FunctionsImpl; import org.apache.pulsar.functions.worker.rest.api.FunctionsImplV2; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; @@ -123,7 +128,7 @@ public String process(String input, Context context) { } private static final int parallelism = 1; - private WorkerService mockedWorkerService; + private PulsarWorkerService mockedWorkerService; private PulsarAdmin mockedPulsarAdmin; private Tenants mockedTenants; private Namespaces mockedNamespaces; @@ -158,7 +163,7 @@ public void setup() throws Exception { this.mockedFunctionMetadata = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); namespaceList.add(tenant + "/" + namespace); - this.mockedWorkerService = mock(WorkerService.class); + this.mockedWorkerService = mock(PulsarWorkerService.class); when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); when(mockedWorkerService.getLeaderService()).thenReturn(mockedLeaderService); when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager); diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java index 395951abd5c9d..97b2e9629da0f 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java @@ -74,7 +74,12 @@ import org.apache.pulsar.functions.runtime.RuntimeFactory; import org.apache.pulsar.functions.source.TopicSchema; import org.apache.pulsar.functions.utils.FunctionConfigUtils; -import org.apache.pulsar.functions.worker.*; +import org.apache.pulsar.functions.worker.FunctionMetaDataManager; +import org.apache.pulsar.functions.worker.FunctionRuntimeManager; +import org.apache.pulsar.functions.worker.LeaderService; +import org.apache.pulsar.functions.worker.PulsarWorkerService; +import org.apache.pulsar.functions.worker.WorkerConfig; +import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.rest.api.FunctionsImpl; import org.apache.pulsar.functions.worker.rest.api.v2.FunctionsApiV2Resource; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; @@ -128,7 +133,7 @@ public void accept(String s) { } private static final int parallelism = 1; - private WorkerService mockedWorkerService; + private PulsarWorkerService mockedWorkerService; private PulsarAdmin mockedPulsarAdmin; private Tenants mockedTenants; private Namespaces mockedNamespaces; @@ -163,7 +168,7 @@ public void setup() throws Exception { this.mockedFunctionMetadata = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); namespaceList.add(tenant + "/" + namespace); - this.mockedWorkerService = mock(WorkerService.class); + this.mockedWorkerService = mock(PulsarWorkerService.class); when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager); when(mockedWorkerService.getLeaderService()).thenReturn(mockedLeaderService); @@ -1438,7 +1443,7 @@ public void testOnlyGetSources() { public void testDownloadFunctionHttpUrl() throws Exception { String jarHttpUrl = "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-common/2.4.2/pulsar-common-2.4.2.jar"; String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - WorkerService worker = mock(WorkerService.class); + PulsarWorkerService worker = mock(PulsarWorkerService.class); doReturn(true).when(worker).isInitialized(); WorkerConfig config = mock(WorkerConfig.class); when(config.isAuthorizationEnabled()).thenReturn(false); @@ -1460,7 +1465,7 @@ public void testDownloadFunctionFile() throws Exception { File file = Paths.get(fileUrl.toURI()).toFile(); String fileLocation = file.getAbsolutePath(); String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - WorkerService worker = mock(WorkerService.class); + PulsarWorkerService worker = mock(PulsarWorkerService.class); doReturn(true).when(worker).isInitialized(); WorkerConfig config = mock(WorkerConfig.class); when(config.isAuthorizationEnabled()).thenReturn(false); diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java index 6b5510650a93a..e579205053231 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java @@ -42,7 +42,12 @@ import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.SinkConfigUtils; import org.apache.pulsar.functions.utils.io.ConnectorUtils; -import org.apache.pulsar.functions.worker.*; +import org.apache.pulsar.functions.worker.FunctionMetaDataManager; +import org.apache.pulsar.functions.worker.FunctionRuntimeManager; +import org.apache.pulsar.functions.worker.LeaderService; +import org.apache.pulsar.functions.worker.PulsarWorkerService; +import org.apache.pulsar.functions.worker.WorkerConfig; +import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.rest.api.SinksImpl; import org.apache.pulsar.io.cassandra.CassandraStringSink; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; @@ -108,7 +113,7 @@ public IObjectFactory getObjectFactory() { private String JAR_FILE_PATH; private String INVALID_JAR_FILE_PATH; - private WorkerService mockedWorkerService; + private PulsarWorkerService mockedWorkerService; private PulsarAdmin mockedPulsarAdmin; private Tenants mockedTenants; private Namespaces mockedNamespaces; @@ -142,7 +147,7 @@ public void setup() throws Exception { this.mockedLeaderService = mock(LeaderService.class); namespaceList.add(tenant + "/" + namespace); - this.mockedWorkerService = mock(WorkerService.class); + this.mockedWorkerService = mock(PulsarWorkerService.class); when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); when(mockedWorkerService.getLeaderService()).thenReturn(mockedLeaderService); when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager); diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java index cf6945f1f9c4e..2b16c42e75741 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java @@ -105,7 +105,7 @@ public IObjectFactory getObjectFactory() { private String JAR_FILE_PATH; private String INVALID_JAR_FILE_PATH; - private WorkerService mockedWorkerService; + private PulsarWorkerService mockedWorkerService; private PulsarAdmin mockedPulsarAdmin; private Tenants mockedTenants; private Namespaces mockedNamespaces; @@ -139,7 +139,7 @@ public void setup() throws Exception { this.mockedLeaderService = mock(LeaderService.class); namespaceList.add(tenant + "/" + namespace); - this.mockedWorkerService = mock(WorkerService.class); + this.mockedWorkerService = mock(PulsarWorkerService.class); when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); when(mockedWorkerService.getLeaderService()).thenReturn(mockedLeaderService); when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager);