You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by zh...@apache.org on 2022/12/09 11:53:48 UTC

[rocketmq] 07/26: [ISSUE #5486] polish clean expire request in remoting server

This is an automated email from the ASF dual-hosted git repository.

zhouxzhan pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/rocketmq.git

commit 8a23c54a9193a6fd76446a5d6b8b222d3bdaed77
Author: kaiyi.lk <ka...@alibaba-inc.com>
AuthorDate: Wed Nov 9 15:43:52 2022 +0800

    [ISSUE #5486] polish clean expire request in remoting server
---
 .../thread/FutureTaskExtThreadPoolExecutor.java    | 41 ++++++++++++++++++++++
 .../rocketmq/common/thread/ThreadPoolMonitor.java  |  2 +-
 .../proxy/remoting/RemotingProtocolServer.java     | 33 ++++++-----------
 3 files changed, 52 insertions(+), 24 deletions(-)

diff --git a/common/src/main/java/org/apache/rocketmq/common/thread/FutureTaskExtThreadPoolExecutor.java b/common/src/main/java/org/apache/rocketmq/common/thread/FutureTaskExtThreadPoolExecutor.java
new file mode 100644
index 000000000..411da9221
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/thread/FutureTaskExtThreadPoolExecutor.java
@@ -0,0 +1,41 @@
+/*
+ * 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.rocketmq.common.thread;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.RejectedExecutionHandler;
+import java.util.concurrent.RunnableFuture;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.apache.rocketmq.common.future.FutureTaskExt;
+
+public class FutureTaskExtThreadPoolExecutor extends ThreadPoolExecutor {
+
+    public FutureTaskExtThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime,
+        TimeUnit unit,
+        BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory,
+        RejectedExecutionHandler handler) {
+        super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler);
+    }
+
+    @Override
+    protected <T> RunnableFuture<T> newTaskFor(final Runnable runnable, final T value) {
+        return new FutureTaskExt<>(runnable, value);
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/thread/ThreadPoolMonitor.java b/common/src/main/java/org/apache/rocketmq/common/thread/ThreadPoolMonitor.java
index 72d4384c4..49d97a5d7 100644
--- a/common/src/main/java/org/apache/rocketmq/common/thread/ThreadPoolMonitor.java
+++ b/common/src/main/java/org/apache/rocketmq/common/thread/ThreadPoolMonitor.java
@@ -81,7 +81,7 @@ public class ThreadPoolMonitor {
         String name,
         int queueCapacity,
         List<ThreadPoolStatusMonitor> threadPoolStatusMonitors) {
-        ThreadPoolExecutor executor = new ThreadPoolExecutor(
+        ThreadPoolExecutor executor = new FutureTaskExtThreadPoolExecutor(
             corePoolSize,
             maximumPoolSize,
             keepAliveTime,
diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/RemotingProtocolServer.java b/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/RemotingProtocolServer.java
index fdf1870a5..91c4422d2 100644
--- a/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/RemotingProtocolServer.java
+++ b/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/RemotingProtocolServer.java
@@ -17,22 +17,21 @@
 
 package org.apache.rocketmq.proxy.remoting;
 
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import io.netty.channel.Channel;
-import java.lang.reflect.Field;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.FutureTask;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-import org.apache.rocketmq.broker.latency.FutureTaskExt;
 import org.apache.rocketmq.client.exception.MQClientException;
 import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.future.FutureTaskExt;
 import org.apache.rocketmq.common.protocol.RequestCode;
 import org.apache.rocketmq.common.protocol.ResponseCode;
 import org.apache.rocketmq.common.thread.ThreadPoolMonitor;
 import org.apache.rocketmq.common.thread.ThreadPoolStatusMonitor;
-import org.apache.rocketmq.proxy.common.ReflectionCache;
 import org.apache.rocketmq.proxy.common.StartAndShutdown;
 import org.apache.rocketmq.proxy.config.ConfigurationManager;
 import org.apache.rocketmq.proxy.config.ProxyConfig;
@@ -78,8 +77,7 @@ public class RemotingProtocolServer implements StartAndShutdown, RemotingProxyOu
     protected final ThreadPoolExecutor updateOffsetExecutor;
     protected final ThreadPoolExecutor topicRouteExecutor;
     protected final ThreadPoolExecutor defaultExecutor;
-
-    private final ReflectionCache reflectionCache = new ReflectionCache();
+    protected final ScheduledExecutorService timerExecutor;
 
     public RemotingProtocolServer(MessagingProcessor messagingProcessor) {
         this.messagingProcessor = messagingProcessor;
@@ -173,10 +171,11 @@ public class RemotingProtocolServer implements StartAndShutdown, RemotingProxyOu
             config.getRemotingDefaultThreadPoolQueueCapacity(),
             new ThreadPoolHeadSlowTimeMillsMonitor(config.getRemotingWaitTimeMillsInDefaultQueue())
         );
-    }
-
-    protected void init() {
 
+        this.timerExecutor = Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder().setNameFormat("RemotingServerScheduler-%d").build()
+        );
+        this.timerExecutor.scheduleAtFixedRate(this::cleanExpireRequest, 10, 10, TimeUnit.SECONDS);
     }
 
     protected void registerRemotingServer(RemotingServer remotingServer) {
@@ -342,19 +341,7 @@ public class RemotingProtocolServer implements StartAndShutdown, RemotingProxyOu
 
     private RequestTask castRunnable(final Runnable runnable) {
         try {
-            if (runnable instanceof FutureTask) {
-                Field callableField = reflectionCache.getDeclaredField(FutureTask.class, "callable");
-                Callable callable = (Callable) callableField.get(runnable);
-                if (callable == null) {
-                    return null;
-                }
-                Field taskField = reflectionCache.getDeclaredField(callable.getClass(), "task");
-                if (taskField == null) {
-                    log.warn("get task from FutureTask failed. class:{}", runnable.getClass().getName());
-                    return null;
-                }
-                return (RequestTask) taskField.get(callable);
-            } else if (runnable instanceof FutureTaskExt) {
+            if (runnable instanceof FutureTaskExt) {
                 FutureTaskExt futureTaskExt = (FutureTaskExt) runnable;
                 return (RequestTask) futureTaskExt.getRunnable();
             }