You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by hu...@apache.org on 2022/07/29 01:41:11 UTC

[iotdb] branch master updated: [IOTDB-3727] Support clear cache in new cluster (#6756)

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

hui pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 2db18ee7fb [IOTDB-3727] Support clear cache in new cluster (#6756)
2db18ee7fb is described below

commit 2db18ee7fba96535263a0219d3d6fd3ce742f803
Author: ly <33...@users.noreply.github.com>
AuthorDate: Fri Jul 29 09:41:06 2022 +0800

    [IOTDB-3727] Support clear cache in new cluster (#6756)
---
 .../org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4   |  2 +-
 .../confignode/client/DataNodeRequestType.java     |  4 +-
 .../async/datanode/AsyncDataNodeClientPool.java    | 14 ++++
 .../client/async/handlers/ClearCacheHandler.java   | 81 ++++++++++++++++++++++
 .../iotdb/confignode/manager/ConfigManager.java    |  9 +++
 .../apache/iotdb/confignode/manager/IManager.java  |  3 +
 .../iotdb/confignode/manager/NodeManager.java      | 13 ++++
 .../thrift/ConfigNodeRPCServiceProcessor.java      |  6 ++
 .../Maintenance-Tools/Maintenance-Command.md       |  4 +-
 .../Maintenance-Tools/Maintenance-Command.md       |  4 +-
 .../apache/iotdb/db/client/ConfigNodeClient.java   | 17 +++++
 .../iotdb/db/localconfignode/LocalConfigNode.java  | 12 ++++
 .../mpp/plan/execution/config/ClearCacheTask.java  | 53 ++++++++++++++
 .../plan/execution/config/ConfigTaskVisitor.java   |  6 ++
 .../config/executor/ClusterConfigTaskExecutor.java | 21 ++++++
 .../config/executor/IConfigTaskExecutor.java       |  3 +
 .../executor/StandaloneConfigTaskExecutor.java     | 14 ++++
 .../iotdb/db/mpp/plan/parser/ASTVisitor.java       | 15 ++++
 .../db/mpp/plan/statement/StatementVisitor.java    |  5 ++
 .../plan/statement/sys/ClearCacheStatement.java    | 67 ++++++++++++++++++
 .../impl/DataNodeInternalRPCServiceImpl.java       | 12 ++++
 thrift-commons/src/main/thrift/common.thrift       |  4 ++
 .../src/main/thrift/confignode.thrift              |  4 ++
 thrift/src/main/thrift/datanode.thrift             |  2 +
 24 files changed, 370 insertions(+), 5 deletions(-)

diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
index 24579af389..a617c85ee9 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
@@ -644,7 +644,7 @@ flush
 
 // Clear Cache
 clearCache
-    : CLEAR CACHE
+    : CLEAR CACHE (ON (LOCAL | CLUSTER))?
     ;
 
 // Settle
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/client/DataNodeRequestType.java b/confignode/src/main/java/org/apache/iotdb/confignode/client/DataNodeRequestType.java
index 31d951849a..82e6974176 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/client/DataNodeRequestType.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/client/DataNodeRequestType.java
@@ -36,6 +36,6 @@ public enum DataNodeRequestType {
   FLUSH,
   UPDATE_REGION_ROUTE_MAP,
   BROADCAST_LATEST_CONFIG_NODE_GROUP,
-
-  UPDATE_TEMPLATE
+  UPDATE_TEMPLATE,
+  CLEAR_CACHE
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/client/async/datanode/AsyncDataNodeClientPool.java b/confignode/src/main/java/org/apache/iotdb/confignode/client/async/datanode/AsyncDataNodeClientPool.java
index d12c0f2705..813a25d468 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/client/async/datanode/AsyncDataNodeClientPool.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/client/async/datanode/AsyncDataNodeClientPool.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.confignode.client.async.datanode;
 
+import org.apache.iotdb.common.rpc.thrift.TClearCacheReq;
 import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
@@ -30,6 +31,7 @@ import org.apache.iotdb.commons.client.async.AsyncDataNodeInternalServiceClient;
 import org.apache.iotdb.confignode.client.ConfigNodeClientPoolFactory;
 import org.apache.iotdb.confignode.client.DataNodeRequestType;
 import org.apache.iotdb.confignode.client.async.handlers.AbstractRetryHandler;
+import org.apache.iotdb.confignode.client.async.handlers.ClearCacheHandler;
 import org.apache.iotdb.confignode.client.async.handlers.CreateRegionHandler;
 import org.apache.iotdb.confignode.client.async.handlers.DataNodeHeartbeatHandler;
 import org.apache.iotdb.confignode.client.async.handlers.FlushHandler;
@@ -115,6 +117,15 @@ public class AsyncDataNodeClientPool {
                     dataNodeLocationMap,
                     dataNodeResponseStatus);
             break;
+          case CLEAR_CACHE:
+            handler =
+                new ClearCacheHandler(
+                    countDownLatch,
+                    requestType,
+                    targetDataNode,
+                    dataNodeLocationMap,
+                    dataNodeResponseStatus);
+            break;
           case UPDATE_REGION_ROUTE_MAP:
             handler =
                 new UpdateRegionRouteMapHandler(
@@ -169,6 +180,9 @@ public class AsyncDataNodeClientPool {
         case FLUSH:
           client.flush((TFlushReq) req, (FlushHandler) handler);
           break;
+        case CLEAR_CACHE:
+          client.clearCache((TClearCacheReq) req, (ClearCacheHandler) handler);
+          break;
         case UPDATE_REGION_ROUTE_MAP:
           client.updateRegionCache((TRegionRouteReq) req, (UpdateRegionRouteMapHandler) handler);
           break;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/ClearCacheHandler.java b/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/ClearCacheHandler.java
new file mode 100644
index 0000000000..67a02843e9
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/ClearCacheHandler.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.confignode.client.async.handlers;
+
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.confignode.client.DataNodeRequestType;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.TSStatusCode;
+
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+
+public class ClearCacheHandler extends AbstractRetryHandler
+    implements AsyncMethodCallback<TSStatus> {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(ClearCacheHandler.class);
+
+  private final List<TSStatus> dataNodeResponseStatus;
+
+  public ClearCacheHandler(
+      CountDownLatch countDownLatch,
+      DataNodeRequestType requestType,
+      TDataNodeLocation targetDataNode,
+      Map<Integer, TDataNodeLocation> dataNodeLocationMap,
+      List<TSStatus> dataNodeResponseStatus) {
+    super(countDownLatch, requestType, targetDataNode, dataNodeLocationMap);
+    this.dataNodeResponseStatus = dataNodeResponseStatus;
+  }
+
+  @Override
+  public void onComplete(TSStatus response) {
+    if (response.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      dataNodeResponseStatus.add(response);
+      dataNodeLocationMap.remove(targetDataNode.getDataNodeId());
+      LOGGER.info("Successfully Clear Cache on DataNode: {}", targetDataNode);
+    } else {
+      LOGGER.error(
+          "Failed to Clear Cache on DataNode {}, {}",
+          dataNodeLocationMap.get(targetDataNode.getDataNodeId()),
+          response);
+    }
+    countDownLatch.countDown();
+  }
+
+  @Override
+  public void onError(Exception exception) {
+    countDownLatch.countDown();
+    dataNodeResponseStatus.add(
+        new TSStatus(
+            RpcUtils.getStatus(
+                TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode(),
+                "Clear Cache error on DataNode: {id="
+                    + targetDataNode.getDataNodeId()
+                    + ", internalEndPoint="
+                    + targetDataNode.getInternalEndPoint()
+                    + "}"
+                    + exception.getMessage())));
+  }
+}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
index 14d727d6a5..a5c253906e 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.confignode.manager;
 
+import org.apache.iotdb.common.rpc.thrift.TClearCacheReq;
 import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 import org.apache.iotdb.common.rpc.thrift.TDataNodeConfiguration;
@@ -842,6 +843,14 @@ public class ConfigManager implements IManager {
         : status;
   }
 
+  @Override
+  public TSStatus clearCache(TClearCacheReq req) {
+    TSStatus status = confirmLeader();
+    return status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()
+        ? RpcUtils.squashResponseStatusList(nodeManager.clearCache(req))
+        : status;
+  }
+
   @Override
   public TRegionRouteMapResp getLatestRegionRouteMap() {
     TSStatus status = confirmLeader();
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
index 9c7c0975f2..b7fc84264b 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.confignode.manager;
 
+import org.apache.iotdb.common.rpc.thrift.TClearCacheReq;
 import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TFlushReq;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
@@ -270,6 +271,8 @@ public interface IManager {
 
   TSStatus flush(TFlushReq req);
 
+  TSStatus clearCache(TClearCacheReq req);
+
   /**
    * Get the latest RegionRouteMap
    *
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/NodeManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/NodeManager.java
index 983ea2f558..e8993e6de3 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/NodeManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/NodeManager.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.confignode.manager;
 
+import org.apache.iotdb.common.rpc.thrift.TClearCacheReq;
 import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TDataNodeConfiguration;
 import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
@@ -458,12 +459,24 @@ public class NodeManager {
         configManager.getNodeManager().getRegisteredDataNodeLocations(req.dataNodeId);
     List<TSStatus> dataNodeResponseStatus =
         Collections.synchronizedList(new ArrayList<>(dataNodeLocationMap.size()));
+
     AsyncDataNodeClientPool.getInstance()
         .sendAsyncRequestToDataNodeWithRetry(
             req, dataNodeLocationMap, DataNodeRequestType.FLUSH, dataNodeResponseStatus);
     return dataNodeResponseStatus;
   }
 
+  public List<TSStatus> clearCache(TClearCacheReq req) {
+    Map<Integer, TDataNodeLocation> dataNodeLocationMap =
+        configManager.getNodeManager().getRegisteredDataNodeLocations(req.dataNodeId);
+    List<TSStatus> dataNodeResponseStatus =
+        Collections.synchronizedList(new ArrayList<>(dataNodeLocationMap.size()));
+    AsyncDataNodeClientPool.getInstance()
+        .sendAsyncRequestToDataNodeWithRetry(
+            req, dataNodeLocationMap, DataNodeRequestType.CLEAR_CACHE, dataNodeResponseStatus);
+    return dataNodeResponseStatus;
+  }
+
   private LoadManager getLoadManager() {
     return configManager.getLoadManager();
   }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
index 0ecc43bea0..62c82aa1e8 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.confignode.service.thrift;
 
+import org.apache.iotdb.common.rpc.thrift.TClearCacheReq;
 import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TFlushReq;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
@@ -524,6 +525,11 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
     return configManager.flush(req);
   }
 
+  @Override
+  public TSStatus clearCache(TClearCacheReq req) throws TException {
+    return configManager.clearCache(req);
+  }
+
   @Override
   public TShowRegionResp showRegion(TShowRegionReq showRegionReq) throws TException {
     GetRegionInfoListPlan getRegionInfoListPlan = new GetRegionInfoListPlan(showRegionReq);
diff --git a/docs/UserGuide/Maintenance-Tools/Maintenance-Command.md b/docs/UserGuide/Maintenance-Tools/Maintenance-Command.md
index 01c8e9b607..3b3087e218 100644
--- a/docs/UserGuide/Maintenance-Tools/Maintenance-Command.md
+++ b/docs/UserGuide/Maintenance-Tools/Maintenance-Command.md
@@ -48,10 +48,12 @@ IoTDB> FULL MERGE
 
 ## CLEAR CACHE
 
-Clear the cache of chunk, chunk metadata and timeseries metadata to release the memory footprint.
+Clear the cache of chunk, chunk metadata and timeseries metadata to release the memory footprint. In cluster mode, we provide commands to clear a single node cache and clear the cluster cache.
 
 ```sql
 IoTDB> CLEAR CACHE
+IoTDB> CLEAR CACHE ON LOCAL
+IoTDB> CLEAR CACHE ON CLUSTER
 ```
 
 
diff --git a/docs/zh/UserGuide/Maintenance-Tools/Maintenance-Command.md b/docs/zh/UserGuide/Maintenance-Tools/Maintenance-Command.md
index 9747693eab..d983f05c06 100644
--- a/docs/zh/UserGuide/Maintenance-Tools/Maintenance-Command.md
+++ b/docs/zh/UserGuide/Maintenance-Tools/Maintenance-Command.md
@@ -48,10 +48,12 @@ IoTDB> FULL MERGE
 ## CLEAR CACHE
 
 
-手动清除chunk, chunk metadata和timeseries metadata的缓存,在内存资源紧张时,可以通过此命令,释放查询时缓存所占的内存空间。
+手动清除chunk, chunk metadata和timeseries metadata的缓存,在内存资源紧张时,可以通过此命令,释放查询时缓存所占的内存空间。在集群模式下,我们提供了清空单个节点缓存、清空整个集群缓存命令。
 
 ```sql
 IoTDB> CLEAR CACHE
+IoTDB> CLEAR CACHE ON LOCAL
+IoTDB> CLEAR CACHE ON CLUSTER
 ```
 
 ## SET STSTEM TO READONLY / WRITABLE
diff --git a/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java b/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
index c44217e7ab..a1517dcc82 100644
--- a/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
+++ b/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.client;
 
+import org.apache.iotdb.common.rpc.thrift.TClearCacheReq;
 import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.common.rpc.thrift.TFlushReq;
@@ -763,6 +764,22 @@ public class ConfigNodeClient
     throw new TException(MSG_RECONNECTION_FAIL);
   }
 
+  @Override
+  public TSStatus clearCache(TClearCacheReq req) throws TException {
+    for (int i = 0; i < RETRY_NUM; i++) {
+      try {
+        TSStatus status = client.clearCache(req);
+        if (!updateConfigNodeLeader(status)) {
+          return status;
+        }
+      } catch (TException e) {
+        configLeader = null;
+      }
+      reconnect();
+    }
+    throw new TException(MSG_RECONNECTION_FAIL);
+  }
+
   @Override
   public TShowRegionResp showRegion(TShowRegionReq req) throws TException {
     for (int i = 0; i < RETRY_NUM; i++) {
diff --git a/server/src/main/java/org/apache/iotdb/db/localconfignode/LocalConfigNode.java b/server/src/main/java/org/apache/iotdb/db/localconfignode/LocalConfigNode.java
index aea7b4e71d..a13035438e 100644
--- a/server/src/main/java/org/apache/iotdb/db/localconfignode/LocalConfigNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/localconfignode/LocalConfigNode.java
@@ -51,6 +51,9 @@ import org.apache.iotdb.commons.utils.AuthUtils;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngineV2;
+import org.apache.iotdb.db.engine.cache.BloomFilterCache;
+import org.apache.iotdb.db.engine.cache.ChunkCache;
+import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
 import org.apache.iotdb.db.engine.storagegroup.DataRegion;
 import org.apache.iotdb.db.exception.DataRegionException;
 import org.apache.iotdb.db.exception.metadata.PathNotExistException;
@@ -82,6 +85,8 @@ import org.apache.iotdb.db.qp.physical.sys.SetTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
 import org.apache.iotdb.db.rescon.MemTableManager;
 import org.apache.iotdb.db.sync.sender.manager.SchemaSyncManager;
+import org.apache.iotdb.rpc.RpcUtils;
+import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 
@@ -1269,4 +1274,11 @@ public class LocalConfigNode {
   public TSStatus executeFlushOperation(TFlushReq tFlushReq) {
     return storageEngine.operateFlush(tFlushReq);
   }
+
+  public TSStatus executeClearCacheOperation() {
+    ChunkCache.getInstance().clear();
+    TimeSeriesMetadataCache.getInstance().clear();
+    BloomFilterCache.getInstance().clear();
+    return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ClearCacheTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ClearCacheTask.java
new file mode 100644
index 0000000000..69e80d462a
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ClearCacheTask.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.plan.execution.config;
+
+import org.apache.iotdb.common.rpc.thrift.TClearCacheReq;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.mpp.plan.execution.config.executor.IConfigTaskExecutor;
+import org.apache.iotdb.db.mpp.plan.statement.sys.ClearCacheStatement;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+public class ClearCacheTask implements IConfigTask {
+
+  private ClearCacheStatement clearCacheStatement;
+
+  public ClearCacheTask(ClearCacheStatement clearCacheStatement) {
+    this.clearCacheStatement = clearCacheStatement;
+  }
+
+  @Override
+  public ListenableFuture<ConfigTaskResult> execute(IConfigTaskExecutor configTaskExecutor)
+      throws InterruptedException {
+    TClearCacheReq tClearCacheReq = new TClearCacheReq();
+
+    IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+    if (clearCacheStatement.isCluster()) {
+      tClearCacheReq.setDataNodeId(-1);
+    } else {
+      tClearCacheReq.setDataNodeId(config.getDataNodeId());
+    }
+    // If the action is executed successfully, return the Future.
+    // If your operation is async, you can return the corresponding future directly.
+    return configTaskExecutor.clearCache(tClearCacheReq);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java
index ea88f0856c..c5c4b2f822 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java
@@ -42,6 +42,7 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowNodesInSchem
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.AuthorStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.ClearCacheStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.FlushStatement;
 import org.apache.iotdb.tsfile.exception.NotImplementedException;
 
@@ -119,6 +120,11 @@ public class ConfigTaskVisitor
     return new FlushTask(flushStatement);
   }
 
+  @Override
+  public IConfigTask visitClearCache(ClearCacheStatement clearCacheStatement, TaskContext context) {
+    return new ClearCacheTask(clearCacheStatement);
+  }
+
   @Override
   public IConfigTask visitDropFunction(
       DropFunctionStatement dropFunctionStatement, TaskContext context) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java
index bced879b4c..104241e811 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.mpp.plan.execution.config.executor;
 
+import org.apache.iotdb.common.rpc.thrift.TClearCacheReq;
 import org.apache.iotdb.common.rpc.thrift.TFlushReq;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.common.rpc.thrift.TSetTTLReq;
@@ -287,6 +288,26 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     return future;
   }
 
+  @Override
+  public SettableFuture<ConfigTaskResult> clearCache(TClearCacheReq tClearCacheReq) {
+    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+
+    try (ConfigNodeClient client =
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.partitionRegionId)) {
+      // Send request to some API server
+      TSStatus tsStatus = client.clearCache(tClearCacheReq);
+      // Get response or throw exception
+      if (tsStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS));
+      } else {
+        future.setException(new StatementExecutionException(tsStatus));
+      }
+    } catch (IOException | TException e) {
+      future.setException(e);
+    }
+    return future;
+  }
+
   @Override
   public SettableFuture<ConfigTaskResult> showCluster() {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/IConfigTaskExecutor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/IConfigTaskExecutor.java
index b037cd4cee..8a36ea40ac 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/IConfigTaskExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/IConfigTaskExecutor.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.mpp.plan.execution.config.executor;
 
+import org.apache.iotdb.common.rpc.thrift.TClearCacheReq;
 import org.apache.iotdb.common.rpc.thrift.TFlushReq;
 import org.apache.iotdb.db.mpp.plan.execution.config.ConfigTaskResult;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.CountStorageGroupStatement;
@@ -62,6 +63,8 @@ public interface IConfigTaskExecutor {
 
   SettableFuture<ConfigTaskResult> flush(TFlushReq tFlushReq);
 
+  SettableFuture<ConfigTaskResult> clearCache(TClearCacheReq tClearCacheReq);
+
   SettableFuture<ConfigTaskResult> showCluster();
 
   SettableFuture<ConfigTaskResult> showTTL(ShowTTLStatement showTTLStatement);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/StandaloneConfigTaskExecutor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/StandaloneConfigTaskExecutor.java
index 3ac9fce614..acfac53c2f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/StandaloneConfigTaskExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/StandaloneConfigTaskExecutor.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.mpp.plan.execution.config.executor;
 
+import org.apache.iotdb.common.rpc.thrift.TClearCacheReq;
 import org.apache.iotdb.common.rpc.thrift.TFlushReq;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.exception.IoTDBException;
@@ -235,6 +236,19 @@ public class StandaloneConfigTaskExecutor implements IConfigTaskExecutor {
     return future;
   }
 
+  @Override
+  public SettableFuture<ConfigTaskResult> clearCache(TClearCacheReq tclearCacheReq) {
+    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    LocalConfigNode localConfigNode = LocalConfigNode.getInstance();
+    TSStatus tsStatus = localConfigNode.executeClearCacheOperation();
+    if (tsStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS));
+    } else {
+      future.setException(new StatementExecutionException(tsStatus));
+    }
+    return future;
+  }
+
   @Override
   public SettableFuture<ConfigTaskResult> showCluster() {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
index 4eb62b83b0..e8a550df78 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
@@ -115,6 +115,7 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTempl
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathsUsingTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.AuthorStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.ClearCacheStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.ExplainStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.FlushStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.ShowVersionStatement;
@@ -2327,6 +2328,20 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
     return flushStatement;
   }
 
+  // Clear Cache
+
+  @Override
+  public Statement visitClearCache(IoTDBSqlParser.ClearCacheContext ctx) {
+    ClearCacheStatement clearCacheStatement = new ClearCacheStatement(StatementType.CLEAR_CACHE);
+
+    if (ctx.LOCAL() != null) {
+      clearCacheStatement.setCluster(false);
+    } else {
+      clearCacheStatement.setCluster(true);
+    }
+    return clearCacheStatement;
+  }
+
   // show region
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
index 785d56831c..d20b0a4ecb 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
@@ -65,6 +65,7 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTempl
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathsUsingTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.AuthorStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.ClearCacheStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.ExplainStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.FlushStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.ShowVersionStatement;
@@ -262,6 +263,10 @@ public abstract class StatementVisitor<R, C> {
     return visitStatement(flushStatement, context);
   }
 
+  public R visitClearCache(ClearCacheStatement clearCacheStatement, C context) {
+    return visitStatement(clearCacheStatement, context);
+  }
+
   public R visitShowRegion(ShowRegionStatement showRegionStatement, C context) {
     return visitStatement(showRegionStatement, context);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/sys/ClearCacheStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/sys/ClearCacheStatement.java
new file mode 100644
index 0000000000..cf44c8f8d2
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/sys/ClearCacheStatement.java
@@ -0,0 +1,67 @@
+/*
+ * 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.iotdb.db.mpp.plan.statement.sys;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.mpp.plan.analyze.QueryType;
+import org.apache.iotdb.db.mpp.plan.constant.StatementType;
+import org.apache.iotdb.db.mpp.plan.statement.IConfigStatement;
+import org.apache.iotdb.db.mpp.plan.statement.Statement;
+import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+
+public class ClearCacheStatement extends Statement implements IConfigStatement {
+
+  private static final Logger logger = LoggerFactory.getLogger(ClearCacheStatement.class);
+
+  private boolean isCluster;
+
+  public ClearCacheStatement(StatementType clearCacheType) {
+    this.statementType = clearCacheType;
+  }
+
+  public boolean isCluster() {
+    return isCluster;
+  }
+
+  public void setCluster(boolean isCluster) {
+    this.isCluster = isCluster;
+  }
+
+  @Override
+  public QueryType getQueryType() {
+    return QueryType.WRITE;
+  }
+
+  @Override
+  public List<PartialPath> getPaths() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
+    return visitor.visitClearCache(this, context);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java
index 8f9281a480..31b8728813 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.service.thrift.impl;
 
+import org.apache.iotdb.common.rpc.thrift.TClearCacheReq;
 import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation;
 import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
 import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
@@ -46,6 +47,9 @@ import org.apache.iotdb.db.client.ConfigNodeInfo;
 import org.apache.iotdb.db.consensus.DataRegionConsensusImpl;
 import org.apache.iotdb.db.consensus.SchemaRegionConsensusImpl;
 import org.apache.iotdb.db.engine.StorageEngineV2;
+import org.apache.iotdb.db.engine.cache.BloomFilterCache;
+import org.apache.iotdb.db.engine.cache.ChunkCache;
+import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
 import org.apache.iotdb.db.exception.DataRegionException;
 import org.apache.iotdb.db.exception.sql.SemanticException;
 import org.apache.iotdb.db.metadata.cache.DataNodeSchemaCache;
@@ -484,6 +488,14 @@ public class DataNodeInternalRPCServiceImpl implements IDataNodeRPCService.Iface
     return StorageEngineV2.getInstance().operateFlush(req);
   }
 
+  @Override
+  public TSStatus clearCache(TClearCacheReq req) throws TException {
+    ChunkCache.getInstance().clear();
+    TimeSeriesMetadataCache.getInstance().clear();
+    BloomFilterCache.getInstance().clear();
+    return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+  }
+
   @Override
   public TSStatus setTTL(TSetTTLReq req) throws TException {
     return StorageEngineV2.getInstance().setTTL(req);
diff --git a/thrift-commons/src/main/thrift/common.thrift b/thrift-commons/src/main/thrift/common.thrift
index 4c2ae8ac04..fe38f6bd32 100644
--- a/thrift-commons/src/main/thrift/common.thrift
+++ b/thrift-commons/src/main/thrift/common.thrift
@@ -102,6 +102,10 @@ struct TFlushReq {
    3: optional i32 dataNodeId
 }
 
+struct TClearCacheReq {
+   1: optional i32 dataNodeId
+}
+
 struct TSetTTLReq {
   1: required string storageGroup
   2: required i64 TTL
diff --git a/thrift-confignode/src/main/thrift/confignode.thrift b/thrift-confignode/src/main/thrift/confignode.thrift
index efe52e5c2c..ece12758d3 100644
--- a/thrift-confignode/src/main/thrift/confignode.thrift
+++ b/thrift-confignode/src/main/thrift/confignode.thrift
@@ -431,6 +431,10 @@ service IConfigNodeRPCService {
 
   common.TSStatus flush(common.TFlushReq req)
 
+  /* ClearCache */
+
+  common.TSStatus clearCache(common.TClearCacheReq req)
+
   /* Cluster Tools */
 
   TShowClusterResp showCluster()
diff --git a/thrift/src/main/thrift/datanode.thrift b/thrift/src/main/thrift/datanode.thrift
index 9ce30dc697..e7941e4e1e 100644
--- a/thrift/src/main/thrift/datanode.thrift
+++ b/thrift/src/main/thrift/datanode.thrift
@@ -332,6 +332,8 @@ service IDataNodeRPCService {
 
   common.TSStatus flush(common.TFlushReq req)
 
+  common.TSStatus clearCache(common.TClearCacheReq req)
+
   common.TSStatus setTTL(common.TSetTTLReq req)
   /**
    * configNode will notify all DataNodes when the capacity of the ConfigNodeGroup is expanded or reduced