You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by "xu1009 (via GitHub)" <gi...@apache.org> on 2023/04/14 07:00:52 UTC

[GitHub] [skywalking-java] xu1009 opened a new pull request, #505: Feature/support forkjoinpool

xu1009 opened a new pull request, #505:
URL: https://github.com/apache/skywalking-java/pull/505

   
   - [ ] If this is non-trivial feature, paste the links/URLs to the design doc.
   - [ ] Update the documentation to include this new feature.
   - [ ] Tests(including UT, IT, E2E) are added to verify the new feature.
   - [ ] If it's UI related, attach the screenshots below.
   
   
   - [x] If this pull request closes/resolves/fixes an existing issue, replace the issue number. https://github.com/apache/skywalking/issues/10680
   - [x] Update the [`CHANGES` log](https://github.com/apache/skywalking-java/blob/main/CHANGES.md).
   


-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-java] wu-sheng commented on a diff in pull request #505: Feature/support forkjoinpool

Posted by "wu-sheng (via GitHub)" <gi...@apache.org>.
wu-sheng commented on code in PR #505:
URL: https://github.com/apache/skywalking-java/pull/505#discussion_r1166491622


##########
apm-sniffer/bootstrap-plugins/jdk-forkjoinpool-plugin/src/main/java/org/apache/skywalking/apm/plugin/jdk/forkjoinpool/ForkJoinWorkerQueueMethodInterceptor.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.skywalking.apm.plugin.jdk.forkjoinpool;
+
+import java.lang.reflect.Method;
+import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import org.apache.skywalking.apm.agent.core.context.ContextSnapshot;
+import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.EnhancedInstance;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.InstanceMethodsAroundInterceptor;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.MethodInterceptResult;
+import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
+
+public class ForkJoinWorkerQueueMethodInterceptor implements InstanceMethodsAroundInterceptor {

Review Comment:
   Try to use `InstanceMethodsAroundInterceptorV2`. And the newly created span could be set and read through MethodInvocationContext.
   ForkJoin should be super fast, and creating an extra span leads to a payload. Let's keep the load as light as possible.



-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-java] wu-sheng commented on a diff in pull request #505: Feature/support forkjoinpool

Posted by "wu-sheng (via GitHub)" <gi...@apache.org>.
wu-sheng commented on code in PR #505:
URL: https://github.com/apache/skywalking-java/pull/505#discussion_r1168252001


