You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2018/03/06 05:46:46 UTC

[GitHub] merlimat closed pull request #1332: Merge pulsar-functions dist package into pulsar binary distribution

merlimat closed pull request #1332:  Merge pulsar-functions dist package into pulsar binary distribution
URL: https://github.com/apache/incubator-pulsar/pull/1332
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/all/pom.xml b/all/pom.xml
index 0c02d51ef..6273f4bf4 100644
--- a/all/pom.xml
+++ b/all/pom.xml
@@ -104,6 +104,15 @@
       <artifactId>pulsar-zookeeper</artifactId>
       <version>${project.version}</version>
     </dependency>
+
+    <!-- function examples -->
+    <dependency>
+      <groupId>org.apache.pulsar</groupId>
+      <artifactId>pulsar-functions-api-examples</artifactId>
+      <version>${project.version}</version>
+      <!-- make sure the api examples are compiled before assembly -->
+      <scope>provided</scope>
+    </dependency>
   </dependencies>
 
   <build>
diff --git a/all/src/assemble/bin.xml b/all/src/assemble/bin.xml
index b295bab9e..3a994acf9 100644
--- a/all/src/assemble/bin.xml
+++ b/all/src/assemble/bin.xml
@@ -43,6 +43,10 @@
     <fileSet>
       <directory>${basedir}/licenses</directory>
     </fileSet>
+    <fileSet>
+      <directory>${basedir}/../pulsar-functions/runtime/target/python-instance</directory>
+      <outputDirectory>instances/python-instance</outputDirectory>
+    </fileSet>
   </fileSets>
   <files>
     <file>
@@ -68,6 +72,16 @@
       <outputDirectory>.</outputDirectory>
       <fileMode>644</fileMode>
     </file>
+    <file>
+      <source>${basedir}/../pulsar-functions/runtime/target/java-instance.jar</source>
+      <destName>java-instance.jar</destName>
+      <outputDirectory>instances</outputDirectory>
+    </file>
+    <file>
+      <source>${basedir}/../pulsar-functions/java-examples/target/pulsar-functions-api-examples.jar</source>
+      <destName>api-examples.jar</destName>
+      <outputDirectory>examples</outputDirectory>
+    </file>
   </files>
   <dependencySets>
     <dependencySet>
