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 2021/06/26 13:38:56 UTC

[GitHub] [pulsar] wuchenxi123 opened a new pull request #11112: pulsar-function:Preload and release of external resources

wuchenxi123 opened a new pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112


   <!--
   ### Contribution Checklist
     
     - Name the pull request in the form "[Issue XYZ][component] Title of the pull request", where *XYZ* should be replaced by the actual issue number.
       Skip *Issue XYZ* if there is no associated github issue for this pull request.
       Skip *component* if you are unsure about which is the best component. E.g. `[docs] Fix typo in produce method`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   *(If this PR fixes a github issue, please add `Fixes #<xyz>`.)*
   
   Fixes #<xyz>
   
   *(or if this PR is one task of a github issue, please add `Master Issue: #<xyz>` to link to the master issue.)*
   
   Master Issue: #<xyz>
   
   ### Motivation
   
   1. Before Function is started, some resources only need to be initialized once, and there is no need to make various judgments in the process() method of the Function interface
   2. After the Function is closed, in the process of using process(), the referenced external resources need to be manually closed(), and they need to be released separately in the close() of javaInstance
   
   ### Modifications
   1. Before starting, some resources only need to be loaded once
   2. When Function Instance manually releases some resources, users need to implement the release method and provide support
   
   @Slf4j
   public class HelloWorldFunction implements Function<String, Void> {
   	@Override
   	public String process(byte[] input, Context context) {
   	}
   }
   eg:
   @Slf4j
   public class HelloWorldFunction implements Function<String, Void>,Hook {
   	@Override
   	public String process(byte[] input, Context context) {
   		
   	}
   	/**
   		resource init
   	*/
   	@Override
   	public void preProcess(Context context) throws Exception {
   		database Connection
   		other resource initialize
   	}
   	/**
   		resource close
   	*/
   	@Override
   	public void postProcess() {
   		connection close
   		call the callback function
   	}
   }
   1. Provide an interface for initialization and unified release of external resources for users to use, easy to manage and expand
   2. The order of the release of external resources is controlled by the pulsar function. Users only need to pay attention to the event processing logic. The code is clearer and more concise, and the hierarchy is clear
   
   ### Verifying this change
   
   - [ ] Make sure that the change passes the CI checks.
   
   *(Please pick either of the following options)*
   
   This change is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This change is already covered by existing tests, such as *(please describe tests)*.
   
   *(or)*
   
   This change added tests and can be verified as follows:
   yes
   
   *(example:)*
     - The initialization and release functions have been applied and verified in the online project
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): (yes / no)
     - The public API: (yes / no) yes
     - The schema: (yes / no / don't know)
     - The default values of configurations: (yes / no)
     - The wire protocol: (yes / no)
     - The rest endpoints: (yes / no)
     - The admin cli options: (yes / no)
     - Anything that affects deployment: (yes / no / don't know)
   
   ### Documentation
   
     - Does this pull request introduce a new feature? (yes / no) yes
     - If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)
     - If a feature is not applicable for documentation, explain why?
     - If a feature is not documented yet in this PR, please create a followup issue for adding the documentation
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wangjialing218 commented on pull request #11112: [PIP 86][Function] Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wangjialing218 commented on pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#issuecomment-939213911


   @BewareMyPower @nlu90 @jerrypeng @eolivelli PTAL when you have time since Pulsar 2.9.0 release cut is coming.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 closed pull request #11112: [PIP 86][Function] Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 closed pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 commented on a change in pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 commented on a change in pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#discussion_r664972759



##########
File path: pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java
##########
@@ -158,9 +177,25 @@ private void processAsyncResults(BiConsumer<Record, JavaExecutionResult> resultC
     }
 
     @Override
-    public void close() {
+    public void close() throws Exception {
         context.close();
         executor.shutdown();
+        if (null != function && function instanceof HookFunction) {
+            try {
+                ((HookFunction) function).cleanup();
+            } catch (Exception e) {
+                log.error("function closeResource occurred exception", e);
+                throw e;
+            }
+        }
+        if (null != javaUtilFunction && javaUtilFunction instanceof HookFunction) {
+            try {
+                ((HookFunction) javaUtilFunction).cleanup();
+            } catch (Exception e) {

Review comment:
       I think HookFunction can provide users with additional choices, and it also makes the responsibilities of each method of Function more single and clear, please think again




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] nlu90 commented on pull request #11112: [PIP 86][Function] Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
nlu90 commented on pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#issuecomment-981937380


   @jerrypeng @eolivelli Should we resume the discussion of this useful feature?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] nlu90 commented on pull request #11112: [PIP 86][Function] Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
nlu90 commented on pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#issuecomment-982138242


   @wuchenxi123 Do you want to provide some update to Enrico's above questions?
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 commented on a change in pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 commented on a change in pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#discussion_r663489945



##########
File path: pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Hook.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.api;
+
+import org.apache.pulsar.common.classification.InterfaceAudience;
+
+/**
+ * An interface for hook
+ * Initial and close of external resources
+ */
+@InterfaceAudience.Public
+public interface Hook {

Review comment:
       I have modified it, please check




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 edited a comment on pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 edited a comment on pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#issuecomment-869162324


   >Could you explain why did you swallow the exception in close() while fail fast in setup()? For your case, preProcess should not throw a checked exception.
   >
   >In addition, IMO it's not a good practice to just use Exception as the checked exception but not the subclass of Exception.
   
   Accept your suggestion, I have corrected your problem
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 commented on a change in pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 commented on a change in pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#discussion_r659874676



##########
File path: pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Hook.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.api;
+
+import org.apache.pulsar.common.classification.InterfaceAudience;
+
+/**
+ * An interface for hook
+ * Preload and release of external resources
+ */
+@InterfaceAudience.Public
+public interface Hook {
+
+    /**
+     * Pre-Process Function Hook
+     *
+     * @throws Exception
+     */
+    void preProcess(Context context) throws RuntimeException;

Review comment:
       I have corrected it




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 commented on pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 commented on pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#issuecomment-869162324


   > 你能解释一下为什么你在`close()`while fail fast in 中吞下了异常`setup()`吗?对于您的情况,`preProcess`不应抛出已检查的异常。
   > 
   > 此外,IMO 仅用`Exception`作已检查异常而不是`Exception`.
   
   
   
   > Could you explain why did you swallow the exception in `close()` while fail fast in `setup()`? For your case, `preProcess` should not throw a checked exception.
   > 
   > In addition, IMO it's not a good practice to just use `Exception` as the checked exception but not the subclass of `Exception`.
   
   Accept your suggestion, I have corrected your problem
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 commented on a change in pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 commented on a change in pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#discussion_r659875261



##########
File path: pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java
##########
@@ -74,6 +75,24 @@ public JavaInstance(ContextImpl contextImpl, Object userClassObject, InstanceCon
         }
     }
 
+    public void setup() {
+        if (null != function && function instanceof Hook) {
+            try {
+                ((Hook) function).preProcess(context);
+            } catch (RuntimeException e) {
+                log.error("setup error:", e);
+                throw new RuntimeException("function preProcess occurred exception", e);
+            }
+        }
+        if (null != javaUtilFunction && javaUtilFunction instanceof Hook) {

Review comment:
       I have corrected it




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #11112: [PIP 86][Function] Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#discussion_r758711420



##########
File path: pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java
##########
@@ -158,9 +177,25 @@ private void processAsyncResults(BiConsumer<Record, JavaExecutionResult> resultC
     }
 
     @Override
-    public void close() {
+    public void close() throws Exception {
         context.close();
         executor.shutdown();
+        if (null != function && function instanceof RichFunction) {

Review comment:
       Actually you don't need the null check as null is never a instance of a class

##########
File path: pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/RichFunction.java
##########
@@ -0,0 +1,45 @@
+/**
+ * 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.api;
+
+import org.apache.pulsar.common.classification.InterfaceAudience;
+
+/**
+ * An interface for richFunction
+ * Extension for Function interface
+ * Initial and close of external resources
+ */
+@InterfaceAudience.Public
+public interface RichFunction extends Function{
+
+    /**
+     * InitialResource Function Hook
+     *
+     * @throws Exception
+     */
+    void setup() throws Exception;

Review comment:
       I would pass some Context in order to let the Function initialise using the information about the env.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 commented on pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 commented on pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#issuecomment-869770568


   > You may misunderstand my comment.
   > 
   > > IMO it's not a good practice to just use Exception as the checked exception but not the subclass of Exception.
   > 
   > I mean use a derived class of `Exception` as the checked exception. You replaced `Exception` with `RuntimeException`, which is worse because `RuntimeException` is an unchecked exception. Take `org.apache.pulsar.metadata.api.MetadataSerde` as an example:
   > 
   > ```java
   > public interface MetadataSerde<T> {
   > 
   >     byte[] serialize(T value) throws IOException;
   > 
   >     T deserialize(byte[] content) throws IOException;
   > }
   > ```
   > 
   > It checks `IOException` that derives from `Exception`. You can also some many other interfaces in Pulsar that checks `PulsarAdminException` or `PulsarClientException`, not `Exception` directly.
   > 
   > Though IMO it's a better practice, using `Exception` directly is also okay but don't use `RuntimeException` as the checked exception.
   
   I have modified it, please check


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 commented on pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 commented on pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#issuecomment-877573226


   > Some of my thoughts:
   > 
   > 1. If we are going to introduce new methods or stages in the lifecycle of a function such as "open" and "close",  I would recommend we follow what we already have for Pulsar IO sources and Sink so that we are consistent.
   > 2. I am also not a particular fan of the name "HookFunction".  Can we call it "RichFunction"?  This naming is consistent with other projects such as Apache Storm and Flink.
   > 3. In regards to the topic
   > 
   > If this rule must be followed, then one possible way is we define a separate `HookFunction` interface which includes `setup`, `process`, `tearDown` methods, and add support in the new runtime to handle this interface. Just as currently the runtime handles `pulsar.Function` and `java.util.Function` separately. The old runtime will not be able to recognize the new interface and thus won't execute functions which need initialization.
   > 
   > I would rather not define a completely new interface. Ideally we have a "RichFunction" interface that extends the existing. So that we can minimize interface duplication and if we want to introduce new interfaces in the future we can build on top of that. In regards to the issue about the old runtime executing a function that implements the new interface, is it sufficient to document somewhere that the worker has to be updated to a certain version to run the functions that implement the new interface?
   
   I agree with your point of view, and made the corresponding changes, please codereview


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] nlu90 commented on pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
nlu90 commented on pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#issuecomment-874993996


   Sorry, I accidentally add the comment with `approve`. 
   
   Please let me know if you update the PR and I'll review again.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#issuecomment-874722368


   > I have sent an email, please check
   
   thanks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] BewareMyPower commented on pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#issuecomment-869261030


   You may misunderstand my comment.
   
   > IMO it's not a good practice to just use Exception as the checked exception but not the subclass of Exception.
   
   I mean use a derived class of `Exception` as the checked exception. You replaced `Exception` with `RuntimeException`, which is worse because `RuntimeException` is an unchecked exception. Take `org.apache.pulsar.metadata.api.MetadataSerde` as an example:
   
   ```java
   public interface MetadataSerde<T> {
   
       byte[] serialize(T value) throws IOException;
   
       T deserialize(byte[] content) throws IOException;
   }
   ```
   
   It checks `IOException` that derives from `Exception`. You can also some many other interfaces in Pulsar that checks `PulsarAdminException` or `PulsarClientException`, not `Exception` directly.
   
   Though IMO it's a better practice, using `Exception` directly is also okay but don't use `RuntimeException` as the checked exception.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 commented on a change in pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 commented on a change in pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#discussion_r659874295



##########
File path: pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java
##########
@@ -160,7 +179,22 @@ private void processAsyncResults(BiConsumer<Record, JavaExecutionResult> resultC
     @Override
     public void close() {
         context.close();
-        executor.shutdown();

Review comment:
       I have corrected it

##########
File path: pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Hook.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.api;
+
+import org.apache.pulsar.common.classification.InterfaceAudience;
+
+/**
+ * An interface for hook
+ * Preload and release of external resources
+ */
+@InterfaceAudience.Public
+public interface Hook {
+
+    /**
+     * Pre-Process Function Hook
+     *
+     * @throws Exception
+     */
+    void preProcess(Context context) throws RuntimeException;

Review comment:
       I have corrected it

##########
File path: pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java
##########
@@ -74,6 +75,24 @@ public JavaInstance(ContextImpl contextImpl, Object userClassObject, InstanceCon
         }
     }
 
+    public void setup() {
+        if (null != function && function instanceof Hook) {
+            try {
+                ((Hook) function).preProcess(context);
+            } catch (RuntimeException e) {
+                log.error("setup error:", e);
+                throw new RuntimeException("function preProcess occurred exception", e);
+            }
+        }
+        if (null != javaUtilFunction && javaUtilFunction instanceof Hook) {

Review comment:
       I have corrected it




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 commented on pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 commented on pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#issuecomment-869770568


   > You may misunderstand my comment.
   > 
   > > IMO it's not a good practice to just use Exception as the checked exception but not the subclass of Exception.
   > 
   > I mean use a derived class of `Exception` as the checked exception. You replaced `Exception` with `RuntimeException`, which is worse because `RuntimeException` is an unchecked exception. Take `org.apache.pulsar.metadata.api.MetadataSerde` as an example:
   > 
   > ```java
   > public interface MetadataSerde<T> {
   > 
   >     byte[] serialize(T value) throws IOException;
   > 
   >     T deserialize(byte[] content) throws IOException;
   > }
   > ```
   > 
   > It checks `IOException` that derives from `Exception`. You can also some many other interfaces in Pulsar that checks `PulsarAdminException` or `PulsarClientException`, not `Exception` directly.
   > 
   > Though IMO it's a better practice, using `Exception` directly is also okay but don't use `RuntimeException` as the checked exception.
   
   I have modified it, please check


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] freeznet commented on a change in pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
freeznet commented on a change in pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#discussion_r659331813



##########
File path: pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Hook.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.api;
+
+import org.apache.pulsar.common.classification.InterfaceAudience;
+
+/**
+ * An interface for hook
+ * Preload and release of external resources
+ */
+@InterfaceAudience.Public
+public interface Hook {
+
+    /**
+     * Pre-Process Function Hook
+     *
+     * @throws Exception
+     */
+    void preProcess(Context context) throws RuntimeException;

Review comment:
       `preProcess` and `postProcess` is likely to be invoked with each message each time before and after `process`, so the naming is kind confuse from my understanding. 

##########
File path: pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java
##########
@@ -74,6 +75,24 @@ public JavaInstance(ContextImpl contextImpl, Object userClassObject, InstanceCon
         }
     }
 
+    public void setup() {
+        if (null != function && function instanceof Hook) {
+            try {
+                ((Hook) function).preProcess(context);
+            } catch (RuntimeException e) {
+                log.error("setup error:", e);
+                throw new RuntimeException("function preProcess occurred exception", e);
+            }
+        }
+        if (null != javaUtilFunction && javaUtilFunction instanceof Hook) {

Review comment:
       according to the [docs](http://pulsar.apache.org/docs/en/next/functions-package/#java), `java.util.function.Function` is not intend to interact with `Context`, so we should avoid to passing `Context` to `java.util.function.Function`.

##########
File path: pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java
##########
@@ -160,7 +179,22 @@ private void processAsyncResults(BiConsumer<Record, JavaExecutionResult> resultC
     @Override
     public void close() {
         context.close();
-        executor.shutdown();

Review comment:
       why remove this line?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 edited a comment on pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 edited a comment on pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#issuecomment-869162324


   >Could you explain why did you swallow the exception in close() while fail fast in setup()? For your case, preProcess should not throw a checked exception.
   >
   >In addition, IMO it's not a good practice to just use Exception as the checked exception but not the subclass of Exception.
   
   Accept your suggestion, I have corrected your problem.Please codereview
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 commented on pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 commented on pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#issuecomment-874721000


   > I find this feature very useful.
   > 
   > Can you please start aa discussion on [dev@pulsar.apache.org](mailto:dev@pulsar.apache.org) in order to show your needs and how you want to address them?
   > 
   > The mailing list is the place in which the community discusses about new features
   
   I have sent an email, please check


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 edited a comment on pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 edited a comment on pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#issuecomment-869162324


   >Could you explain why did you swallow the exception in close() while fail fast in setup()? For your case, preProcess should not throw a checked exception.
   >
   >In addition, IMO it's not a good practice to just use Exception as the checked exception but not the subclass of Exception.
   
   
   
   > Could you explain why did you swallow the exception in `close()` while fail fast in `setup()`? For your case, `preProcess` should not throw a checked exception.
   > 
   > In addition, IMO it's not a good practice to just use `Exception` as the checked exception but not the subclass of `Exception`.
   
   Accept your suggestion, I have corrected your problem
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] wuchenxi123 commented on a change in pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
wuchenxi123 commented on a change in pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#discussion_r659874295



##########
File path: pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java
##########
@@ -160,7 +179,22 @@ private void processAsyncResults(BiConsumer<Record, JavaExecutionResult> resultC
     @Override
     public void close() {
         context.close();
-        executor.shutdown();

Review comment:
       I have corrected it




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] nlu90 commented on a change in pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
nlu90 commented on a change in pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#discussion_r664791831



##########
File path: pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java
##########
@@ -158,9 +177,25 @@ private void processAsyncResults(BiConsumer<Record, JavaExecutionResult> resultC
     }
 
     @Override
-    public void close() {
+    public void close() throws Exception {
         context.close();
         executor.shutdown();
+        if (null != function && function instanceof HookFunction) {
+            try {
+                ((HookFunction) function).cleanup();
+            } catch (Exception e) {
+                log.error("function closeResource occurred exception", e);
+                throw e;
+            }
+        }
+        if (null != javaUtilFunction && javaUtilFunction instanceof HookFunction) {
+            try {
+                ((HookFunction) javaUtilFunction).cleanup();
+            } catch (Exception e) {

Review comment:
       I suggest we keep `java.util.Function` as simple as possible, and not add hook method for it

##########
File path: pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java
##########
@@ -74,6 +75,24 @@ public JavaInstance(ContextImpl contextImpl, Object userClassObject, InstanceCon
         }
     }
 
+    public void setup() throws Exception {
+        if (null != function && function instanceof HookFunction) {
+            try {
+                ((HookFunction) function).setup();
+            } catch (Exception e) {
+                log.error("setup error:", e);
+                throw e;
+            }
+        }
+        if (null != javaUtilFunction && javaUtilFunction instanceof HookFunction) {
+            try {
+                ((HookFunction) javaUtilFunction).setup();
+            } catch (Exception e) {
+                log.error("setup error:", e);
+                throw e;
+            }
+        }

Review comment:
       I suggest we keep `java.util.Function` as simple as possible, and not add hook method for it




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] nlu90 commented on a change in pull request #11112: pulsar-function:Preload and release of external resources

Posted by GitBox <gi...@apache.org>.
nlu90 commented on a change in pull request #11112:
URL: https://github.com/apache/pulsar/pull/11112#discussion_r663251253



##########
File path: pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Hook.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.api;
+
+import org.apache.pulsar.common.classification.InterfaceAudience;
+
+/**
+ * An interface for hook
+ * Initial and close of external resources
+ */
+@InterfaceAudience.Public
+public interface Hook {

Review comment:
       I suggest this new api extends the `Function` interface since it's strongly coupled with the `Function` api and should not be used individually in other cases.
   
   ```
   public interface HookFunction extends Function {
       
       void setup();
   
       void cleanup();
   }
   ```
   
   And we should keep `java.util.Function` as it is to keep its simplicity.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org