##########
apm-sniffer/bootstrap-plugins/jdk-forkjoinpool-plugin/src/main/java/org/apache/skywalking/apm/plugin/jdk/forkjoinpool/define/ForkJoinTaskInstrumentation.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.skywalking.apm.plugin.jdk.forkjoinpool.define;
+
+import static net.bytebuddy.matcher.ElementMatchers.any;
+
+import net.bytebuddy.description.method.MethodDescription;
+import net.bytebuddy.matcher.ElementMatcher;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.ConstructorInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.InstanceMethodsInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.StaticMethodsInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.ClassEnhancePluginDefine;
+import org.apache.skywalking.apm.agent.core.plugin.match.ClassMatch;
+import org.apache.skywalking.apm.agent.core.plugin.match.HierarchyMatch;
+
+public class ForkJoinTaskInstrumentation extends ClassEnhancePluginDefine {

Review Comment:
   ```suggestion
   public class ForkJoinTaskInstrumentation extends ClassInstanceMethodsEnhancePluginDefine {
   ```



##########
apm-sniffer/bootstrap-plugins/jdk-forkjoinpool-plugin/src/main/java/org/apache/skywalking/apm/plugin/jdk/forkjoinpool/define/ForkJoinWorkerQueueInstrumentation.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.skywalking.apm.plugin.jdk.forkjoinpool.define;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+
+import net.bytebuddy.description.method.MethodDescription;
+import net.bytebuddy.matcher.ElementMatcher;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.ConstructorInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.v2.ClassEnhancePluginDefineV2;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.v2.InstanceMethodsInterceptV2Point;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.v2.StaticMethodsInterceptV2Point;
+import org.apache.skywalking.apm.agent.core.plugin.match.ClassMatch;
+import org.apache.skywalking.apm.agent.core.plugin.match.NameMatch;
+
+public class ForkJoinWorkerQueueInstrumentation extends ClassEnhancePluginDefineV2 {
+
+    private static final String FORK_JOIN_WORKER_QUEUE_CLASS = "java.util.concurrent.ForkJoinPool$WorkQueue";
+
+    private static final String FORK_JOIN_WORKER_QUEUE_RUN_TASK_METHOD = "runTask";
+    private static final String FORK_JOIN_WORKER_QUEUE_RUN_TASK_METHOD_INTERCEPTOR = "org.apache.skywalking.apm.plugin.jdk.forkjoinpool.ForkJoinWorkerQueueMethodInterceptor";
+
+    @Override
+    protected ClassMatch enhanceClass() {
+        return NameMatch.byName(FORK_JOIN_WORKER_QUEUE_CLASS);
+    }
+
+    @Override
+    public ConstructorInterceptPoint[] getConstructorsInterceptPoints() {
+        return new ConstructorInterceptPoint[0];
+    }
+
+    @Override
+    public InstanceMethodsInterceptV2Point[] getInstanceMethodsInterceptV2Points() {
+        return new InstanceMethodsInterceptV2Point[]{
+                new InstanceMethodsInterceptV2Point() {
+                    @Override
+                    public ElementMatcher<MethodDescription> getMethodsMatcher() {
+                        return named(FORK_JOIN_WORKER_QUEUE_RUN_TASK_METHOD);
+                    }
+
+                    @Override
+                    public String getMethodsInterceptorV2() {
+                        return FORK_JOIN_WORKER_QUEUE_RUN_TASK_METHOD_INTERCEPTOR;
+                    }
+
+                    @Override
+                    public boolean isOverrideArgs() {
+                        return false;
+                    }
+                }
+        };
+    }
+
+    @Override
+    public StaticMethodsInterceptV2Point[] getStaticMethodsInterceptV2Points() {
+        return new StaticMethodsInterceptV2Point[0];
+    }
+
+    @Override
+    public boolean isBootstrapInstrumentation() {
+        return true;
+    }

Review Comment:
   ```suggestion
   ```
   
   I think we don't need this once we inherit from the `ClassInstanceMethodsEnhancePluginDefineV2`.



##########
apm-sniffer/bootstrap-plugins/jdk-forkjoinpool-plugin/src/main/java/org/apache/skywalking/apm/plugin/jdk/forkjoinpool/define/ForkJoinWorkerQueueInstrumentation.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.skywalking.apm.plugin.jdk.forkjoinpool.define;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+
+import net.bytebuddy.description.method.MethodDescription;
+import net.bytebuddy.matcher.ElementMatcher;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.ConstructorInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.v2.ClassEnhancePluginDefineV2;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.v2.InstanceMethodsInterceptV2Point;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.v2.StaticMethodsInterceptV2Point;
+import org.apache.skywalking.apm.agent.core.plugin.match.ClassMatch;
+import org.apache.skywalking.apm.agent.core.plugin.match.NameMatch;
+
+public class ForkJoinWorkerQueueInstrumentation extends ClassEnhancePluginDefineV2 {

Review Comment:
   ```suggestion
   public class ForkJoinWorkerQueueInstrumentation extends ClassInstanceMethodsEnhancePluginDefineV2 {
   ```



##########
apm-sniffer/bootstrap-plugins/jdk-forkjoinpool-plugin/src/main/java/org/apache/skywalking/apm/plugin/jdk/forkjoinpool/define/ForkJoinTaskInstrumentation.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.skywalking.apm.plugin.jdk.forkjoinpool.define;
+
+import static net.bytebuddy.matcher.ElementMatchers.any;
+
+import net.bytebuddy.description.method.MethodDescription;
+import net.bytebuddy.matcher.ElementMatcher;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.ConstructorInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.InstanceMethodsInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.StaticMethodsInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.ClassEnhancePluginDefine;
+import org.apache.skywalking.apm.agent.core.plugin.match.ClassMatch;
+import org.apache.skywalking.apm.agent.core.plugin.match.HierarchyMatch;
+
+public class ForkJoinTaskInstrumentation extends ClassEnhancePluginDefine {
+
+    private static final String FORK_JOIN_TASK_CLASS = "java.util.concurrent.ForkJoinTask";
+    private static final String FORK_JOIN_TASK_INTERCEPTOR = "org.apache.skywalking.apm.plugin.jdk.forkjoinpool.ForkJoinTaskConstructorInterceptor";
+
+    @Override
+    protected ClassMatch enhanceClass() {
+        return HierarchyMatch.byHierarchyMatch(FORK_JOIN_TASK_CLASS);
+    }
+
+    @Override
+    public ConstructorInterceptPoint[] getConstructorsInterceptPoints() {
+        return new ConstructorInterceptPoint[]{
+                new ConstructorInterceptPoint() {
+                    @Override
+                    public ElementMatcher<MethodDescription> getConstructorMatcher() {
+                        return any();
+                    }
+
+                    @Override
+                    public String getConstructorInterceptor() {
+                        return FORK_JOIN_TASK_INTERCEPTOR;
+                    }
+                }
+        };
+    }
+
+    @Override
+    public InstanceMethodsInterceptPoint[] getInstanceMethodsInterceptPoints() {
+        return new InstanceMethodsInterceptPoint[0];
+    }
+
+    @Override
+    public StaticMethodsInterceptPoint[] getStaticMethodsInterceptPoints() {
+        return new StaticMethodsInterceptPoint[0];
+    }
+
+    @Override
+    public boolean isBootstrapInstrumentation() {
+        return true;
+    }

Review Comment:
   ```suggestion
   ```
   
   I think we don't need this once we inherit from the `ClassInstanceMethodsEnhancePluginDefine`. 



-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-java] wu-sheng merged pull request #505: Feature/support forkjoinpool

Posted by "wu-sheng (via GitHub)" <gi...@apache.org>.
wu-sheng merged PR #505:
URL: https://github.com/apache/skywalking-java/pull/505


-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-java] wu-sheng commented on pull request #505: Feature/support forkjoinpool

Posted by "wu-sheng (via GitHub)" <gi...@apache.org>.
wu-sheng commented on PR #505:
URL: https://github.com/apache/skywalking-java/pull/505#issuecomment-1508049074

   This doc should be updated, https://skywalking.apache.org/docs/skywalking-java/next/en/setup/service-agent/java-agent/bootstrap-plugins/
   
   And as a new plugin is being added, you should update `supported-list` and `plugin-list`, otherwise, CI would fail.


-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-java] xu1009 commented on a diff in pull request #505: Feature/support forkjoinpool

Posted by "xu1009 (via GitHub)" <gi...@apache.org>.
xu1009 commented on code in PR #505:
URL: https://github.com/apache/skywalking-java/pull/505#discussion_r1168236436


##########
apm-sniffer/bootstrap-plugins/jdk-forkjoinpool-plugin/src/main/java/org/apache/skywalking/apm/plugin/jdk/forkjoinpool/ForkJoinWorkerQueueMethodInterceptor.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.skywalking.apm.plugin.jdk.forkjoinpool;
+
+import java.lang.reflect.Method;
+import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import org.apache.skywalking.apm.agent.core.context.ContextSnapshot;
+import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.EnhancedInstance;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.InstanceMethodsAroundInterceptor;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.MethodInterceptResult;
+import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
+
+public class ForkJoinWorkerQueueMethodInterceptor implements InstanceMethodsAroundInterceptor {

Review Comment:
   ok,the performance will be better read from MethodInvocationContext than threadlocal



-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-java] wu-sheng commented on a diff in pull request #505: Feature/support forkjoinpool

Posted by "wu-sheng (via GitHub)" <gi...@apache.org>.
wu-sheng commented on code in PR #505:
URL: https://github.com/apache/skywalking-java/pull/505#discussion_r1168322268


##########
apm-sniffer/bootstrap-plugins/jdk-forkjoinpool-plugin/src/main/java/org/apache/skywalking/apm/plugin/jdk/forkjoinpool/define/ForkJoinTaskInstrumentation.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.skywalking.apm.plugin.jdk.forkjoinpool.define;
+
+import static net.bytebuddy.matcher.ElementMatchers.any;
+
+import net.bytebuddy.description.method.MethodDescription;
+import net.bytebuddy.matcher.ElementMatcher;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.ConstructorInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.InstanceMethodsInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.StaticMethodsInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.ClassEnhancePluginDefine;
+import org.apache.skywalking.apm.agent.core.plugin.match.ClassMatch;
+import org.apache.skywalking.apm.agent.core.plugin.match.HierarchyMatch;
+
+public class ForkJoinTaskInstrumentation extends ClassEnhancePluginDefine {

Review Comment:
   @xu1009 I think you missed this too.



-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-java] xu1009 commented on pull request #505: Feature/support forkjoinpool

Posted by "xu1009 (via GitHub)" <gi...@apache.org>.
xu1009 commented on PR #505:
URL: https://github.com/apache/skywalking-java/pull/505#issuecomment-1508097483

   ok
   


-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-java] wu-sheng commented on pull request #505: Feature/support forkjoinpool