diff --git a/pulsar-functions/bin/pulsar b/pulsar-functions/bin/pulsar
deleted file mode 100755
index ba219a1b6..000000000
--- a/pulsar-functions/bin/pulsar
+++ /dev/null
@@ -1,192 +0,0 @@
-#!/usr/bin/env bash
-#
-# 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.
-#
-
-BINDIR=$(dirname "$0")
-PULSAR_HOME=`cd $BINDIR/..;pwd`
-
-DEFAULT_WORKER_CONF=$PULSAR_HOME/conf/function_worker.yml
-DEFAULT_LOG_CONF=$PULSAR_HOME/conf/log4j2.yml
-DEFAULT_JAVA_INSTANCE_JAR=$PULSAR_HOME/instances/java-instance.jar
-JAVA_INSTANCE_JAR=${PULSAR_JAVA_INSTANCE_JAR:-"${DEFAULT_JAVA_INSTANCE_JAR}"}
-DEFAULT_PY_INSTANCE_FILE=$PULSAR_HOME/instances/python-instance/python_instance_main.py
-PY_INSTANCE_FILE=${PULSAR_PY_INSTANCE_FILE:-"${DEFAULT_PY_INSTANCE_FILE}"}
-
-if [ -f "$PULSAR_HOME/conf/pulsar_env.sh" ]
-then
-    . "$PULSAR_HOME/conf/pulsar_env.sh"
-fi
-
-# Check for the java to use
-if [[ -z $JAVA_HOME ]]; then
-    JAVA=$(which java)
-    if [ $? != 0 ]; then
-        echo "Error: JAVA_HOME not set, and no java executable found in $PATH." 1>&2
-        exit 1
-    fi
-else
-    JAVA=$JAVA_HOME/bin/java
-fi
-
-# exclude tests jar
-RELEASE_JAR=`ls $PULSAR_HOME/lib/*pulsar-functions-worker*.jar 2> /dev/null | grep -v tests | tail -1`
-if [ $? == 0 ]; then
-    PULSAR_JAR=$RELEASE_JAR
-fi
-
-# exclude tests jar
-if [ -z "$PULSAR_JAR" ]; then
-    BUILT_JAR=`ls $PULSAR_HOME/worker-runner/target/pulsar-*.jar 2> /dev/null | grep -v tests | tail -1`
-    if [ -z "${BUILT_JAR}" ]; then
-        echo "\nCouldn't find pulsar jar.";
-        echo "Make sure you've run 'mvn package'\n";
-        exit 1;
-    fi
-    PULSAR_JAR=$BUILT_JAR
-fi
-
-# find the java instance location
-if [ ! -f "${JAVA_INSTANCE_JAR}" ]; then
-    # didn't find a released jar, then search the built jar
-    BUILT_JAVA_INSTANCE_JAR="${PULSAR_HOME}/runtime/target/java-instance.jar"
-    if [ -z "${BUILT_JAVA_INSTANCE_JAR}" ]; then
-        echo "\nCouldn't find pulsar java instance jar.";
-        echo "Make sure you've run 'mvn package'\n";
-        exit 1;
-    fi
-    JAVA_INSTANCE_JAR=${BUILT_JAVA_INSTANCE_JAR}
-fi
-
-# find the python instance location
-if [ ! -f "${PY_INSTANCE_FILE}" ]; then
-    # didn't find a released python instance, then search the built python instance
-    BUILT_PY_INSTANCE_FILE="${PULSAR_HOME}/runtime/target/python-instance/python_instance_main.py"
-    if [ -z "${BUILT_PY_INSTANCE_FILE}" ]; then
-        echo "\nCouldn't find pulsar python instance.";
-        echo "Make sure you've run 'mvn package'\n";
-        exit 1;
-    fi
-    PY_INSTANCE_FILE=${BUILT_PY_INSTANCE_FILE}
-fi
-
-pulsar_help() {
-    cat <<EOF
-Usage: pulsar <command>
-where command is one of:
-    worker              Run a function worker server
-
-    help                This help message
-
-or command is the full name of a class with a defined main() method.
-
-Environment variables:
-   PULSAR_LOG_CONF               Log4j configuration file (default $DEFAULT_LOG_CONF)
-   PULSAR_WORKER_CONF            Configuration file for function worker (default: $DEFAULT_WORKER_CONF)
-   PULSAR_EXTRA_OPTS             Extra options to be passed to the jvm
-   PULSAR_EXTRA_CLASSPATH        Add extra paths to the pulsar classpath
-   PULSAR_PID_DIR                Folder where the pulsar server PID file should be stored
-   PULSAR_STOP_TIMEOUT           Wait time before forcefully kill the pulsar server instance, if the stop is not successful
-   PULSAR_JAVA_INSTANCE_JAR      The java instance jar that pulsar worker uses to run functions (default: $DEFAULT_JAVA_INSTANCE_JAR)
-   PULSAR_PY_INSTANCE_FILE       The python instance that pulsar worker uses to run functions (default: $DEFAULT_PY_INSTANCE_FILE)
-
-These variable can also be set in conf/pulsar_env.sh
-EOF
-}
-
-add_maven_deps_to_classpath() {
-    MVN="mvn"
-    if [ "$MAVEN_HOME" != "" ]; then
-	    MVN=${MAVEN_HOME}/bin/mvn
-    fi
-
-    # Need to generate classpath from maven pom. This is costly so generate it
-    # and cache it. Save the file into our target dir so a mvn clean will get
-    # clean it up and force us create a new one.
-    f="${PULSAR_HOME}/worker-runner/target/classpath_shaded.txt"
-    if [ ! -f "${f}" ]
-    then
-	    ${MVN} -f "${PULSAR_HOME}/worker-runner/pom.xml" dependency:build-classpath -DincludeScope=compile -Dmdep.outputFile="${f}" &> /dev/null
-    fi
-    PULSAR_CLASSPATH=${CLASSPATH}:`cat "${f}"`
-}
-
-if [ -d "$PULSAR_HOME/lib" ]; then
-	PULSAR_CLASSPATH=$PULSAR_CLASSPATH:$PULSAR_HOME/lib/*
-else
-    add_maven_deps_to_classpath
-fi
-
-# if no args specified, show usage
-if [ $# = 0 ]; then
-    pulsar_help;
-    exit 1;
-fi
-
-# get arguments
-COMMAND=$1
-shift
-
-if [ -z "$PULSAR_WORKER_CONF" ]; then
-    PULSAR_WORKER_CONF=$DEFAULT_WORKER_CONF
-fi
-
-if [ -z "$PULSAR_LOG_CONF" ]; then
-    PULSAR_LOG_CONF=$DEFAULT_LOG_CONF
-fi
-
-PULSAR_CLASSPATH="$PULSAR_JAR:$PULSAR_CLASSPATH:$PULSAR_EXTRA_CLASSPATH"
-PULSAR_CLASSPATH="`dirname $PULSAR_LOG_CONF`:$PULSAR_CLASSPATH"
-OPTS="$OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF`"
-
-# Ensure we can read bigger content from ZK. (It might be
-# rarely needed when trying to list many z-nodes under a
-# directory)
-OPTS="$OPTS -Djute.maxbuffer=10485760 -Djava.net.preferIPv4Stack=true"
-
-OPTS="-cp $PULSAR_CLASSPATH $OPTS"
-
-OPTS="$OPTS $PULSAR_EXTRA_OPTS"
-
-# log directory & file
-PULSAR_LOG_DIR=${PULSAR_LOG_DIR:-"$PULSAR_HOME/logs"}
-PULSAR_LOG_APPENDER=${PULSAR_LOG_APPENDER:-"RoutingAppender"}
-PULSAR_LOG_LEVEL=${PULSAR_LOG_LEVEL:-"info"}
-PULSAR_ROUTING_APPENDER_DEFAULT=${PULSAR_ROUTING_APPENDER_DEFAULT:-"Console"}
-
-#Configure log configuration system properties
-OPTS="$OPTS -Dpulsar.log.appender=$PULSAR_LOG_APPENDER"
-OPTS="$OPTS -Dpulsar.log.dir=$PULSAR_LOG_DIR"
-OPTS="$OPTS -Dpulsar.log.level=$PULSAR_LOG_LEVEL"
-OPTS="$OPTS -Dpulsar.routing.appender.default=$PULSAR_ROUTING_APPENDER_DEFAULT"
-OPTS="$OPTS -Dpulsar.functions.process.container.log.dir=$PULSAR_LOG_DIR"
-
-# instance
-OPTS="$OPTS -Dpulsar.functions.java.instance.jar=${JAVA_INSTANCE_JAR}"
-OPTS="$OPTS -Dpulsar.functions.python.instance.file=${PY_INSTANCE_FILE}"
-
-#Change to PULSAR_HOME to support relative paths
-cd "$PULSAR_HOME"
-if [ $COMMAND == "worker" ]; then
-    PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-function-worker.log"}
-    exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.functions.worker.FunctionWorkerStarter -c $PULSAR_WORKER_CONF $@
-elif [ $COMMAND == "help" ]; then
-    pulsar_help;
-else
-    exec $JAVA $OPTS $COMMAND $@
-fi
diff --git a/pulsar-functions/bin/pulsar-functions b/pulsar-functions/bin/pulsar-functions
deleted file mode 100755
index ecd12379e..000000000
--- a/pulsar-functions/bin/pulsar-functions
+++ /dev/null
@@ -1,148 +0,0 @@
-#!/usr/bin/env bash
-#
-# 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.
-#
-
-BINDIR=$(dirname "$0")
-PULSAR_HOME=`cd $BINDIR/..;pwd`
-
-DEFAULT_CLIENT_CONF=$PULSAR_HOME/conf/client.conf
-DEFAULT_LOG_CONF=$PULSAR_HOME/conf/log4j2.yml
-DEFAULT_JAVA_INSTANCE_JAR=$PULSAR_HOME/instances/java-instance.jar
-JAVA_INSTANCE_JAR=${PULSAR_JAVA_INSTANCE_JAR:-"${DEFAULT_JAVA_INSTANCE_JAR}"}
-DEFAULT_PY_INSTANCE_FILE=$PULSAR_HOME/instances/python-instance/python_instance_main.py
-PY_INSTANCE_FILE=${PULSAR_PY_INSTANCE_FILE:-"${DEFAULT_PY_INSTANCE_FILE}"}
-
-if [ -f "$PULSAR_HOME/conf/pulsar_tools_env.sh" ]
-then
-    . "$PULSAR_HOME/conf/pulsar_tools_env.sh"
-fi
-
-# Check for the java to use
-if [[ -z $JAVA_HOME ]]; then
-    JAVA=$(which java)
-    if [ $? != 0 ]; then
-        echo "Error: JAVA_HOME not set, and no java executable found in $PATH." 1>&2
-        exit 1
-    fi
-else
-    JAVA=$JAVA_HOME/bin/java
-fi
-
-# exclude tests jar
-RELEASE_JAR=`ls $PULSAR_HOME/lib/*pulsar-functions-cli*.jar 2> /dev/null | grep -v tests | tail -1`
-if [ $? == 0 ]; then
-    PULSAR_JAR=$RELEASE_JAR
-fi
-
-# exclude tests jar
-if [ -z "$PULSAR_JAR" ]; then
-    BUILT_JAR=`ls $PULSAR_HOME/cli/target/pulsar-*.jar 2> /dev/null | grep -v tests | tail -1`
-    if [ -z "${BUILT_JAR}" ]; then 
-        echo "\nCouldn't find pulsar jar.";
-        echo "Make sure you've run 'mvn package'\n";
-        exit 1;
-    fi
-    PULSAR_JAR=$BUILT_JAR
-fi
-
-add_maven_deps_to_classpath() {
-    MVN="mvn"
-    if [ "$MAVEN_HOME" != "" ]; then
-	MVN=${MAVEN_HOME}/bin/mvn
-    fi
-    
-    # Need to generate classpath from maven pom. This is costly so generate it
-    # and cache it. Save the file into our target dir so a mvn clean will get
-    # clean it up and force us create a new one.
-    f="${PULSAR_HOME}/cli/target/classpath_shaded.txt"
-    if [ ! -f "${f}" ]
-    then
-	${MVN} -f "${PULSAR_HOME}/cli/pom.xml" dependency:build-classpath -DincludeScope=compile -Dmdep.outputFile="${f}" &> /dev/null
-    fi
-    PULSAR_CLASSPATH=${CLASSPATH}:`cat "${f}"`
-}
-
-if [ -d "$PULSAR_HOME/lib" ]; then
-    PULSAR_CLASSPATH="$PULSAR_CLASSPATH:$PULSAR_HOME/lib/*"
-else
-    add_maven_deps_to_classpath
-fi
-
-if [ -z "$PULSAR_CLIENT_CONF" ]; then
-    PULSAR_CLIENT_CONF=$DEFAULT_CLIENT_CONF
-fi
-if [ -z "$PULSAR_LOG_CONF" ]; then
-    PULSAR_LOG_CONF=$DEFAULT_LOG_CONF
-fi
-
-PULSAR_CLASSPATH="$PULSAR_JAR:$PULSAR_CLASSPATH:$PULSAR_EXTRA_CLASSPATH"
-PULSAR_CLASSPATH="`dirname $PULSAR_LOG_CONF`:$PULSAR_CLASSPATH"
-OPTS="$OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF`"
-OPTS="$OPTS -Djava.net.preferIPv4Stack=true"
-
-OPTS="-cp $PULSAR_CLASSPATH $OPTS"
-
-OPTS="$OPTS $PULSAR_EXTRA_OPTS"
-
-# log directory & file
-PULSAR_LOG_DIR=${PULSAR_LOG_DIR:-"$PULSAR_HOME/logs"}
-PULSAR_LOG_APPENDER=${PULSAR_LOG_APPENDER:-"RoutingAppender"}
-PULSAR_LOG_LEVEL=${PULSAR_LOG_LEVEL:-"info"}
-PULSAR_ROUTING_APPENDER_DEFAULT=${PULSAR_ROUTING_APPENDER_DEFAULT:-"Console"}
-
-#Configure log configuration system properties
-OPTS="$OPTS -Dpulsar.log.appender=$PULSAR_LOG_APPENDER"
-OPTS="$OPTS -Dpulsar.log.dir=$PULSAR_LOG_DIR"
-OPTS="$OPTS -Dpulsar.log.level=$PULSAR_LOG_LEVEL"
-OPTS="$OPTS -Dpulsar.routing.appender.default=$PULSAR_ROUTING_APPENDER_DEFAULT"
-
-# find the java instance location
-if [ ! -f "${JAVA_INSTANCE_JAR}" ]; then
-    # didn't find a released jar, then search the built jar
-    BUILT_JAVA_INSTANCE_JAR="${PULSAR_HOME}/runtime/target/java-instance.jar"
-    if [ -f "${BUILT_JAVA_INSTANCE_JAR}" ]; then
-        JAVA_INSTANCE_JAR=${BUILT_JAVA_INSTANCE_JAR}
-    else
-        echo "\nCouldn't find pulsar java instance jar.";
-        echo "Make sure you've run 'mvn package'\n";
-        exit 1;
-    fi
-fi
-
-# find the python instance location
-if [ ! -f "${PY_INSTANCE_FILE}" ]; then
-    # didn't find a released python instance, then search the built python instance
-    BUILT_PY_INSTANCE_FILE="${PULSAR_HOME}/runtime/target/python-instance/python_instance_main.py"
-    if [ -f "${BUILT_PY_INSTANCE_FILE}" ]; then
-        PY_INSTANCE_FILE=${BUILT_PY_INSTANCE_FILE}
-    else
-        echo "\nCouldn't find pulsar python instance.";
-        echo "Make sure you've run 'mvn package'\n";
-        exit 1;
-    fi
-fi
-
-# functions
-OPTS="$OPTS -Dpulsar.functions.java.instance.jar=${JAVA_INSTANCE_JAR}"
-OPTS="$OPTS -Dpulsar.functions.python.instance.file=${PY_INSTANCE_FILE}"
-
-#Change to PULSAR_HOME to support relative paths
-cd "$PULSAR_HOME"
-    
-exec $JAVA $OPTS org.apache.pulsar.admin.cli.FunctionsTool $PULSAR_CLIENT_CONF "$@"
diff --git a/pulsar-functions/conf/client.conf b/pulsar-functions/conf/client.conf
deleted file mode 100644
index fabe68c3e..000000000
--- a/pulsar-functions/conf/client.conf
+++ /dev/null
@@ -1,27 +0,0 @@
-#
-# 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 Client configuration
-webServiceUrl=http://localhost:6750/
-brokerServiceUrl=pulsar://localhost:6650/
-#authPlugin=
-#authParams=
-#useTls=
-#tlsAllowInsecureConnection
-#tlsTrustCertsFilePath
diff --git a/pulsar-functions/conf/function_worker.yml b/pulsar-functions/conf/function_worker.yml
deleted file mode 100644
index 2bc51d175..000000000
--- a/pulsar-functions/conf/function_worker.yml
+++ /dev/null
@@ -1,46 +0,0 @@
-#
-# 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.
-#
-
-workerId: standalone
-workerHostname: localhost
-workerPort: 6750
-functionMetadataTopicName: metadata
-functionMetadataSnapshotsTopicPath: snapshots
-clusterCoordinationTopicName: coordinate
-pulsarFunctionsNamespace: sample/standalone/functions
-pulsarServiceUrl: pulsar://localhost:6650
-pulsarWebServiceUrl: http://localhost:8080
-numFunctionPackageReplicas: 1
-downloadDirectory: /tmp/pulsar_functions
-metricsConfig:
-  metricsSinkClassName: org.apache.pulsar.functions.metrics.sink.PrometheusSink
-  metricsCollectionInterval: 30
-  metricsSinkConfig:
-    path: /metrics
-    port: 9099
-# threadContainerFactory:
-#   threadGroupName: "Thread Function Container Group"
-processContainerFactory:
-  logDirectory:
-  
-schedulerClassName: "org.apache.pulsar.functions.worker.scheduler.RoundRobinScheduler"
-functionAssignmentTopicName: "assignments"
-failureCheckFreqMs: 30000
-rescheduleTimeoutMs: 60000
-initialBrokerReconnectMaxRetries: 60
diff --git a/pulsar-functions/conf/log4j2.yml b/pulsar-functions/conf/log4j2.yml
deleted file mode 100644
index 47b8dcb23..000000000
--- a/pulsar-functions/conf/log4j2.yml
+++ /dev/null
@@ -1,167 +0,0 @@
-#
-# 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.
-#
-
-Configuration:
-  name: pulsar-functions
-  monitorInterval: 30
-
-  Properties:
-    Property:
-      - name: "pulsar.log.dir"
-        value: "logs"
-      - name: "pulsar.log.file"
-        value: "pulsar-functions.log"
-      - name: "pulsar.log.appender"
-        value: "RoutingAppender"
-      - name: "pulsar.log.level"
-        value: "info"
-      - name: "pulsar.routing.appender.default"
-        value: "Console"
-      - name: "bk.log.level"
-        value: "info"
-
-  Appenders:
-
-    # Console
-    Console:
-      name: Console
-      target: SYSTEM_OUT
-      PatternLayout:
-        Pattern: "%d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n"
-
-    # Rolling file appender configuration
-    RollingFile:
-      name: RollingFile
-      fileName: "${sys:pulsar.log.dir}/${sys:pulsar.log.file}"
-      filePattern: "${sys:pulsar.log.dir}/${sys:pulsar.log.file}-%d{MM-dd-yyyy}-%i.log.gz"
-      immediateFlush: true
-      PatternLayout:
-        Pattern: "%d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n"
-      Policies:
-        TimeBasedTriggeringPolicy:
-          interval: 1
-          modulate: true
-        SizeBasedTriggeringPolicy:
-          size: 1 GB
-        # Trigger every day at midnight that also scan
-        # roll-over strategy that deletes older file
-        CronTriggeringPolicy:
-          schedule: "0 0 0 * * ?"
-      # Delete file older than 30days
-      DefaultRolloverStrategy:
-          Delete:
-            basePath: ${sys:pulsar.log.dir}
-            maxDepth: 2
-            IfFileName:
-              glob: "*/${sys:pulsar.log.file}*log.gz"
-            IfLastModified:
-              age: 30d
-
-    # Rolling file appender configuration for bk
-    RollingRandomAccessFile:
-      name: BkRollingFile
-      fileName: "${sys:pulsar.log.dir}/${sys:pulsar.log.file}.bk"
-      filePattern: "${sys:pulsar.log.dir}/${sys:pulsar.log.file}.bk-%d{MM-dd-yyyy}-%i.log.gz"
-      immediateFlush: true
-      PatternLayout:
-        Pattern: "%d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n"
-      Policies:
-        TimeBasedTriggeringPolicy:
-          interval: 1
-          modulate: true
-        SizeBasedTriggeringPolicy:
-          size: 1 GB
-        # Trigger every day at midnight that also scan
-        # roll-over strategy that deletes older file
-        CronTriggeringPolicy:
-          schedule: "0 0 0 * * ?"
-      # Delete file older than 30days
-      DefaultRolloverStrategy:
-          Delete:
-            basePath: ${sys:pulsar.log.dir}
-            maxDepth: 2
-            IfFileName:
-              glob: "*/${sys:pulsar.log.file}.bk*log.gz"
-            IfLastModified:
-              age: 30d
-
-    # Routing
-    Routing:
-      name: RoutingAppender
-      Routes:
-        pattern: "$${ctx:function}"
-        Route:
-          -
-            Routing:
-              name: InstanceRoutingAppender
-              Routes:
-                pattern: "$${ctx:instance}"
-                Route:
-                  -
-                    RollingFile:
-                      name: "Rolling-${ctx:function}"
-                      fileName : "${sys:pulsar.log.dir}/functions/${ctx:function}/function.log"
-                      filePattern : "${sys:pulsar.log.dir}/functions/${ctx:function}-%d{MM-dd-yyyy}-%i.log.gz"
-                      PatternLayout:
-                        Pattern: "%d{ABSOLUTE} %level{length=5} [%thread] [instance: %X{instance}] %logger{1} - %msg%n"
-                      Policies:
-                        TimeBasedTriggeringPolicy:
-                          interval: 1
-                          modulate: true
-                        SizeBasedTriggeringPolicy:
-                          size: "20MB"
-                        # Trigger every day at midnight that also scan
-                        # roll-over strategy that deletes older file
-                        CronTriggeringPolicy:
-                          schedule: "0 0 0 * * ?"
-                      # Delete file older than 30days
-                      DefaultRolloverStrategy:
-                          Delete:
-                            basePath: ${sys:pulsar.log.dir}
-                            maxDepth: 2
-                            IfFileName:
-                              glob: "*/${sys:pulsar.log.file}*log.gz"
-                            IfLastModified:
-                              age: 30d
-                  - ref: "${sys:pulsar.routing.appender.default}"
-                    key: "${ctx:function}"
-          - ref: "${sys:pulsar.routing.appender.default}"
-            key: "${ctx:function}"
-
-  Loggers:
-
-    Logger:
-      name: org.apache.bookkeeper
-      level: "${sys:bk.log.level}"
-      additivity: false
-      AppenderRef:
-        - ref: BkRollingFile
-
-    Logger:
-      name: org.apache.distributedlog
-      level: "${sys:bk.log.level}"
-      additivity: false
-      AppenderRef:
-        - ref: BkRollingFile
-
-    Root:
-      level: info
-      AppenderRef:
-        - ref: "${sys:pulsar.log.appender}"
-          level: "${sys:pulsar.log.level}"
diff --git a/pulsar-functions/conf/pulsar_env.sh b/pulsar-functions/conf/pulsar_env.sh
deleted file mode 100644
index f600e6d20..000000000
--- a/pulsar-functions/conf/pulsar_env.sh
+++ /dev/null
@@ -1,60 +0,0 @@
-#!/usr/bin/env bash
-#
-# 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.
-#
-
-# Set JAVA_HOME here to override the environment setting
-# JAVA_HOME=
-
-# default settings for starting pulsar broker
-
-# Log4j configuration file
-# PULSAR_LOG_CONF=
-
-# Logs location
-# PULSAR_LOG_DIR=
-
-# Configuration file of settings used in broker server
-# PULSAR_BROKER_CONF=
-
-# Configuration file of settings used in bookie server
-# PULSAR_BOOKKEEPER_CONF=
-
-# Configuration file of settings used in zookeeper server
-# PULSAR_ZK_CONF=
-
-# Configuration file of settings used in global zookeeper server
-# PULSAR_GLOBAL_ZK_CONF=
-
-# Extra options to be passed to the jvm
-PULSAR_MEM=" -Xms2g -Xmx2g -XX:MaxDirectMemorySize=4g"
-
-# Garbage collection options
-PULSAR_GC=" -XX:+UseG1GC -XX:MaxGCPauseMillis=10 -XX:+ParallelRefProcEnabled -XX:+UnlockExperimentalVMOptions -XX:+AggressiveOpts -XX:+DoEscapeAnalysis -XX:ParallelGCThreads=32 -XX:ConcGCThreads=32 -XX:G1NewSizePercent=50 -XX:+DisableExplicitGC -XX:-ResizePLAB"
-
-# Extra options to be passed to the jvm
-PULSAR_EXTRA_OPTS="${PULSAR_EXTRA_OPTS} ${PULSAR_MEM} ${PULSAR_GC} -Dio.netty.leakDetectionLevel=disabled -Dio.netty.recycler.maxCapacity.default=1000 -Dio.netty.recycler.linkCapacity=1024"
-
-# Add extra paths to the bookkeeper classpath
-# PULSAR_EXTRA_CLASSPATH=
-
-#Folder where the Bookie server PID file should be stored
-#PULSAR_PID_DIR=
-
-#Wait time before forcefully kill the pulser server instance, if the stop is not successful
-#PULSAR_STOP_TIMEOUT=
diff --git a/pulsar-functions/conf/pulsar_tools_env.sh b/pulsar-functions/conf/pulsar_tools_env.sh
deleted file mode 100755
index 7eca6071d..000000000
--- a/pulsar-functions/conf/pulsar_tools_env.sh
+++ /dev/null
@@ -1,60 +0,0 @@
-#!/usr/bin/env bash
-#
-# 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.
-#
-
-# Set JAVA_HOME here to override the environment setting
-# JAVA_HOME=
-
-# default settings for starting pulsar broker
-
-# Log4j configuration file
-# PULSAR_LOG_CONF=
-
-# Logs location
-# PULSAR_LOG_DIR=
-
-# Configuration file of settings used in broker server
-# PULSAR_BROKER_CONF=
-
-# Configuration file of settings used in bookie server
-# PULSAR_BOOKKEEPER_CONF=
-
-# Configuration file of settings used in zookeeper server
-# PULSAR_ZK_CONF=
-
-# Configuration file of settings used in global zookeeper server
-# PULSAR_GLOBAL_ZK_CONF=
-
-# Extra options to be passed to the jvm
-PULSAR_MEM=" -Xmx256m -XX:MaxDirectMemorySize=256m"
-
-# Garbage collection options
-PULSAR_GC=" -client "
-
-# Extra options to be passed to the jvm
-PULSAR_EXTRA_OPTS="${PULSAR_EXTRA_OPTS} ${PULSAR_MEM} ${PULSAR_GC} -Dio.netty.leakDetectionLevel=disabled"
-
-# Add extra paths to the bookkeeper classpath
-# PULSAR_EXTRA_CLASSPATH=
-
-#Folder where the Bookie server PID file should be stored
-#PULSAR_PID_DIR=
-
-#Wait time before forcefully kill the pulser server instance, if the stop is not successful
-#PULSAR_STOP_TIMEOUT=
diff --git a/pulsar-functions/dist/pom.xml b/pulsar-functions/dist/pom.xml
deleted file mode 100644
index 35066c23d..000000000
--- a/pulsar-functions/dist/pom.xml
+++ /dev/null
@@ -1,103 +0,0 @@
-<!--
-
-    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.
-
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-
-  <parent>
-    <groupId>org.apache.pulsar</groupId>
-    <artifactId>pulsar-functions</artifactId>
-    <version>2.0.0-incubating-SNAPSHOT</version>
-    <relativePath>..</relativePath>
-  </parent>
-
-  <artifactId>pulsar-functions-dist</artifactId>
-  <name>Pulsar Functions :: Distribution</name>
-  <packaging>pom</packaging>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.pulsar</groupId>
-      <artifactId>pulsar-functions-worker</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.pulsar</groupId>
-      <artifactId>pulsar-functions-api-examples</artifactId>
-      <version>${project.version}</version>
-      <!-- make sure the api examples are compiled before assembly -->
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-all</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.logging.log4j</groupId>
-      <artifactId>log4j-slf4j-impl</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.logging.log4j</groupId>
-      <artifactId>log4j-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.logging.log4j</groupId>
-      <artifactId>log4j-core</artifactId>
-    </dependency>
-
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <artifactId>maven-assembly-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>functions-dist-assembly</id>
-            <phase>package</phase>
-            <goals>
-              <goal>single</goal>
-            </goals>
-            <configuration>
-              <attach>true</attach>
-              <tarLongFileMode>posix</tarLongFileMode>
-              <finalName>apache-pulsar-functions-${project.version}</finalName>
-              <descriptors>
-                <descriptor>src/assemble/bin.xml</descriptor>
-                <descriptor>src/assemble/src.xml</descriptor>
-              </descriptors>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-deploy-plugin</artifactId>
-        <configuration>
-          <skip>true</skip>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-</project>
diff --git a/pulsar-functions/dist/src/assemble/bin.xml b/pulsar-functions/dist/src/assemble/bin.xml
deleted file mode 100644
index b4def906e..000000000
--- a/pulsar-functions/dist/src/assemble/bin.xml
+++ /dev/null
@@ -1,83 +0,0 @@
-<!--
-
-    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.
-
--->
-<assembly
-  xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
-  <id>bin</id>
-  <formats>
-    <format>tar.gz</format>
-  </formats>
-  <includeBaseDirectory>true</includeBaseDirectory>
-  <moduleSets>
-    <moduleSet>
-      <useAllReactorProjects>true</useAllReactorProjects>
-    </moduleSet>
-  </moduleSets>
-  <fileSets>
-    <fileSet>
-      <directory>${basedir}/../conf</directory>
-    </fileSet>
-    <fileSet>
-      <directory>${basedir}/../bin</directory>
-      <fileMode>755</fileMode>
-    </fileSet>
-    <fileSet>
-      <directory>${basedir}/../runtime/target/python-instance</directory>
-      <outputDirectory>instances/python-instance</outputDirectory>
-    </fileSet>
-  </fileSets>
-  <files>
-    <file>
-      <source>${basedir}/../runtime/target/java-instance.jar</source>
-      <destName>java-instance.jar</destName>
-      <outputDirectory>instances</outputDirectory>
-    </file>
-    <file>
-      <source>${basedir}/../java-examples/target/pulsar-functions-api-examples.jar</source>
-      <destName>api-examples.jar</destName>
-      <outputDirectory>examples</outputDirectory>
-    </file>
-  </files>
-  <dependencySets>
-    <dependencySet>
-      <outputDirectory>lib</outputDirectory>
-      <unpack>false</unpack>
-      <scope>compile</scope>
-      <useProjectArtifact>false</useProjectArtifact>
-      <!-- Include 'groupId' in the dependencies Jar names to better identify
-           the provenance of the jar -->
-      <outputFileNameMapping>${artifact.groupId}-${artifact.artifactId}-${artifact.version}${dashClassifier?}.${artifact.extension}</outputFileNameMapping>
-
-      <excludes>
-        <!-- All these dependencies are already included in netty-all -->
-        <exclude>io.netty:netty-common</exclude>
-        <exclude>io.netty:netty-resolver</exclude>
-        <exclude>io.netty:netty-buffer</exclude>
-        <exclude>io.netty:netty-codec-http</exclude>
-        <exclude>io.netty:netty-codec</exclude>
-        <exclude>io.netty:netty-transport</exclude>
-        <exclude>io.netty:netty-handler</exclude>
-        <exclude>io.netty:netty-codec-http</exclude>
-      </excludes>
-    </dependencySet>
-  </dependencySets>
-</assembly>
diff --git a/pulsar-functions/dist/src/assemble/src.xml b/pulsar-functions/dist/src/assemble/src.xml
deleted file mode 100644
index 0c0526358..000000000
--- a/pulsar-functions/dist/src/assemble/src.xml
+++ /dev/null
@@ -1,92 +0,0 @@
-<!--
-
-    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.
-
--->
-<assembly
-  xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
-  <id>src</id>
-  <formats>
-    <format>tar.gz</format>
-  </formats>
-  <includeBaseDirectory>true</includeBaseDirectory>
-  <fileSets>
-    <fileSet>
-      <directory>..</directory>
-      <useDefaultExcludes>true</useDefaultExcludes>
-      <includes>
-        <include>**/README.md</include>
-        <include>**/DISCLAIMER</include>
-        <include>**/LICENSE</include>
-        <include>**/NOTICE</include>
-        <include>**/pom.xml</include>
-        <include>**/src/**</include>
-        <include>**/pulsar-client-cpp/**</include>
-        <include>**/conf/**</include>
-        <include>**/bin/**</include>
-        <include>**/*.txt</include>
-        <include>docker/**</include>
-        <include>dashboard/**</include>
-        <include>deployment/**</include>
-      </includes>
-      <excludes>
-        <exclude>.git/**</exclude>
-        <exclude>**/.gitignore</exclude>
-        <exclude>**/.svn</exclude>
-        <exclude>**/*.iws</exclude>
-        <exclude>**/*.ipr</exclude>
-        <exclude>**/*.iml</exclude>
-        <exclude>**/*.cbp</exclude>
-        <exclude>**/*.pyc</exclude>
-        <exclude>**/.classpath</exclude>
-        <exclude>**/.project</exclude>
-        <exclude>**/.settings</exclude>
-        <exclude>**/target/**</exclude>
-        <exclude>**/CMakeFiles/**</exclude>
-        <exclude>**/CMakeCache.txt</exclude>
-        <exclude>**/cmake_install.cmake</exclude>
-        <exclude>pulsar-client-cpp/**/Makefile</exclude>
-        <exclude>pulsar-client-cpp/tests/main</exclude>
-        <exclude>pulsar-client-cpp/examples/SampleAsyncProducer</exclude>
-        <exclude>pulsar-client-cpp/examples/SampleConsumer</exclude>
-        <exclude>pulsar-client-cpp/examples/SampleConsumerListener</exclude>
-        <exclude>pulsar-client-cpp/examples/SampleProducer</exclude>
-        <exclude>pulsar-client-cpp/perf/perfProducer</exclude>
-        <exclude>pulsar-client-cpp/perf/perfConsumer</exclude>
-
-        <exclude>**/python/dist/**</exclude>
-        <exclude>**/python/wheelhouse/**</exclude>
-        <exclude>**/python/MANIFEST</exclude>
-        <exclude>**/*.egg-info/**</exclude>
-        <!-- until the code that does this is fixed -->
-        <exclude>**/*.log</exclude>
-        <exclude>**/build/**</exclude>
-        <exclude>**/file:/**</exclude>
-        <exclude>**/SecurityAuth.audit*</exclude>
-        <exclude>**/site/**</exclude>
-        <exclude>**/.idea/**</exclude>
-        <exclude>**/*.a</exclude>
-        <exclude>**/*.so</exclude>
-        <exclude>**/*.so.*</exclude>
-        <exclude>**/*.dylib</exclude>
-      </excludes>
-    </fileSet>
-  </fileSets>
-</assembly>
diff --git a/pulsar-functions/pom.xml b/pulsar-functions/pom.xml
index 5aacec56f..829f080ee 100644
--- a/pulsar-functions/pom.xml
+++ b/pulsar-functions/pom.xml
@@ -40,9 +40,7 @@
     <module>runtime</module>
     <module>worker</module>
     <module>worker-shaded</module>
-    <module>worker-runner</module>
     <module>java-examples</module>
-    <module>dist</module>
   </modules>
 
   <dependencyManagement>
diff --git a/pulsar-functions/worker-runner/pom.xml b/pulsar-functions/worker-runner/pom.xml
deleted file mode 100644
index 1e621d5cc..000000000
--- a/pulsar-functions/worker-runner/pom.xml
+++ /dev/null
@@ -1,111 +0,0 @@
-<?xml version="1.0"?>
-<!--
-
-    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.
-
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-
-  <parent>
-    <groupId>org.apache.pulsar</groupId>
-    <artifactId>pulsar-functions</artifactId>
-    <version>2.0.0-incubating-SNAPSHOT</version>
-    <relativePath>..</relativePath>
-  </parent>
-
-  <artifactId>pulsar-functions-worker-runner</artifactId>
-  <name>Pulsar Functions :: Worker Runner</name>
-
-  <dependencies>
-
-    <dependency>
-      <groupId>${project.groupId}</groupId>
-      <artifactId>pulsar-functions-worker-shaded</artifactId>
-      <version>${project.parent.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>com.google.protobuf</groupId>
-          <artifactId>protobuf-lite</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.google.protobuf</groupId>
-          <artifactId>protobuf-java</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.google.protobuf.nano</groupId>
-          <artifactId>protobuf-javanano</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.google.protobuf</groupId>
-          <artifactId>protobuf-java-util</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>io.grpc</groupId>
-          <artifactId>grpc-protobuf</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>io.grpc</groupId>
-          <artifactId>grpc-protobuf-lite</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>io.grpc</groupId>
-          <artifactId>grpc-protobuf-nano</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.pulsar</groupId>
-          <artifactId>pulsar-functions-proto</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.pulsar</groupId>
-          <artifactId>pulsar-functions-utils</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.pulsar</groupId>
-          <artifactId>pulsar-functions-metrics</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.pulsar</groupId>
-          <artifactId>pulsar-functions-runtime</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.pulsar</groupId>
-          <artifactId>pulsar-functions-instance</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.pulsar</groupId>
-          <artifactId>pulsar-functions-worker</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>${project.groupId}</groupId>
-      <artifactId>pulsar-client-original</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>${project.groupId}</groupId>
-      <artifactId>pulsar-client-admin-original</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-
-  </dependencies>
-</project>


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services