Posted by "wu-sheng (via GitHub)" <gi...@apache.org>.
wu-sheng commented on PR #505:
URL: https://github.com/apache/skywalking-java/pull/505#issuecomment-1510799883

   Notice, the code format should be fixed.
   
   ```
     Error:  /home/runner/work/skywalking-java/skywalking-java/test/plugin/scenarios/jdk-forkjoinpool-scenario/src/main/java/org/apache/skywalking/apm/testcase/jdk/forkjoinpool/controller/CaseController.java:44:5: 'VARIABLE_DEF' has more than 1 empty lines before. [EmptyLineSeparator]
     Audit done.
     [INFO] There is 1 error reported by Checkstyle 8.38 with /home/runner/work/skywalking-java/skywalking-java/apm-checkstyle/checkStyle.xml ruleset.
     Error:  scenarios/jdk-forkjoinpool-scenario/src/main/java/org/apache/skywalking/apm/testcase/jdk/forkjoinpool/controller/CaseController.java:[44,5] (whitespace) EmptyLineSeparator: 'VARIABLE_DEF' has more than 1 empty lines before.
   ```


-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-java] wu-sheng commented on a diff in pull request #505: Feature/support forkjoinpool

Posted by "wu-sheng (via GitHub)" <gi...@apache.org>.
wu-sheng commented on code in PR #505:
URL: https://github.com/apache/skywalking-java/pull/505#discussion_r1168137143


##########
apm-sniffer/bootstrap-plugins/jdk-forkjoinpool-plugin/src/main/java/org/apache/skywalking/apm/plugin/jdk/forkjoinpool/ForkJoinWorkerQueueMethodInterceptor.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.skywalking.apm.plugin.jdk.forkjoinpool;
+
+import java.lang.reflect.Method;
+import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import org.apache.skywalking.apm.agent.core.context.ContextSnapshot;
+import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.EnhancedInstance;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.InstanceMethodsAroundInterceptor;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.MethodInterceptResult;
+import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
+
+public class ForkJoinWorkerQueueMethodInterceptor implements InstanceMethodsAroundInterceptor {

Review Comment:
   You created one span in before, and stop in the after. The STOP is actually calling active span from threadlocal.
   A better way is, you put the created span in the MethodInvocationContext, and read from it in the after.



-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-java] xu1009 commented on a diff in pull request #505: Feature/support forkjoinpool

Posted by "xu1009 (via GitHub)" <gi...@apache.org>.
xu1009 commented on code in PR #505:
URL: https://github.com/apache/skywalking-java/pull/505#discussion_r1168134660


##########
apm-sniffer/bootstrap-plugins/jdk-forkjoinpool-plugin/src/main/java/org/apache/skywalking/apm/plugin/jdk/forkjoinpool/ForkJoinWorkerQueueMethodInterceptor.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.skywalking.apm.plugin.jdk.forkjoinpool;
+
+import java.lang.reflect.Method;
+import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import org.apache.skywalking.apm.agent.core.context.ContextSnapshot;
+import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.EnhancedInstance;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.InstanceMethodsAroundInterceptor;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.MethodInterceptResult;
+import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
+
+public class ForkJoinWorkerQueueMethodInterceptor implements InstanceMethodsAroundInterceptor {

Review Comment:
   not get your mean,i have checked InstanceMethodsAroundInterceptorV2, it seems not suitable in current



-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-java] xu1009 commented on a diff in pull request #505: Feature/support forkjoinpool

Posted by "xu1009 (via GitHub)" <gi...@apache.org>.
xu1009 commented on code in PR #505:
URL: https://github.com/apache/skywalking-java/pull/505#discussion_r1168263576


##########
apm-sniffer/bootstrap-plugins/jdk-forkjoinpool-plugin/src/main/java/org/apache/skywalking/apm/plugin/jdk/forkjoinpool/define/ForkJoinWorkerQueueInstrumentation.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.skywalking.apm.plugin.jdk.forkjoinpool.define;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+
+import net.bytebuddy.description.method.MethodDescription;
+import net.bytebuddy.matcher.ElementMatcher;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.ConstructorInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.v2.ClassEnhancePluginDefineV2;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.v2.InstanceMethodsInterceptV2Point;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.v2.StaticMethodsInterceptV2Point;
+import org.apache.skywalking.apm.agent.core.plugin.match.ClassMatch;
+import org.apache.skywalking.apm.agent.core.plugin.match.NameMatch;
+
+public class ForkJoinWorkerQueueInstrumentation extends ClassEnhancePluginDefineV2 {
+
+    private static final String FORK_JOIN_WORKER_QUEUE_CLASS = "java.util.concurrent.ForkJoinPool$WorkQueue";
+
+    private static final String FORK_JOIN_WORKER_QUEUE_RUN_TASK_METHOD = "runTask";
+    private static final String FORK_JOIN_WORKER_QUEUE_RUN_TASK_METHOD_INTERCEPTOR = "org.apache.skywalking.apm.plugin.jdk.forkjoinpool.ForkJoinWorkerQueueMethodInterceptor";
+
+    @Override
+    protected ClassMatch enhanceClass() {
+        return NameMatch.byName(FORK_JOIN_WORKER_QUEUE_CLASS);
+    }
+
+    @Override
+    public ConstructorInterceptPoint[] getConstructorsInterceptPoints() {
+        return new ConstructorInterceptPoint[0];
+    }
+
+    @Override
+    public InstanceMethodsInterceptV2Point[] getInstanceMethodsInterceptV2Points() {
+        return new InstanceMethodsInterceptV2Point[]{
+                new InstanceMethodsInterceptV2Point() {
+                    @Override
+                    public ElementMatcher<MethodDescription> getMethodsMatcher() {
+                        return named(FORK_JOIN_WORKER_QUEUE_RUN_TASK_METHOD);
+                    }
+
+                    @Override
+                    public String getMethodsInterceptorV2() {
+                        return FORK_JOIN_WORKER_QUEUE_RUN_TASK_METHOD_INTERCEPTOR;
+                    }
+
+                    @Override
+                    public boolean isOverrideArgs() {
+                        return false;
+                    }
+                }
+        };
+    }
+
+    @Override
+    public StaticMethodsInterceptV2Point[] getStaticMethodsInterceptV2Points() {
+        return new StaticMethodsInterceptV2Point[0];
+    }
+
+    @Override
+    public boolean isBootstrapInstrumentation() {
+        return true;
+    }

Review Comment:
   ok



-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-java] wu-sheng commented on pull request #505: Feature/support forkjoinpool

Posted by "wu-sheng (via GitHub)" <gi...@apache.org>.
wu-sheng commented on PR #505:
URL: https://github.com/apache/skywalking-java/pull/505#issuecomment-1510816507

   > > 
   > 
   > not report this in my loacl when execute the test, it seems not check test code style
   
   CI reports this, you could check the logs.
   
   <img width="923" alt="image" src="https://user-images.githubusercontent.com/5441976/232409936-013fc1ca-1836-435f-820b-e07306fd4ab9.png">
   


-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-java] xu1009 commented on pull request #505: Feature/support forkjoinpool

Posted by "xu1009 (via GitHub)" <gi...@apache.org>.
xu1009 commented on PR #505:
URL: https://github.com/apache/skywalking-java/pull/505#issuecomment-1510814433

   > 
   
   not report this in my loacl when execute the test, it seems not check test code style 


-- 
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: notifications-unsubscribe@skywalking.apache.org

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