merge from main
diff --git a/pom.xml b/pom.xml
index 5eac49b..a865740 100644
--- a/pom.xml
+++ b/pom.xml
@@ -92,7 +92,6 @@
         <module>rocketmq-streams-schedule</module>
         <module>rocketmq-streams-lease</module>
         <module>rocketmq-streams-db-operator</module>
-        <module>rocketmq-streams-dbinit</module>
         <module>rocketmq-streams-window</module>
         <module>rocketmq-streams-clients</module>
         <module>rocketmq-streams-channel-rocketmq</module>
@@ -133,7 +132,7 @@
         <groovy.version>2.1.8</groovy.version>
         <disruptor.version>3.2.0</disruptor.version>
         <rocksdbjni.version>6.6.4</rocksdbjni.version>
-        <rocketmq.version>4.5.2</rocketmq.version>
+        <rocketmq.version>4.9.3-SNAPSHOT</rocketmq.version>
         <hyperscan.version>5.4.0-2.0.0</hyperscan.version>
         <platform.version>3.5.2</platform.version>
         <gson.version>2.8.5</gson.version>
@@ -188,6 +187,7 @@
                         <exclude>**/*.out</exclude>
                         <exclude>**/*.properties</exclude>
                         <exclude>docs/**/*</exclude>
+                        <exclude>**/*.sql</exclude>
                     </excludes>
                 </configuration>
             </plugin>
@@ -277,11 +277,6 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.rocketmq</groupId>
-                <artifactId>rocketmq-streams-dbinit</artifactId>
-                <version>${project.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.rocketmq</groupId>
                 <artifactId>rocketmq-streams-clients</artifactId>
                 <version>${project.version}</version>
             </dependency>
@@ -421,6 +416,12 @@
                 <version>${rocketmq.version}</version>
             </dependency>
 
+            <dependency>
+                <groupId>org.apache.rocketmq</groupId>
+                <artifactId>rocketmq-tools</artifactId>
+                <version>${rocketmq.version}</version>
+            </dependency>
+
             <!-- ================================================= -->
             <!-- tool library -->
             <!-- ================================================= -->
@@ -508,12 +509,6 @@
             </dependency>
 
             <dependency>
-                <groupId>mysql</groupId>
-                <artifactId>mysql-connector-java</artifactId>
-                <version>${mysql-connector.version}</version>
-            </dependency>
-
-            <dependency>
                 <groupId>org.quartz-scheduler</groupId>
                 <artifactId>quartz</artifactId>
                 <version>${quartz.version}</version>
diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/sink/RocketMQSink.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/sink/RocketMQSink.java
index e80a7e4..bb6e945 100644
--- a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/sink/RocketMQSink.java
+++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/sink/RocketMQSink.java
@@ -27,6 +27,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
+import org.apache.rocketmq.client.exception.MQClientException;
 import org.apache.rocketmq.client.producer.DefaultMQProducer;
 import org.apache.rocketmq.common.TopicConfig;
 import org.apache.rocketmq.common.message.Message;
@@ -201,6 +202,7 @@
         defaultMQAdminExt.setVipChannelEnabled(false);
         defaultMQAdminExt.setNamesrvAddr(this.getNamesrvAddr());
         defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis()));
+        defaultMQAdminExt.setAdminExtGroup(topic.trim());
         TopicConfig topicConfig = new TopicConfig();
         topicConfig.setReadQueueNums(splitNum);
         topicConfig.setWriteQueueNums(splitNum);
@@ -243,19 +245,17 @@
         List<ISplit> messageQueues = new ArrayList<>();
         try {
 
-            if (messageQueues == null || messageQueues.size() == 0) {
-                List<MessageQueue> metaqQueueSet = producer.fetchPublishMessageQueues(topic);
-                List<ISplit> queueList = new ArrayList<>();
-                for (MessageQueue queue : metaqQueueSet) {
-                    RocketMQMessageQueue rocketMQMessageQueue = new RocketMQMessageQueue(queue);
-                    queueList.add(rocketMQMessageQueue);
+            List<MessageQueue> messageQueueSet = producer.fetchPublishMessageQueues(topic);
+            List<ISplit> queueList = new ArrayList<>();
+            for (MessageQueue queue : messageQueueSet) {
+                RocketMQMessageQueue rocketMQMessageQueue = new RocketMQMessageQueue(queue);
+                queueList.add(rocketMQMessageQueue);
 
-                }
-                Collections.sort(queueList);
-                messageQueues = queueList;
             }
-        } catch (Exception e) {
-            throw new RuntimeException(e);
+            Collections.sort(queueList);
+            messageQueues = queueList;
+        } catch (MQClientException e) {
+            return messageQueues;
         }
 
         return messageQueues;
diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/source/RocketMQSource.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/source/RocketMQSource.java
index 0cdd266..5d60532 100644
--- a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/source/RocketMQSource.java
+++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/source/RocketMQSource.java
@@ -73,7 +73,7 @@
     /**
      * 消息队列命名空间接入点
      */
-    protected String namesrvAddr;
+
 
     protected Long pullIntervalMs;
 
@@ -286,7 +286,7 @@
         if (consumer.getMessageModel() == MessageModel.CLUSTERING) {
             consumer.changeInstanceNameToPID();
         }
-        MQClientInstance mQClientFactory = MQClientManager.getInstance().getAndCreateMQClientInstance(defaultMQPushConsumer.getDefaultMQPushConsumer());
+        MQClientInstance mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(defaultMQPushConsumer.getDefaultMQPushConsumer());
         RemoteBrokerOffsetStore offsetStore = new RemoteBrokerOffsetStore(mQClientFactory, NamespaceUtil.wrapNamespace(consumer.getNamespace(), consumer.getConsumerGroup())) {
             Set<MessageQueue> firstComing = new HashSet<>();
             @Override
@@ -388,14 +388,6 @@
         this.pullIntervalMs = pullIntervalMs;
     }
 
-    public String getNamesrvAddr() {
-        return namesrvAddr;
-    }
-
-    public void setNamesrvAddr(String namesrvAddr) {
-        this.namesrvAddr = namesrvAddr;
-    }
-
     public ConsumeFromWhere getConsumeFromWhere() {
         return consumeFromWhere;
     }
diff --git a/rocketmq-streams-clients/pom.xml b/rocketmq-streams-clients/pom.xml
index 3cee17d..8fb3167 100644
--- a/rocketmq-streams-clients/pom.xml
+++ b/rocketmq-streams-clients/pom.xml
@@ -52,10 +52,6 @@
         </dependency>
         <dependency>
             <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-dbinit</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
             <artifactId>rocketmq-streams-window</artifactId>
         </dependency>
         <dependency>
diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/JoinStream.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/JoinStream.java
index 22c1f2a..8d16150 100644
--- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/JoinStream.java
+++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/JoinStream.java
@@ -142,6 +142,11 @@
         return this;
     }
 
+    public JoinStream setLocalStorageOnly(boolean localStorageOnly) {
+        this.joinWindow.setLocalStorageOnly(localStorageOnly);
+        return this;
+    }
+
     /**
      * 增加条件,用表达式形式表达(leftFieldName,function,rightFieldName)&&({name,==,otherName}||(age,==,age)) 后续再增加结构化的方法 。 后续该方法将下线,推荐使用on
      *
diff --git a/rocketmq-streams-commons/pom.xml b/rocketmq-streams-commons/pom.xml
index 8bb5833..5ea155d 100755
--- a/rocketmq-streams-commons/pom.xml
+++ b/rocketmq-streams-commons/pom.xml
@@ -98,6 +98,10 @@
             <artifactId>re2j</artifactId>
             <version>1.6</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-tools</artifactId>
+        </dependency>
 
 
     </dependencies>
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/AbstractChannel.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/AbstractChannel.java
index bf6634a..313ac7f 100644
--- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/AbstractChannel.java
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/AbstractChannel.java
@@ -240,6 +240,16 @@
     }
 
     @Override
+    public void setNamesrvAddr(String namesrvAddr) {
+        source.setNamesrvAddr(namesrvAddr);
+    }
+
+    @Override
+    public String getNamesrvAddr() {
+        return source.getNamesrvAddr();
+    }
+
+    @Override
     public void atomicSink(ISystemMessage message){
 
     }
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSource.java
index ad675e9..eebcf54 100644
--- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSource.java
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/AbstractSource.java
@@ -18,6 +18,7 @@
 
 import com.alibaba.fastjson.JSONArray;
 import com.alibaba.fastjson.JSONObject;
+
 import java.io.UnsupportedEncodingException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -28,6 +29,7 @@
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.rocketmq.streams.common.batchsystem.BatchFinishMessage;
 import org.apache.rocketmq.streams.common.channel.source.systemmsg.NewSplitMessage;
 import org.apache.rocketmq.streams.common.channel.source.systemmsg.RemoveSplitMessage;
@@ -64,6 +66,7 @@
     protected int maxThread = Runtime.getRuntime().availableProcessors();
     @ENVDependence
     protected String topic = "";
+    protected String namesrvAddr;
     /**
      * 多长时间做一次checkpoint
      */
@@ -134,7 +137,7 @@
      * @return
      */
     public AbstractContext doReceiveMessage(JSONObject message, boolean needSetCheckPoint, String queueId,
-        String offset) {
+                                            String offset) {
         Message msg = createMessage(message, queueId, offset, needSetCheckPoint);
         AbstractContext context = executeMessage(msg);
         return context;
@@ -326,7 +329,7 @@
             messageQueueChangedCheck(channelMessage.getHeader());
         }
 
-        boolean needFlush = channelMessage.getHeader().isSystemMessage() == false && channelMessage.getHeader().isNeedFlush();
+        boolean needFlush = !channelMessage.getHeader().isSystemMessage() && channelMessage.getHeader().isNeedFlush();
 
         if (receiver != null) {
             receiver.doMessage(channelMessage, context);
@@ -374,9 +377,7 @@
      * @param header
      */
     protected void messageQueueChangedCheck(MessageHeader header) {
-        if (supportNewSplitFind() && supportRemoveSplitFind()) {
-            return;
-        }
+
         Set<String> queueIds = new HashSet<>();
         String msgQueueId = header.getQueueId();
         if (StringUtil.isNotEmpty(msgQueueId)) {
@@ -386,8 +387,9 @@
         if (checkpointQueueIds != null) {
             queueIds.addAll(checkpointQueueIds);
         }
+
         Set<String> newQueueIds = new HashSet<>();
-        Set<String> removeQueueIds = new HashSet<>();
+
         for (String queueId : queueIds) {
             if (isNotDataSplit(queueId)) {
                 continue;
@@ -405,7 +407,9 @@
                 }
             }
         }
-        if (!supportNewSplitFind()) {
+
+
+        if (newQueueIds.size() != 0) {
             addNewSplit(newQueueIds);
         }
 
@@ -622,6 +626,16 @@
         this.topic = topic;
     }
 
+    @Override
+    public String getNamesrvAddr() {
+        return this.namesrvAddr;
+    }
+
+    @Override
+    public void setNamesrvAddr(String namesrvAddr) {
+        this.namesrvAddr = namesrvAddr;
+    }
+
     public void setCheckpointTime(long checkpointTime) {
         this.checkpointTime = checkpointTime;
     }
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/ISource.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/ISource.java
index 5dd9940..4b023d5 100644
--- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/ISource.java
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/channel/source/ISource.java
@@ -91,4 +91,8 @@
      */
     String createCheckPointName();
 
+    String getNamesrvAddr();
+
+    void setNamesrvAddr(String namesrvAddr);
+
 }
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/CreateTopicUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/CreateTopicUtil.java
new file mode 100644
index 0000000..90e49f8
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/CreateTopicUtil.java
@@ -0,0 +1,52 @@
+package org.apache.rocketmq.streams.common.utils;
+/*
+ * 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.
+ */
+
+import org.apache.rocketmq.common.TopicConfig;
+import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
+import org.apache.rocketmq.tools.command.CommandUtil;
+
+import java.util.Set;
+
+public class CreateTopicUtil {
+
+    public static boolean create(String clusterName, String topic, int queueNum, String namesrv) {
+        DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt();
+        defaultMQAdminExt.setVipChannelEnabled(false);
+        defaultMQAdminExt.setNamesrvAddr(namesrv);
+        defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis()));
+        defaultMQAdminExt.setAdminExtGroup(topic.trim());
+        TopicConfig topicConfig = new TopicConfig();
+        topicConfig.setReadQueueNums(queueNum);
+        topicConfig.setWriteQueueNums(queueNum);
+        topicConfig.setTopicName(topic.trim());
+
+        try {
+            defaultMQAdminExt.start();
+            Set<String> masterSet = CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName);
+            for (String master : masterSet) {
+                defaultMQAdminExt.createAndUpdateTopicConfig(master, topicConfig);
+            }
+
+            return true;
+        } catch (Throwable t) {
+            throw new RuntimeException(t);
+        } finally {
+            defaultMQAdminExt.shutdown();
+        }
+    }
+}
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLUtil.java
index 7bce3cb..51375c1 100644
--- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLUtil.java
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SQLUtil.java
@@ -244,8 +244,7 @@
         return createInsertSql(metaData, fieldName2Value, null);
     }
 
-    public static String createInsertSql(MetaData metaData, Map<String, Object> fieldName2Value,
-        Boolean containsIdField) {
+    public static String createInsertSql(MetaData metaData, Map<String, Object> fieldName2Value, Boolean containsIdField) {
 
         StringBuilder sql = new StringBuilder(INSERT + " " + metaData.getTableName() + "(");
         StringBuilder fieldSql = new StringBuilder();
@@ -256,10 +255,10 @@
             createInsertValuesSQL(metaData, fieldName2Value, fieldSql, valueSql, containsIdField);
         }
 
-        sql.append(fieldSql.toString());
+        sql.append(fieldSql);
         sql.append(")");
         sql.append(" values");
-        sql.append(valueSql.toString());
+        sql.append(valueSql);
         return sql.toString();
     }
 
diff --git a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SerializeUtil.java b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SerializeUtil.java
index d0cd1f9..c7d21b3 100644
--- a/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SerializeUtil.java
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/SerializeUtil.java
@@ -23,8 +23,10 @@
 import java.io.ObjectOutputStream;
 import java.io.Serializable;
 import java.lang.reflect.Array;
+import java.nio.charset.StandardCharsets;
 import java.util.Optional;
 import java.util.concurrent.atomic.AtomicInteger;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.rocketmq.streams.common.datatype.ArrayDataType;
@@ -84,7 +86,16 @@
      * @param bytes
      */
     public static <T> T deserialize(byte[] bytes) {
-        return deserialize(bytes, new AtomicInteger(0));
+        T result = null;
+        try {
+            result = deserialize(bytes, new AtomicInteger(0));
+        } catch (NullPointerException npe) {
+            if (bytes != null && bytes.length != 0) {
+                String temp = new String(bytes, StandardCharsets.UTF_8);
+                result = (T) temp;
+            }
+        }
+        return result;
     }
 
     public static byte[] serializeByJava(Object object) {
@@ -110,5 +121,4 @@
             return null;
         }
     }
-
 }
diff --git a/rocketmq-streams-db-operator/pom.xml b/rocketmq-streams-db-operator/pom.xml
index ac26139..38b48fd 100755
--- a/rocketmq-streams-db-operator/pom.xml
+++ b/rocketmq-streams-db-operator/pom.xml
@@ -40,8 +40,14 @@
         <dependency>
             <groupId>mysql</groupId>
             <artifactId>mysql-connector-java</artifactId>
+            <version>${mysql-connector.version}</version>
         </dependency>
 
+        <dependency>
+            <groupId>mysql</groupId>
+            <artifactId>mysql-connector-java</artifactId>
+            <version>8.0.19</version>
+        </dependency>
 
     </dependencies>
 </project>
diff --git a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/orm/ORMUtil.java b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/orm/ORMUtil.java
index e4e825d..5d1f1ce 100644
--- a/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/orm/ORMUtil.java
+++ b/rocketmq-streams-db-operator/src/main/java/org/apache/rocketmq/streams/db/driver/orm/ORMUtil.java
@@ -360,37 +360,15 @@
         Object object = values.get(0);
         Map<String, Object> paras = new HashMap<>(16);
         MetaData metaData = createMetaDate(object, paras);
-        boolean containsIdField = false;
-        if (metaData.getIdFieldName() != null) {
-            for (Object o : values) {
-                Object id = ReflectUtil.getDeclaredField(o, metaData.getIdFieldName());
-                if (id == null) {
-                    containsIdField = false;
-                    break;
-                }
-                if (id instanceof Number) {
-                    if (Long.valueOf(id.toString()) == 0) {
-                        containsIdField = false;
-                        break;
-                    }
-                }
-                if (id instanceof String) {
-                    String idStr = (String)id;
-                    if (StringUtil.isEmpty(idStr)) {
-                        containsIdField = false;
-                        break;
-                    }
-                }
-            }
-        }
+
 
         String sql = null;
         if (flag == 0) {
-            sql = SQLUtil.createInsertSql(metaData, paras, containsIdField);
+            sql = SQLUtil.createInsertSql(metaData, paras, false);
         } else if (flag == 1) {
-            sql = SQLUtil.createInsertSql(metaData, paras, containsIdField);
+            sql = SQLUtil.createInsertSql(metaData, paras, false);
         } else if (flag == -1) {
-            sql = SQLUtil.createIgnoreInsertSql(metaData, paras, containsIdField);
+            sql = SQLUtil.createIgnoreInsertSql(metaData, paras, false);
         } else {
             throw new RuntimeException("the flag is not valdate " + flag);
         }
diff --git a/rocketmq-streams-dbinit/pom.xml b/rocketmq-streams-dbinit/pom.xml
deleted file mode 100644
index 40b58a2..0000000
--- a/rocketmq-streams-dbinit/pom.xml
+++ /dev/null
@@ -1,49 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-
-      http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
-  -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <parent>
-        <artifactId>rocketmq-streams</artifactId>
-        <groupId>org.apache.rocketmq</groupId>
-        <version>1.0.2-preview-SNAPSHOT</version>
-    </parent>
-    <modelVersion>4.0.0</modelVersion>
-    <artifactId>rocketmq-streams-dbinit</artifactId>
-    <name>ROCKETMQ STREAMS :: dbinit</name>
-
-    <build>
-        <resources>
-            <resource>
-                <directory>src/main/resources</directory>
-                <includes>
-                    <include>**/*.sql</include>
-                    <include>**/*.properties</include>
-                </includes>
-                <filtering>true</filtering>
-            </resource>
-        </resources>
-    </build>
-
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-db-operator</artifactId>
-        </dependency>
-    </dependencies>
-
-</project>
\ No newline at end of file
diff --git a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegateFactory.java b/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegateFactory.java
deleted file mode 100644
index 649d1cc..0000000
--- a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegateFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.streams.dbinit.mysql.delegate;
-
-import org.apache.rocketmq.streams.common.component.ComponentCreator;
-import org.apache.rocketmq.streams.common.configure.ConfigureFileKey;
-
-public class DBDelegateFactory {
-
-    public static DBDelegate getDelegate() {
-        String dbType = ComponentCreator.getProperties().getProperty(ConfigureFileKey.DB_TYPE);
-        if (dbType == null || "".equalsIgnoreCase(dbType)) {
-            dbType = DBType.DB_MYSQL;
-        }
-        if (DBType.DB_MYSQL.equalsIgnoreCase(dbType)) {
-            return new MysqlDelegate();
-        }
-
-        return new MysqlDelegate();
-    }
-
-    public static DBDelegate getDelegate(String dbType) {
-        if (DBType.DB_MYSQL.equalsIgnoreCase(dbType)) {
-            return new MysqlDelegate();
-        }
-
-        return new MysqlDelegate();
-    }
-}
diff --git a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/MysqlDelegate.java b/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/MysqlDelegate.java
deleted file mode 100644
index 7daa48d..0000000
--- a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/MysqlDelegate.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.streams.dbinit.mysql.delegate;
-
-import java.io.IOException;
-import java.net.URL;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.utils.FileUtil;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
-
-public class MysqlDelegate implements DBDelegate {
-
-    public static final Log LOG = LogFactory.getLog(MysqlDelegate.class);
-
-
-    @Override
-    public void init(String driver, final String url, final String userName,
-                     final String password) {
-        String[] sqls = loadSqls();
-        for (String sql : sqls) {
-            ORMUtil.executeSQL(sql, null, driver, url, userName, password);
-        }
-    }
-
-    @Override
-    public void init() {
-        String[] sqls = loadSqls();
-        for (String sql : sqls) {
-            ORMUtil.executeSQL(sql, null);
-        }
-    }
-
-    private String[] loadSqls() {
-        String[] sqls = null;
-        URL url = this.getClass().getClassLoader().getResource("tables_mysql_innodb.sql");
-        try {
-            String tables = FileUtil.loadFileContent(url.openStream());
-            sqls = tables.split(";");
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Init db sqls : " + tables);
-            }
-        } catch (IOException e) {
-            e.printStackTrace();
-        }
-        return sqls;
-    }
-
-}
diff --git a/rocketmq-streams-dbinit/src/main/resources/tables_mysql_innodb.sql b/rocketmq-streams-dbinit/src/main/resources/tables_mysql_innodb.sql
deleted file mode 100644
index 92605cb..0000000
--- a/rocketmq-streams-dbinit/src/main/resources/tables_mysql_innodb.sql
+++ /dev/null
@@ -1,217 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-CREATE TABLE IF NOT EXISTS  `window_max_value` (
-  `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT,
-  `gmt_create` datetime NOT NULL,
-  `gmt_modified` datetime NOT NULL,
-  `max_offset` varchar(20) NOT NULL,
-  `is_max_offset_long` int(11) DEFAULT NULL,
-  `max_value` bigint(20) unsigned NOT NULL,
-  `max_event_time` bigint(20) unsigned NOT NULL,
-  `msg_key` varchar(256) NOT NULL,
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `uk__ket` (`msg_key`(250)),
-  KEY `idx_modifytime` (`gmt_modified`)
-) ENGINE=InnoDB DEFAULT CHARSET=utf8;
-
-CREATE TABLE IF NOT EXISTS  `window_value` (
-  `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT,
-  `gmt_create` datetime NOT NULL,
-  `gmt_modified` datetime NOT NULL,
-  `start_time` varchar(20) NOT NULL,
-  `end_time` varchar(20) NOT NULL,
-  `max_offset` longtext,
-  `group_by` text,
-  `agg_column_result` longtext,
-  `computed_column_result` longtext,
-  `version` varchar(64) DEFAULT NULL,
-  `name_space` varchar(256) DEFAULT NULL,
-  `configure_name` varchar(256) DEFAULT NULL,
-  `msg_key` varchar(64) NOT NULL,
-  `window_instance_id` varchar(64) NOT NULL,
-  `partition` varchar(512) DEFAULT NULL,
-  `partition_num` bigint(20) DEFAULT NULL,
-  `fire_time` varchar(20) DEFAULT NULL,
-  `update_version` bigint(20) unsigned DEFAULT NULL,
-  `update_flag` bigint(20) DEFAULT NULL,
-  `window_instance_partition_id` varchar(64) DEFAULT NULL,
-  `type` varchar(64) DEFAULT NULL,
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `uk_window_state` (`msg_key`),
-  KEY `idx_window_instance_shuffle` (`window_instance_partition_id`,`partition_num`),
-  KEY `idx_window_instance_firetime` (`window_instance_partition_id`,`fire_time`),
-  KEY `idx_window` (`name_space`(128),`configure_name`(128),`partition`(128))
-) ENGINE=InnoDB DEFAULT CHARSET=utf8;
-
-CREATE TABLE IF NOT EXISTS  `window_task` (
-  `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT,
-  `gmt_create` datetime NOT NULL DEFAULT CURRENT_TIMESTAMP,
-  `gmt_modified` datetime NOT NULL DEFAULT CURRENT_TIMESTAMP,
-  `task_id` varchar(64) NOT NULL,
-  `untreated_flag` int(11) NOT NULL DEFAULT '0',
-  `group_by_value` varchar(1024) NOT NULL,
-  `task_owner` varchar(256) NOT NULL,
-  `task_send_time` datetime DEFAULT NULL,
-  `send_task_msg` text NOT NULL,
-  `msg_send_time` bigint(20) DEFAULT NULL,
-  `name` varchar(128) NOT NULL,
-  `start_time` varchar(20) NOT NULL,
-  `end_time` varchar(20) NOT NULL,
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `uk_taskid` (`task_id`),
-  KEY `idx_flag_modifytime` (`name`,`untreated_flag`,`gmt_modified`)
-) ENGINE=InnoDB DEFAULT CHARSET=utf8;
-
-CREATE TABLE IF NOT EXISTS  `window_instance` (
-  `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT,
-  `gmt_create` datetime NOT NULL,
-  `gmt_modified` datetime NOT NULL,
-  `start_time` varchar(20) NOT NULL,
-  `end_time` varchar(20) NOT NULL,
-  `fire_time` varchar(20) NOT NULL,
-  `window_name` varchar(128) NOT NULL,
-  `window_name_space` varchar(128) NOT NULL,
-  `status` tinyint(4) NOT NULL DEFAULT '0',
-  `version` int(11) DEFAULT '0',
-  `window_instance_key` varchar(128) DEFAULT NULL,
-  `window_instance_name` varchar(128) DEFAULT NULL,
-  `window_Instance_split_name` varchar(128) DEFAULT NULL,
-  `split_id` varchar(128) DEFAULT NULL,
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `uk_window_instance_uniq_index` (`window_instance_key`),
-  KEY `idx_gmt_modified` (`fire_time`,`window_name`,`window_name_space`,`status`),
-  KEY `idx_windowinstance_name` (`window_instance_name`),
-  KEY `idx_windowinstance_split_name` (`window_Instance_split_name`),
-  KEY `idx_windowinstance_split_name_firetime` (`window_Instance_split_name`,`fire_time`)
-) ENGINE=InnoDB DEFAULT CHARSET=utf8;
-
-CREATE TABLE IF NOT EXISTS  `lease_info` (
-  `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT,
-  `gmt_create` datetime NOT NULL,
-  `gmt_modified` datetime NOT NULL,
-  `lease_name` varchar(255) NOT NULL,
-  `lease_user_ip` varchar(255) NOT NULL,
-  `lease_end_time` varchar(255) NOT NULL,
-  `status` int(11) NOT NULL DEFAULT '1',
-  `version` bigint(20) NOT NULL,
-  `candidate_lease_ip` varchar(255) DEFAULT NULL,
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `uk_name` (`lease_name`)
-) ENGINE=InnoDB DEFAULT CHARSET=utf8;
-
-CREATE TABLE IF NOT EXISTS  `dipper_sql_configure` (
-  `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT,
-  `gmt_create` datetime NOT NULL,
-  `gmt_modified` datetime NOT NULL,
-  `namespace` varchar(32) NOT NULL,
-  `type` varchar(32) NOT NULL,
-  `name` varchar(128) NOT NULL,
-  `json_value` longtext NOT NULL,
-  `request_id` varchar(128) NOT NULL,
-  `account_id` varchar(32) NOT NULL,
-  `account_name` varchar(32) NOT NULL,
-  `account_nickname` varchar(32) NOT NULL,
-  `client_ip` varchar(64) NOT NULL,
-  `status` tinyint(3) unsigned NOT NULL DEFAULT '0',
-  `is_publish` int(11) NOT NULL DEFAULT '0',
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `uk_namespace_type_name` (`namespace`,`type`,`name`),
-  KEY `idx_namespace` (`namespace`)
-) ENGINE=InnoDB DEFAULT CHARSET=utf8;
-
-CREATE TABLE IF NOT EXISTS  `dipper_configure` (
-  `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT,
-  `gmt_create` datetime NOT NULL,
-  `gmt_modified` datetime NOT NULL,
-  `namespace` varchar(32) NOT NULL,
-  `type` varchar(32) NOT NULL,
-  `name` varchar(128) NOT NULL,
-  `json_value` text NOT NULL,
-  `request_id` varchar(128) DEFAULT NULL,
-  `account_id` varchar(32) DEFAULT NULL,
-  `account_name` varchar(32) DEFAULT NULL,
-  `account_nickname` varchar(32) DEFAULT NULL,
-  `client_ip` varchar(64) DEFAULT NULL,
-  `status` tinyint(3) unsigned NOT NULL DEFAULT '0',
-  `isPublish` int(1) NOT NULL DEFAULT '0',
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `uk_namespace_type_name` (`namespace`,`type`,`name`),
-  KEY `idx_namespace` (`namespace`)
-) ENGINE=InnoDB DEFAULT CHARSET=utf8;
-
-CREATE TABLE IF NOT EXISTS  `join_right_state` (
-  `id` bigint(20) NOT NULL AUTO_INCREMENT,
-  `gmt_create` datetime DEFAULT NULL,
-  `gmt_modified` datetime DEFAULT NULL,
-  `window_id` bigint(20) DEFAULT NULL,
-  `window_name` varchar(200) DEFAULT NULL,
-  `window_name_space` varchar(45) DEFAULT NULL,
-  `message_id` varchar(200) DEFAULT NULL,
-  `message_key` varchar(32) DEFAULT NULL,
-  `message_time` datetime DEFAULT NULL,
-  `message_body` longtext,
-  `msg_key` varchar(400) DEFAULT NULL,
-  `window_instance_id` varchar(200) DEFAULT NULL,
-  `partition` varchar(200) DEFAULT NULL,
-  `partition_num` bigint(20) DEFAULT NULL,
-  `window_instance_partition_id` varchar(200) DEFAULT NULL,
-  `version` varchar(64) DEFAULT NULL,
-  `update_flag` bigint(20) DEFAULT NULL,
-  `name_space` varchar(256) DEFAULT NULL,
-  `configure_name` varchar(256) DEFAULT NULL,
-  `type` varchar(64) DEFAULT NULL,
-  `name` varchar(64) DEFAULT NULL,
-  `update_version` bigint(20) unsigned DEFAULT NULL,
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `uk_message_id_unique` (`message_id`),
-  KEY `idx_message_key_index` (`message_key`),
-  KEY `idx_gmt_create_index` (`gmt_create`),
-  KEY `idx_window_name_index` (`window_name`(70)),
-  KEY `idx_message_key_gmt_create_index` (`message_key`,`gmt_create`)
-) ENGINE=InnoDB DEFAULT CHARSET=utf8;
-
-CREATE TABLE IF NOT EXISTS  `join_left_state` (
-  `id` bigint(20) NOT NULL AUTO_INCREMENT,
-  `gmt_create` datetime DEFAULT NULL,
-  `gmt_modified` datetime DEFAULT NULL,
-  `window_id` bigint(20) DEFAULT NULL,
-  `window_name` varchar(200) DEFAULT NULL,
-  `window_name_space` varchar(45) DEFAULT NULL,
-  `message_id` varchar(200) DEFAULT NULL,
-  `message_key` varchar(32) DEFAULT NULL,
-  `message_time` datetime DEFAULT NULL,
-  `message_body` longtext,
-  `msg_key` varchar(400) DEFAULT NULL,
-  `window_instance_id` varchar(200) DEFAULT NULL,
-  `partition` varchar(200) DEFAULT NULL,
-  `partition_num` bigint(20) DEFAULT NULL,
-  `window_instance_partition_id` varchar(200) DEFAULT NULL,
-  `version` varchar(64) DEFAULT NULL,
-  `update_flag` bigint(20) DEFAULT NULL,
-  `name_space` varchar(256) DEFAULT NULL,
-  `configure_name` varchar(256) DEFAULT NULL,
-  `type` varchar(64) DEFAULT NULL,
-  `name` varchar(64) DEFAULT NULL,
-  `update_version` bigint(20) unsigned DEFAULT NULL,
-  PRIMARY KEY (`id`),
-  UNIQUE KEY `uk_message_id_unique` (`message_id`),
-  KEY `idx_message_key_index` (`message_key`),
-  KEY `idx_gmt_create_index` (`gmt_create`),
-  KEY `idx_window_name_index` (`window_name`(70)),
-  KEY `idx_message_key_gmt_create_index` (`message_key`,`gmt_create`)
-) ENGINE=InnoDB DEFAULT CHARSET=utf8;
-
diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/PageDimensionExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/PageDimensionExample.java
index 55ed468..d12356a 100644
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/PageDimensionExample.java
+++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/PageDimensionExample.java
@@ -25,6 +25,7 @@
 import org.apache.rocketmq.streams.client.strategy.WindowStrategy;
 import org.apache.rocketmq.streams.client.transform.window.Time;
 import org.apache.rocketmq.streams.client.transform.window.TumblingWindow;
+import org.apache.rocketmq.streams.examples.send.ProducerFromFile;
 import org.junit.Test;
 
 public class PageDimensionExample {
@@ -37,7 +38,7 @@
      * @param args
      */
     public static void main(String[] args) {
-        ProducerFromFile.produce("pageClickData.txt", namesrv, topic);
+        ProducerFromFile.produce("pageClickData.txt", namesrv, topic, true);
 
         try {
             Thread.sleep(1000 * 3);
diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/RocketMQWindowExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/RocketMQWindowExample.java
index e329dff..1ad68bf 100644
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/RocketMQWindowExample.java
+++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/RocketMQWindowExample.java
@@ -23,6 +23,7 @@
 import org.apache.rocketmq.streams.client.strategy.WindowStrategy;
 import org.apache.rocketmq.streams.client.transform.window.Time;
 import org.apache.rocketmq.streams.client.transform.window.TumblingWindow;
+import org.apache.rocketmq.streams.examples.send.ProducerFromFile;
 
 import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS;
 import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_CONSUMER_GROUP_NAME;
@@ -35,7 +36,7 @@
      * 2、rocketmq allow create topic automatically.
      */
     public static void main(String[] args) {
-        ProducerFromFile.produce("data.txt",NAMESRV_ADDRESS, RMQ_TOPIC);
+        ProducerFromFile.produce("data.txt",NAMESRV_ADDRESS, RMQ_TOPIC, true);
 
         try {
             Thread.sleep(1000 * 3);
@@ -60,13 +61,13 @@
                 })
                 //must convert message to json.
                 .map(message -> JSONObject.parseObject((String) message))
-                .window(TumblingWindow.of(Time.seconds(10)))
+                .window(TumblingWindow.of(Time.seconds(5)))
                 .groupBy("ProjectName","LogStore")
                 .sum("OutFlow", "OutFlow")
                 .sum("InFlow", "InFlow")
                 .count("total")
-                .waterMark(5)
-                .setLocalStorageOnly(true)
+                .waterMark(2)
+                .setLocalStorageOnly(false)
                 .toDataSteam()
                 .toPrint(1)
                 .with(WindowStrategy.highPerformance())
diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/UsersDimensionExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/UsersDimensionExample.java
index a6c163d..e63893e 100644
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/UsersDimensionExample.java
+++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/UsersDimensionExample.java
@@ -25,6 +25,7 @@
 import org.apache.rocketmq.streams.client.strategy.WindowStrategy;
 import org.apache.rocketmq.streams.client.transform.window.Time;
 import org.apache.rocketmq.streams.client.transform.window.TumblingWindow;
+import org.apache.rocketmq.streams.examples.send.ProducerFromFile;
 
 public class UsersDimensionExample {
     private static final String topic = "pageClick";
@@ -35,7 +36,7 @@
      * @param args
      */
     public static void main(String[] args) {
-        ProducerFromFile.produce("pageClickData.txt",namesrv, topic);
+        ProducerFromFile.produce("pageClickData.txt",namesrv, topic, true);
 
         try {
             Thread.sleep(1000 * 3);
diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/checkpoint/RemoteCheckpointExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/checkpoint/RemoteCheckpointExample.java
deleted file mode 100644
index 6c931b8..0000000
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/checkpoint/RemoteCheckpointExample.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- *
- *  * Licensed to the Apache Software Foundation (ASF) under one or more
- *  * contributor license agreements.  See the NOTICE file distributed with
- *  * this work for additional information regarding copyright ownership.
- *  * The ASF licenses this file to You under the Apache License, Version 2.0
- *  * (the "License"); you may not use this file except in compliance with
- *  * the License.  You may obtain a copy of the License at
- *  *
- *  *     http://www.apache.org/licenses/LICENSE-2.0
- *  *
- *  * Unless required by applicable law or agreed to in writing, software
- *  * distributed under the License is distributed on an "AS IS" BASIS,
- *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  * See the License for the specific language governing permissions and
- *  * limitations under the License.
- *
- */
-
-package org.apache.rocketmq.streams.examples.checkpoint;
-
-import com.alibaba.fastjson.JSONObject;
-import org.apache.rocketmq.streams.client.StreamBuilder;
-import org.apache.rocketmq.streams.client.source.DataStreamSource;
-import org.apache.rocketmq.streams.client.transform.window.Time;
-import org.apache.rocketmq.streams.client.transform.window.TumblingWindow;
-import org.apache.rocketmq.streams.common.component.ComponentCreator;
-import org.apache.rocketmq.streams.common.configure.ConfigureFileKey;
-import org.apache.rocketmq.streams.dbinit.mysql.delegate.DBDelegate;
-import org.apache.rocketmq.streams.dbinit.mysql.delegate.DBDelegateFactory;
-import org.apache.rocketmq.streams.examples.aggregate.ProducerFromFile;
-
-import static org.apache.rocketmq.streams.db.driver.DriverBuilder.DEFALUT_JDBC_DRIVER;
-import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS;
-import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_CONSUMER_GROUP_NAME;
-import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_TOPIC;
-
-
-public class RemoteCheckpointExample {
-    //replace with your mysql url, database name can be anyone else.
-    private static final String URL = "jdbc:mysql://localhost:3306/rocketmq_streams";
-    // user name of mysql
-    private static final String USER_NAME = "";
-    //password of mysql
-    private static final String PASSWORD = "";
-
-
-    static  {
-        ComponentCreator.getProperties().put(ConfigureFileKey.CONNECT_TYPE, "DB");
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, URL);
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_DRIVER, DEFALUT_JDBC_DRIVER);
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME, USER_NAME);
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD, PASSWORD);
-    }
-
-    public static void main(String[] args) {
-        ProducerFromFile.produce("data.txt",NAMESRV_ADDRESS, RMQ_TOPIC);
-        DBDelegate delegate = DBDelegateFactory.getDelegate();
-        delegate.init();
-
-        try {
-            Thread.sleep(1000 * 3);
-        } catch (InterruptedException e) {
-        }
-        System.out.println("begin streams code.");
-
-        DataStreamSource source = StreamBuilder.dataStream("namespace", "pipeline");
-        source.fromRocketmq(
-                RMQ_TOPIC,
-                RMQ_CONSUMER_GROUP_NAME,
-                false,
-                NAMESRV_ADDRESS)
-                .filter((message) -> {
-                    try {
-                        JSONObject.parseObject((String) message);
-                    } catch (Throwable t) {
-                        // if can not convert to json, discard it.because all operator are base on json.
-                        return false;
-                    }
-                    return true;
-                })
-                //must convert message to json.
-                .map(message -> JSONObject.parseObject((String) message))
-                .window(TumblingWindow.of(Time.seconds(10)))
-                .groupBy("ProjectName","LogStore")
-                .sum("OutFlow", "OutFlow")
-                .sum("InFlow", "InFlow")
-                .count("total")
-                .waterMark(5)
-                .setLocalStorageOnly(false)
-                .toDataSteam()
-                .toPrint(1)
-                .start();
-    }
-
-}
diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/MultiStreamsExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/MultiStreamsExample.java
index 4ab5bca..f5f3d46 100644
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/MultiStreamsExample.java
+++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/MultiStreamsExample.java
@@ -20,14 +20,17 @@
 package org.apache.rocketmq.streams.examples.mutilconsumer;
 
 import com.alibaba.fastjson.JSONObject;
+
 import java.util.Random;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+
 import org.apache.rocketmq.streams.client.StreamBuilder;
 import org.apache.rocketmq.streams.client.source.DataStreamSource;
 import org.apache.rocketmq.streams.client.strategy.WindowStrategy;
 import org.apache.rocketmq.streams.client.transform.window.Time;
 import org.apache.rocketmq.streams.client.transform.window.TumblingWindow;
+import org.apache.rocketmq.streams.examples.send.ProducerFromFile;
 
 import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS;
 import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_CONSUMER_GROUP_NAME;
@@ -43,7 +46,7 @@
         producerPool.submit(new Runnable() {
             @Override
             public void run() {
-                Producer.produceInLoop("data.txt");
+                ProducerFromFile.produceInLoop("data.txt", NAMESRV_ADDRESS, RMQ_TOPIC, 100);
             }
         });
 
@@ -70,7 +73,6 @@
                 try {
                     JSONObject.parseObject((String) message);
                 } catch (Throwable t) {
-                    // if can not convert to json, discard it.because all operator are base on json.
                     return false;
                 }
                 return true;
diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/Producer.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/Producer.java
deleted file mode 100644
index 66903b4..0000000
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/mutilconsumer/Producer.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- *
- *  * Licensed to the Apache Software Foundation (ASF) under one or more
- *  * contributor license agreements.  See the NOTICE file distributed with
- *  * this work for additional information regarding copyright ownership.
- *  * The ASF licenses this file to You under the Apache License, Version 2.0
- *  * (the "License"); you may not use this file except in compliance with
- *  * the License.  You may obtain a copy of the License at
- *  *
- *  *     http://www.apache.org/licenses/LICENSE-2.0
- *  *
- *  * Unless required by applicable law or agreed to in writing, software
- *  * distributed under the License is distributed on an "AS IS" BASIS,
- *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  * See the License for the specific language governing permissions and
- *  * limitations under the License.
- *
- */
-
-package org.apache.rocketmq.streams.examples.mutilconsumer;
-
-import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.rocketmq.client.producer.DefaultMQProducer;
-import org.apache.rocketmq.common.message.Message;
-import org.apache.rocketmq.remoting.common.RemotingHelper;
-import org.apache.rocketmq.streams.examples.aggregate.ProducerFromFile;
-
-import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS;
-import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_TOPIC;
-
-public class Producer {
-    private static final AtomicInteger count = new AtomicInteger(0);
-
-    /**
-     * total produce 1000 data.
-     *
-     * @param fileName
-     */
-    public static void produceInLoop(String fileName) {
-        DefaultMQProducer producer = new DefaultMQProducer("test-group");
-
-        try {
-            producer.setNamesrvAddr(NAMESRV_ADDRESS);
-            producer.start();
-
-            List<String> result = ProducerFromFile.read(fileName);
-
-            for (int i = 0; i < 100; i++) {
-                if (count.get() % 100 == 0) {
-                    System.out.println("already send message: " + count.get());
-                }
-
-                for (String str : result) {
-                    Message msg = new Message(RMQ_TOPIC, "", str.getBytes(RemotingHelper.DEFAULT_CHARSET));
-                    producer.send(msg);
-                    count.getAndIncrement();
-                }
-
-                Thread.sleep(100);
-            }
-
-        } catch (Throwable t) {
-
-        }
-
-    }
-
-}
diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/ProducerFromFile.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/ProducerFromFile.java
similarity index 71%
rename from rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/ProducerFromFile.java
rename to rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/ProducerFromFile.java
index 58d3710..163d811 100644
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/aggregate/ProducerFromFile.java
+++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/ProducerFromFile.java
@@ -17,7 +17,7 @@
  *
  */
 
-package org.apache.rocketmq.streams.examples.aggregate;
+package org.apache.rocketmq.streams.examples.send;
 
 import java.io.BufferedReader;
 import java.io.File;
@@ -26,28 +26,56 @@
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
 import org.apache.rocketmq.client.producer.DefaultMQProducer;
-import org.apache.rocketmq.client.producer.SendResult;
 import org.apache.rocketmq.common.message.Message;
 import org.apache.rocketmq.remoting.common.RemotingHelper;
 
 public class ProducerFromFile {
+    private static final DefaultMQProducer producer = new DefaultMQProducer("test-group");
+    private static final AtomicLong count = new AtomicLong(0);
+    private static boolean init = false;
 
-    public static void produce(String filePath, String nameServ, String topic) {
-        try {
-            DefaultMQProducer producer = new DefaultMQProducer("test-group");
+    private static synchronized void initProducer(String nameServ) throws Throwable {
+        if (!init) {
             producer.setNamesrvAddr(nameServ);
             producer.start();
+            init = true;
+        }
+    }
+
+    public static void produceInLoop(String filePath, String nameServ, String topic, long interval) {
+        while (true) {
+            try {
+                produce(filePath, nameServ, topic, false);
+
+                Thread.sleep(interval);
+
+                if (count.get() % 500 == 0) {
+                    System.out.println("send message num: " + count.get());
+                }
+            } catch (Throwable t) {
+                t.printStackTrace();
+            }
+        }
+    }
+
+    public static void produce(String filePath, String nameServ, String topic, boolean shutdown) {
+        try {
+            initProducer(nameServ);
 
             List<String> result = ProducerFromFile.read(filePath);
 
             for (String str : result) {
                 Message msg = new Message(topic, "", str.getBytes(RemotingHelper.DEFAULT_CHARSET));
-                SendResult sendResult = producer.send(msg);
-                System.out.printf("%s%n", sendResult);
+                producer.send(msg);
+                count.getAndIncrement();
             }
-            //Shut down once the producer instance is not longer in use.
-            producer.shutdown();
+
+            if (shutdown) {
+                producer.shutdown();
+            }
         } catch (Throwable t) {
             t.printStackTrace();
         }
diff --git a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegate.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/SendMessageContinuously.java
similarity index 66%
rename from rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegate.java
rename to rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/SendMessageContinuously.java
index cc8e289..564313a 100644
--- a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegate.java
+++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/SendMessageContinuously.java
@@ -1,3 +1,4 @@
+package org.apache.rocketmq.streams.examples.send;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -15,12 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.streams.dbinit.mysql.delegate;
+import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS;
+import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_TOPIC;
 
-public interface DBDelegate {
-
-    public void init(String driver, String url, String userName,
-        String password);
-
-    public void init();
+public class SendMessageContinuously {
+    public static void main(String[] args) {
+        ProducerFromFile.produceInLoop("data.txt",NAMESRV_ADDRESS, RMQ_TOPIC, 100);
+    }
 }
diff --git a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegate.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/SendMessageOnce.java
similarity index 65%
copy from rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegate.java
copy to rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/SendMessageOnce.java
index cc8e289..70e6fc2 100644
--- a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegate.java
+++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/SendMessageOnce.java
@@ -1,3 +1,4 @@
+package org.apache.rocketmq.streams.examples.send;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -15,12 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.streams.dbinit.mysql.delegate;
+import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS;
+import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_TOPIC;
 
-public interface DBDelegate {
-
-    public void init(String driver, String url, String userName,
-        String password);
-
-    public void init();
+public class SendMessageOnce {
+    public static void main(String[] args) throws InterruptedException {
+        ProducerFromFile.produce("data.txt", NAMESRV_ADDRESS, RMQ_TOPIC, true);
+    }
 }
diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample1.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample1.java
index 9b293d5..1fe5088 100644
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample1.java
+++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample1.java
@@ -18,7 +18,7 @@
 
 import org.apache.rocketmq.streams.client.StreamBuilder;
 import org.apache.rocketmq.streams.client.source.DataStreamSource;
-import org.apache.rocketmq.streams.examples.aggregate.ProducerFromFile;
+import org.apache.rocketmq.streams.examples.send.ProducerFromFile;
 
 import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS;
 import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_CONSUMER_GROUP_NAME;
@@ -27,7 +27,7 @@
 
 public class RocketmqSourceExample1 {
     public static void main(String[] args) {
-        ProducerFromFile.produce("data.txt",NAMESRV_ADDRESS, RMQ_TOPIC);
+        ProducerFromFile.produce("data.txt",NAMESRV_ADDRESS, RMQ_TOPIC, true);
 
         try {
             Thread.sleep(1000 * 3);
diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample2.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample2.java
index f24fe6c..46fdc07 100644
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample2.java
+++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample2.java
@@ -19,7 +19,7 @@
 import java.util.Arrays;
 import org.apache.rocketmq.streams.client.StreamBuilder;
 import org.apache.rocketmq.streams.client.source.DataStreamSource;
-import org.apache.rocketmq.streams.examples.aggregate.ProducerFromFile;
+import org.apache.rocketmq.streams.examples.send.ProducerFromFile;
 
 import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS;
 import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_CONSUMER_GROUP_NAME;
@@ -30,7 +30,7 @@
      * 1、make sure your rocketmq server has been started.
      */
     public static void main(String[] args) {
-        ProducerFromFile.produce("data.txt", NAMESRV_ADDRESS, RMQ_TOPIC);
+        ProducerFromFile.produce("data.txt", NAMESRV_ADDRESS, RMQ_TOPIC, true);
 
         try {
             Thread.sleep(1000 * 3);
diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample3.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample3.java
index 72d94c1..6cf6da7 100644
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample3.java
+++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample3.java
@@ -24,7 +24,7 @@
 import java.util.Set;
 import org.apache.rocketmq.streams.client.StreamBuilder;
 import org.apache.rocketmq.streams.client.source.DataStreamSource;
-import org.apache.rocketmq.streams.examples.aggregate.ProducerFromFile;
+import org.apache.rocketmq.streams.examples.send.ProducerFromFile;
 
 import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS;
 import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_CONSUMER_GROUP_NAME;
@@ -35,7 +35,7 @@
      * 1、make sure your rocketmq server has been started.
      */
     public static void main(String[] args) {
-        ProducerFromFile.produce("data.txt", NAMESRV_ADDRESS, RMQ_TOPIC);
+        ProducerFromFile.produce("data.txt", NAMESRV_ADDRESS, RMQ_TOPIC, true);
 
         try {
             Thread.sleep(1000 * 3);
diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample4.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample4.java
index 5a4cc6e..268f5bf 100644
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample4.java
+++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/RocketmqSourceExample4.java
@@ -20,19 +20,24 @@
 import com.alibaba.fastjson.JSONObject;
 import org.apache.rocketmq.streams.client.StreamBuilder;
 import org.apache.rocketmq.streams.client.transform.DataStream;
-import org.apache.rocketmq.streams.examples.aggregate.ProducerFromFile;
+import org.apache.rocketmq.streams.examples.send.ProducerFromFile;
 
 import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS;
 import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_CONSUMER_GROUP_NAME;
 import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_TOPIC;
 
 public class RocketmqSourceExample4 {
-
+    /**
+     * right stream will replace at the same key, if condition match.
+     * left stream will always be output.
+     * right stream item will be ignored if condition not match.
+     * @param args
+     */
     public static void main(String[] args) {
         System.out.println("send data to rocketmq");
-        ProducerFromFile.produce("joinData-1.txt", NAMESRV_ADDRESS, RMQ_TOPIC);
+        ProducerFromFile.produce("joinData-1.txt", NAMESRV_ADDRESS, RMQ_TOPIC, false);
 
-        ProducerFromFile.produce("joinData-2.txt", NAMESRV_ADDRESS, RMQ_TOPIC + 2);
+        ProducerFromFile.produce("joinData-2.txt", NAMESRV_ADDRESS, RMQ_TOPIC + 2, true);
 
         try {
             Thread.sleep(1000 * 3);
@@ -55,7 +60,7 @@
             return false;
         });
 
-        leftStream.leftJoin(rightStream).setCondition("(ProjectName,==,ProjectName)&(LogStore,==,LogStore)").toDataSteam().toPrint(1).start();
+        leftStream.leftJoin(rightStream).setLocalStorageOnly(false).setCondition("(ProjectName,==,ProjectName)&(LogStore,==,LogStore)").toDataSteam().toPrint(1).start();
 
         System.out.println("consumer end");
     }
diff --git a/rocketmq-streams-examples/src/main/resources/joinData-1.txt b/rocketmq-streams-examples/src/main/resources/joinData-1.txt
index 3017257..bc0497f 100644
--- a/rocketmq-streams-examples/src/main/resources/joinData-1.txt
+++ b/rocketmq-streams-examples/src/main/resources/joinData-1.txt
@@ -1,4 +1,4 @@
-{"InFlow":"1","ProjectName":"ProjectName-0","LogStore":"LogStore-0","OutFlow":"0"}
-{"InFlow":"2","ProjectName":"ProjectName-1","LogStore":"LogStore-1","OutFlow":"1"}
+{"InFlow":"1","ProjectName":"ProjectName-0","LogStore":"LogStore-0","OutFlow":"0","target":"1-1"}
+{"InFlow":"2","ProjectName":"ProjectName-1","LogStore":"LogStore-1","OutFlow":"1","target":"1-2"}
 
 
diff --git a/rocketmq-streams-window/pom.xml b/rocketmq-streams-window/pom.xml
index 8483a4e..40fcbf9 100755
--- a/rocketmq-streams-window/pom.xml
+++ b/rocketmq-streams-window/pom.xml
@@ -48,5 +48,10 @@
             <groupId>org.rocksdb</groupId>
             <artifactId>rocksdbjni</artifactId>
         </dependency>
+
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-tools</artifactId>
+        </dependency>
     </dependencies>
 </project>
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/builder/WindowBuilder.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/builder/WindowBuilder.java
index abe4847..f5f52c3 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/builder/WindowBuilder.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/builder/WindowBuilder.java
@@ -65,7 +65,6 @@
         joinWindow.setSizeInterval(getIntValue(ConfigureFileKey.DIPPER_WINDOW_JOIN_DEFAULT_ITERVA_SIZE, 5));//默认5分钟一个窗口
         joinWindow.setSlideInterval(getIntValue(ConfigureFileKey.DIPPER_WINDOW_JOIN_DEFAULT_ITERVA_SIZE, 5));
         joinWindow.setRetainWindowCount(getIntValue(ConfigureFileKey.DIPPER_WINDOW_JOIN_RETAIN_WINDOW_COUNT, 6));//join的时间窗口是20分钟
-        joinWindow.setWindowType(AbstractWindow.TUMBLE_WINDOW);
         //  joinWindow.setFireDelaySecond(getIntValue(ConfigureFileKey.DIPPER_WINDOW_DEFAULT_FIRE_DELAY_SECOND,5));//延迟5分钟触发
         joinWindow.setTimeFieldName("");
         joinWindow.setWaterMarkMinute(0);
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/debug/DebugWriter.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/debug/DebugWriter.java
index eea8c23..ed6fb46 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/debug/DebugWriter.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/debug/DebugWriter.java
@@ -85,7 +85,7 @@
             JSONObject msg = new JSONObject();
             msg.put("offset", message.getHeader().getOffset());
             msg.put("queueid", message.getMessageBody().getString(message.getHeader().getQueueId()));
-            msg.put("windowInstaceId", instance.createWindowInstanceId());
+            msg.put("windowInstaceId", instance.getWindowInstanceId());
             msgs.add(msg.toJSONString());
         }
         FileUtil.write(logFilePath, msgs, true);
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowInstance.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowInstance.java
index 3ce7b6a..c749bd8 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowInstance.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/model/WindowInstance.java
@@ -16,14 +16,6 @@
  */
 package org.apache.rocketmq.streams.window.model;
 
-import java.io.Serializable;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -32,12 +24,16 @@
 import org.apache.rocketmq.streams.common.utils.CollectionUtil;
 import org.apache.rocketmq.streams.common.utils.DateUtil;
 import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.common.utils.SQLUtil;
 import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
 import org.apache.rocketmq.streams.window.operator.AbstractWindow;
-import org.apache.rocketmq.streams.window.sqlcache.SQLCache;
-import org.apache.rocketmq.streams.window.sqlcache.impl.SQLElement;
+
+import java.io.Serializable;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 /**
  * 具体的窗口实例
@@ -77,13 +73,13 @@
     protected String windowInstanceName;//默认等于窗口名,需要区分不同窗口时使用
 
     /**
-     * splitId,windowNameSpace,windowName,windowInstanceName,windowInstanceName 数据库中存储的是MD5值
+     * splitId,windowNameSpace,windowName,windowInstanceName,windowInstanceName
      */
     protected String windowInstanceSplitName;
     /**
-     * windowInstanceId, splitId,windowNameSpace,windowName,windowInstanceName,windowInstanceName,startTime,endTime" 数据库中存储的是MD5值
+     * splitId,windowNameSpace,windowName,windowInstanceName,startTime,endTime
      */
-    protected String windowInstanceKey;
+    protected String windowInstanceId;
 
     protected transient Boolean isNewWindowInstance = false;//当第一次创建时设置为true,否则设置为false
 
@@ -97,8 +93,6 @@
 
     private static SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
 
-    private static final String SEPARATOR = "_";
-
     protected transient Long lastMaxUpdateTime;//last max update time for session window
 
     public WindowInstance copy() {
@@ -114,7 +108,7 @@
         windowInstance.setGmtModified(new Date());
         windowInstance.setGmtCreate(new Date());
         windowInstance.setWindowInstanceName(this.windowInstanceName);
-        windowInstance.setWindowInstanceKey(this.windowInstanceKey);
+        windowInstance.setWindowInstanceId(this.windowInstanceId);
         windowInstance.setWindowName(this.windowName);
         windowInstance.setWindowNameSpace(this.windowNameSpace);
         windowInstance.setStatus(this.status);
@@ -122,31 +116,11 @@
         return windowInstance;
     }
 
-    /**
-     * 创建window instance的唯一ID
-     *
-     * @return
-     */
-    public String createWindowInstanceId() {
-        return MapKeyUtil.createKey(splitId, windowNameSpace, windowName, windowInstanceName, startTime, endTime);
-    }
-
-    public String createWindowInstanceIdWithoutSplitid() {
-        return MapKeyUtil.createKey(windowNameSpace, windowName, windowInstanceName, startTime, endTime);
-    }
-
     public String createWindowInstanceTriggerId() {
         return MapKeyUtil.createKey(splitId, windowNameSpace, windowName, windowInstanceName, startTime, endTime, fireTime);
     }
 
-    /**
-     * 创建window instance对象列表
-     *
-     * @param window
-     * @param startAndEndTimeList
-     * @param fireTimeList
-     * @return
-     */
+
     public static List<WindowInstance> createWindowInstances(AbstractWindow window,
         List<Pair<String, String>> startAndEndTimeList, List<String> fireTimeList, String queueId) {
         List<WindowInstance> lostInstanceList = new ArrayList<>();
@@ -158,77 +132,10 @@
         return lostInstanceList;
     }
 
-    public String createWindowInstancePartitionId() {
-        return StringUtil.createMD5Str(MapKeyUtil.createKey(windowNameSpace, windowName, windowInstanceName, startTime, endTime, splitId));
-    }
-
-    /**
-     * 触发时间比lastTime小的所有的有效的instance
-     *
-     * @param
-     * @return
-     */
-    public static List<WindowInstance> queryAllWindowInstance(String lastTime, AbstractWindow window,
-        Collection<String> splitIds) {
-        if (window.isLocalStorageOnly() || splitIds == null) {
-            return null;
-        }
-        List<String> splitIdList = new ArrayList<>();
-        splitIdList.addAll(splitIds);
-        String[] splitNames = new String[splitIds.size()];
-        for (int i = 0; i < splitNames.length; i++) {
-            splitNames[i] = MapKeyUtil.createKey(window.getNameSpace(), window.getConfigureName(), splitIdList.get(i));
-            splitNames[i] = StringUtil.createMD5Str(splitNames[i]);
-        }
-        String sql = "select * from window_instance where "
-            + " status =0 and window_instance_split_name in(" + SQLUtil.createInSql(splitNames) + ")";
-
-        List<WindowInstance> dbWindowInstanceList = null;
-        try {
-            dbWindowInstanceList = ORMUtil.queryForList(sql, null, WindowInstance.class);
-        } catch (Exception e) {
-            LOG.error("failed in getting unfired window instances", e);
-        }
-        return dbWindowInstanceList;
-    }
-
-    /**
-     * 清理window
-     *
-     * @param windowInstance
-     */
-    @Deprecated
-    public static void cleanWindow(WindowInstance windowInstance) {
-        clearInstance(windowInstance, null);
-    }
-
-    public static void clearInstance(WindowInstance windowInstance) {
-        clearInstance(windowInstance, null);
-
-    }
-
-    public static void clearInstance(WindowInstance windowInstance, SQLCache sqlCache) {
-        if (windowInstance == null) {
-            return;
-        }
-
-        String deleteInstanceById = "delete from " + ORMUtil.getTableName(WindowInstance.class)
-            + " where window_instance_key ='" + windowInstance.getWindowInstanceKey() + "'";
-        if (sqlCache != null) {
-            sqlCache.addCache(new SQLElement(windowInstance.getSplitId(), windowInstance.createWindowInstanceId(), deleteInstanceById));
-        } else {
-            ORMUtil.executeSQL(deleteInstanceById, null);
-        }
-
-    }
-
     public static Long getOccurTime(AbstractWindow window, IMessage message) {
         Long occurTime = null;
         if (StringUtil.isEmpty(window.getTimeFieldName())) {
-            // occurTime = message.getMessageBody().getLong("time");
-            // if (occurTime == null) {
             occurTime = message.getHeader().getSendTime();
-            // }
         } else {
             try {
                 occurTime = message.getMessageBody().getLong(window.getTimeFieldName());
@@ -255,15 +162,19 @@
      * @return
      * @Param isWindowInstance2DB 如果是秒级窗口,可能windowinstacne不必存表,只在内存保存,可以通过这个标志设置
      */
-    public static List<WindowInstance> getOrCreateWindowInstance(AbstractWindow window, Long occurTime,
-        int timeUnitAdjust, String queueId) {
+    public static List<WindowInstance> getOrCreateWindowInstance(AbstractWindow window, Long occurTime, int timeUnitAdjust, String queueId) {
         int windowSlideInterval = window.getSlideInterval();
         int windowSizeInterval = window.getSizeInterval();
         if (windowSlideInterval == 0) {
             windowSlideInterval = windowSizeInterval;
         }
+
         int waterMarkMinute = window.getWaterMarkMinute();
-        List<Date> windowBeginTimeList = DateUtil.getWindowBeginTime(occurTime, windowSlideInterval * timeUnitAdjust * 1000, windowSizeInterval * timeUnitAdjust * 1000);
+
+        List<Date> windowBeginTimeList = DateUtil.getWindowBeginTime(occurTime,
+                (long) windowSlideInterval * timeUnitAdjust * 1000,
+                (long) windowSizeInterval * timeUnitAdjust * 1000);
+
         List<WindowInstance> instanceList = new ArrayList<>();
         List<Pair<String, String>> lostWindowTimeList = new ArrayList<>();
         List<String> lostFireList = new ArrayList<>();
@@ -277,18 +188,16 @@
                 if (maxEventTime == null || maxEventTime - end.getTime() < 0) {
                     fire = end;
                 } else {
-                    Long nowEventTime = maxEventTime;
-                    List<Date> currentWindowList = DateUtil.getWindowBeginTime(
-                        nowEventTime, windowSlideInterval * timeUnitAdjust * 1000, windowSizeInterval * timeUnitAdjust * 1000);
+                    List<Date> currentWindowList = DateUtil.getWindowBeginTime(maxEventTime,
+                            (long) windowSlideInterval * timeUnitAdjust * 1000,
+                            (long) windowSizeInterval * timeUnitAdjust * 1000);
+
                     if (!CollectionUtil.isEmpty(currentWindowList)) {
                         Date soonBegin = currentWindowList.get(currentWindowList.size() - 1);
-                        Date soonEnd = DateUtil.addDate(TimeUnit.SECONDS, soonBegin, windowSizeInterval * timeUnitAdjust);
-                        Date soonFire = soonEnd;
-                        fire = soonFire;
+                        fire = DateUtil.addDate(TimeUnit.SECONDS, soonBegin, windowSizeInterval * timeUnitAdjust);
                     }
 
-                    // System.out.println(DateUtil.format(fire));
-                    if (fire.getTime() - end.getTime() - waterMarkMinute * timeUnitAdjust * 1000 > 0) {
+                    if (fire.getTime() - end.getTime() - (long) waterMarkMinute * timeUnitAdjust * 1000 > 0) {
                         //超过最大watermark,消息需要丢弃
                         break;
                     }
@@ -300,8 +209,13 @@
                     Date clearWindowInstanceFireTime = DateUtil.addDate(TimeUnit.SECONDS, end, waterMarkMinute * timeUnitAdjust);
                     WindowInstance lastWindowInstance = window.createWindowInstance(DateUtil.format(begin), DateUtil.format(end), DateUtil.format(clearWindowInstanceFireTime), queueId);
                     lastWindowInstance.setCanClearResource(true);
-                    window.registerWindowInstance(lastWindowInstance);
-                    window.getSqlCache().addCache(new SQLElement(queueId, lastWindowInstance.createWindowInstanceId(), ORMUtil.createBatchReplacetSQL(lastWindowInstance)));
+
+                    //和window.getWindowFireSource().registFireWindowInstanceIfNotExist重复了
+//                    window.registerWindowInstance(lastWindowInstance);
+
+                    //保存windowInstance
+                    window.getStorage().putWindowInstance(queueId,window.getNameSpace(), window.getConfigureName(), lastWindowInstance);
+
                     window.getWindowFireSource().registFireWindowInstanceIfNotExist(lastWindowInstance, window);
                 }
 
@@ -319,6 +233,8 @@
             String startTime = DateUtil.format(begin);
             String endTime = DateUtil.format(end);
             String fireTime = DateUtil.format(fire);
+
+            //todo 这里不是都创建出来WindowInstance了吗
             String windowInstanceTriggerId = window.createWindowInstance(startTime, endTime, fireTime, queueId).createWindowInstanceTriggerId();
             WindowInstance windowInstance = window.searchWindowInstance(windowInstanceTriggerId);
             if (windowInstance == null) {
@@ -330,6 +246,7 @@
             }
         }
         List<WindowInstance> lostInstanceList = null;
+        //todo 这里针对lost的都创建一次
         lostInstanceList = WindowInstance.createWindowInstances(window, lostWindowTimeList, lostFireList, queueId);
         instanceList.addAll(lostInstanceList);
         if (CollectionUtil.isNotEmpty(lostInstanceList)) {
@@ -337,8 +254,12 @@
                 List<WindowInstance> emitInstances = createEmitWindowInstance(window, windowInstance);
                 if (emitInstances != null && emitInstances.size() > 0) {
                     for (WindowInstance emitBeforeInstance : emitInstances) {
-                        window.registerWindowInstance(emitBeforeInstance);
-                        window.getSqlCache().addCache(new SQLElement(queueId, emitBeforeInstance.createWindowInstanceId(), ORMUtil.createBatchReplacetSQL(emitBeforeInstance)));
+                        //和window.getWindowFireSource().registFireWindowInstanceIfNotExist重复了
+//                        window.registerWindowInstance(emitBeforeInstance);
+
+                        //保存windowInstance
+                        window.getStorage().putWindowInstance(queueId, window.getNameSpace(), window.getConfigureName(), emitBeforeInstance);
+
                         window.getWindowFireSource().registFireWindowInstanceIfNotExist(emitBeforeInstance, window);
                     }
                 }
@@ -462,9 +383,6 @@
         this.version = version;
     }
 
-    public String getWindowInstanceKey() {
-        return windowInstanceKey;
-    }
 
     public String getWindowInstanceName() {
         return windowInstanceName;
@@ -474,8 +392,17 @@
         this.windowInstanceName = windowInstanceName;
     }
 
-    public void setWindowInstanceKey(String windowInstanceKey) {
-        this.windowInstanceKey = windowInstanceKey;
+    public String getWindowInstanceId() {
+        if (windowInstanceId != null) {
+            return windowInstanceId;
+        }
+        windowInstanceId = MapKeyUtil.createKey(splitId, windowNameSpace, windowName, windowInstanceName, startTime, endTime);
+
+        return windowInstanceId;
+    }
+
+    public void setWindowInstanceId(String windowInstanceId) {
+        this.windowInstanceId = windowInstanceId;
     }
 
     public Boolean isNewWindowInstance() {
@@ -512,12 +439,12 @@
 
     @Override
     public int hashCode() {
-        return createWindowInstanceId().hashCode();
+        return getWindowInstanceId().hashCode();
     }
 
     @Override
     public String toString() {
-        return createWindowInstanceId().toString();
+        return getWindowInstanceId().toString();
     }
 
     public boolean isCanClearResource() {
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/IWindowMaxValueManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/IWindowMaxValueManager.java
deleted file mode 100644
index ae11400..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/IWindowMaxValueManager.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.offset;
-
-import java.util.Map;
-import java.util.Set;
-import org.apache.rocketmq.streams.window.model.WindowInstance;
-
-/**
- * create split sequence number for window instance output result order by split sequence number when windown instance fire
- */
-public interface IWindowMaxValueManager {
-
-    /**
-     * create split sequence number if the generator is not in memory, need load from db or other storage if instance is new ,set the split sequence number = init value
-     *
-     * @param instance
-     * @param splitId
-     * @return plus one on the current max split sequence number
-     */
-    Long incrementAndGetSplitNumber(WindowInstance instance, String splitId);
-
-
-    WindowMaxValue querySplitNum(WindowInstance instance, String splitId);
-    /**
-     */
-    void initMaxSplitNum(WindowInstance windowInstance, Long maxSplitNum);
-
-
-//    /**
-//     * load mutil window instance split's max split num
-//     *
-//     * @param keys
-//     * @return
-//     */
-//    void loadMaxSplitNum(Set<String> keys);
-//
-    void removeKeyPrefixFromLocalCache(Set<String> queueIds);
-
-    /**
-     * save addition WindowMaxValue
-     */
-//    void flush(String... queueIds);
-
-    void resetSplitNum(WindowInstance instance, String splitId);
-
-//    void resetSplitNum(String key);
-//
-    void deleteSplitNum(WindowInstance instance, String splitId);
-
-    /**
-     * save window saved max offset,can filter the less offset
-     * @param name
-     * @param oriQueueId2Offsets
-     */
-    Map<String,WindowMaxValue> saveMaxOffset(boolean isLong, String name, String shuffleId,
-        Map<String, String> oriQueueId2Offsets);
-
-
-
-    Map<String, String> loadOffsets(String name, String shuffleId);
-
-
-    Map<String, WindowMaxValue> queryOffsets(String name, String shuffleId, Set<String> oriQueueIds);
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValue.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValue.java
deleted file mode 100644
index 7173a78..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValue.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.offset;
-
-import java.util.Date;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.rocketmq.streams.common.model.Entity;
-
-/**
- * save windowintance max offset
- */
-public class WindowMaxValue extends Entity {
-    public static long MAX_VALUE_BASE_VALUE=100000000;
-    protected String msgKey;
-    protected AtomicLong maxValue=new AtomicLong(MAX_VALUE_BASE_VALUE);
-    protected String maxOffset="-1";
-    protected boolean isMaxOffsetLong;
-
-    protected AtomicLong maxEventTime=new AtomicLong();//只有window需要
-
-    public WindowMaxValue(){
-        this.gmtModified=new Date();
-        this.gmtCreate=new Date();
-    }
-
-    public Long getMaxEventTime() {
-        return maxEventTime.get();
-    }
-
-    public void setMaxEventTime(Long maxEventTime) {
-        if(maxEventTime==null){
-            return;
-        }
-        this.maxEventTime.set( maxEventTime);
-    }
-
-    public String getMsgKey() {
-        return msgKey;
-    }
-
-    public void setMsgKey(String msgKey) {
-        this.msgKey = msgKey;
-    }
-
-    public Long getMaxValue() {
-        return maxValue.get();
-    }
-
-    public void setMaxValue(Long maxValue) {
-        this.maxValue.set(maxValue);
-    }
-
-    public long comareAndSet(Long eventTime){
-        if(eventTime==null){
-            return maxEventTime.get();
-        }
-        long old=maxEventTime.get();
-        if(old>=eventTime){
-            return old;
-        }
-        boolean updateSuccess=false;
-        while (!updateSuccess){
-            old=maxEventTime.get();
-            if(eventTime>old){
-                updateSuccess= maxEventTime.compareAndSet(old,eventTime);
-            } else {
-                break;
-            }
-        }
-        return maxEventTime.get();
-    }
-
-    public boolean isMaxOffsetLong() {
-        return isMaxOffsetLong;
-    }
-
-    public void setMaxOffsetLong(boolean maxOffsetLong) {
-        isMaxOffsetLong = maxOffsetLong;
-    }
-
-    public String getMaxOffset() {
-        return maxOffset;
-    }
-
-    public void setMaxOffset(String maxOffset) {
-        this.maxOffset = maxOffset;
-    }
-
-    public long incrementAndGetMaxOffset(){
-        return maxValue.incrementAndGet();
-    }
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueManager.java
deleted file mode 100644
index 78bdcd7..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueManager.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.offset;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import org.apache.rocketmq.streams.window.model.WindowInstance;
-import org.apache.rocketmq.streams.window.operator.AbstractWindow;
-import org.apache.rocketmq.streams.window.sqlcache.SQLCache;
-
-public class WindowMaxValueManager implements IWindowMaxValueManager {
-    protected AbstractWindow window;
-    protected Map<String, WindowMaxValueProcessor> windowMaxValueProcessorMap = new HashMap<>();
-    protected transient ExecutorService executorService;
-    protected transient SQLCache sqlCache;
-
-    public WindowMaxValueManager(AbstractWindow window, SQLCache sqlCache) {
-        this.window = window;
-        this.sqlCache = sqlCache;
-        this.executorService = new ThreadPoolExecutor(10, 10,
-            0L, TimeUnit.MILLISECONDS,
-            new LinkedBlockingQueue<Runnable>());
-    }
-
-    protected WindowMaxValueProcessor getOrCreate(String queueId) {
-        WindowMaxValueProcessor windowMaxValueProcessor = windowMaxValueProcessorMap.get(queueId);
-        if (windowMaxValueProcessor == null) {
-            synchronized (this) {
-                windowMaxValueProcessor = windowMaxValueProcessorMap.get(queueId);
-                if (windowMaxValueProcessor == null) {
-                    windowMaxValueProcessor = new WindowMaxValueProcessor(queueId, this.window, sqlCache);
-                    windowMaxValueProcessorMap.put(queueId, windowMaxValueProcessor);
-                }
-            }
-        }
-        return windowMaxValueProcessor;
-    }
-
-    @Override
-    public Long incrementAndGetSplitNumber(WindowInstance instance, String splitId) {
-        return getOrCreate(splitId).incrementAndGetSplitNumber(instance);
-    }
-
-    @Override public WindowMaxValue querySplitNum(WindowInstance instance, String splitId) {
-        return getOrCreate(splitId).querySplitNum(instance);
-    }
-
-    @Override public void initMaxSplitNum(WindowInstance windowInstance, Long maxSplitNum) {
-        getOrCreate(windowInstance.getSplitId()).initMaxSplitNum(windowInstance, maxSplitNum);
-    }
-
-    @Override
-    public void resetSplitNum(WindowInstance instance, String splitId) {
-        getOrCreate(splitId).resetSplitNum(instance);
-    }
-
-    @Override
-    public void deleteSplitNum(WindowInstance instance, String splitId) {
-        getOrCreate(splitId).deleteSplitNum(instance);
-    }
-
-    @Override public Map<String, WindowMaxValue> saveMaxOffset(boolean isLong, String name, String shuffleId,
-        Map<String, String> queueId2Offsets) {
-        return getOrCreate(shuffleId).saveMaxOffset(isLong, name, queueId2Offsets);
-    }
-
-    @Override public Map<String, String> loadOffsets(String name, String shuffleId) {
-        return getOrCreate(shuffleId).loadOffset(name);
-    }
-
-    @Override public Map<String, WindowMaxValue> queryOffsets(String name, String shuffleId, Set<String> oriQueueIds) {
-        return getOrCreate(shuffleId).queryOffsets(name, oriQueueIds);
-    }
-
-    @Override
-    public synchronized void removeKeyPrefixFromLocalCache(Set<String> queueIds) {
-        for (String queueId : queueIds) {
-            getOrCreate(queueId).removeKeyPrefixFromLocalCache();
-        }
-
-    }
-
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueProcessor.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueProcessor.java
deleted file mode 100644
index cf09bf4..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/offset/WindowMaxValueProcessor.java
+++ /dev/null
@@ -1,256 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.offset;
-
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.rocketmq.streams.common.context.MessageOffset;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.common.utils.SQLUtil;
-import org.apache.rocketmq.streams.db.driver.DriverBuilder;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
-import org.apache.rocketmq.streams.window.model.WindowInstance;
-import org.apache.rocketmq.streams.window.operator.AbstractWindow;
-import org.apache.rocketmq.streams.window.sqlcache.SQLCache;
-import org.apache.rocketmq.streams.window.sqlcache.impl.SQLElement;
-
-import static org.apache.rocketmq.streams.window.offset.WindowMaxValue.MAX_VALUE_BASE_VALUE;
-
-public class WindowMaxValueProcessor {
-    protected AbstractWindow window;
-    protected String splitId;
-    protected SQLCache sqlCache;
-
-    public WindowMaxValueProcessor(String splitId, AbstractWindow window,
-        SQLCache sqlCache) {
-        this.splitId = splitId;
-        this.window = window;
-        this.sqlCache = sqlCache;
-    }
-
-    protected Map<String, WindowMaxValue> windowOffsetMap = new HashMap<>();//all window offsets
-
-    public Long incrementAndGetSplitNumber(WindowInstance instance) {
-        String key = createSplitNumberKey(instance, splitId);
-        WindowMaxValue windowMaxValue = queryOrCreateWindowOffset(key, true);
-        return windowMaxValue.incrementAndGetMaxOffset();
-    }
-
-    public WindowMaxValue querySplitNum(WindowInstance instance) {
-        String key = createSplitNumberKey(instance, splitId);
-        return this.windowOffsetMap.get(key);
-    }
-
-    public void resetSplitNum(WindowInstance instance) {
-        String key = createSplitNumberKey(instance, splitId);
-        WindowMaxValue windowMaxValue = queryOrCreateWindowOffset(key, window.isLocalStorageOnly());
-        windowMaxValue.maxValue.set(MAX_VALUE_BASE_VALUE);
-    }
-
-    public void deleteSplitNum(WindowInstance instance) {
-        String key = createSplitNumberKey(instance, splitId);
-        WindowMaxValue windowMaxValue = queryOrCreateWindowOffset(key, window.isLocalStorageOnly());
-
-        this.windowOffsetMap.remove(key);
-        List<String> dels = new ArrayList<>();
-        dels.add(windowMaxValue.getMsgKey());
-        List<Pair<String, String>> likePairList = dels.stream().map(value -> Pair.of("msg_key", value + "%")).collect(Collectors.toList());
-        String sql = "delete from " + ORMUtil.getTableName(WindowMaxValue.class) + " where " + SQLUtil.createLikeSql(likePairList);
-
-        if (sqlCache != null) {
-            sqlCache.addCache(new SQLElement(this.splitId, instance.createWindowInstanceId(), sql));
-        } else {
-            DriverBuilder.createDriver().execute(sql);
-        }
-    }
-
-    public Map<String, WindowMaxValue> saveMaxOffset(boolean isLong, String name, Map<String, String> queueId2Offsets) {
-        Map<String, WindowMaxValue> result = new HashMap<>();
-        Set<String> keys = new HashSet<>();
-        for (String key : queueId2Offsets.keySet()) {
-            keys.add(MapKeyUtil.createKey(name, splitId, key));
-        }
-        Map<String, WindowMaxValue> windowMaxValueMap = queryOrCreateWindowOffset(keys, window.isLocalStorageOnly());
-        for (String queueId : queueId2Offsets.keySet()) {
-            String key = MapKeyUtil.createKey(name, splitId, queueId);
-            WindowMaxValue windowMaxValue = windowMaxValueMap.get(key);
-            String currentOffset = queueId2Offsets.get(queueId);
-            MessageOffset messageOffset = new MessageOffset(currentOffset, isLong);
-            if (windowMaxValue.getMaxOffset().equals("-1")) {
-                windowMaxValue.setMaxOffset(currentOffset);
-            } else {
-                if (messageOffset.greateThan(windowMaxValue.getMaxOffset())) {
-                    windowMaxValue.setMaxOffset(currentOffset);
-                }
-            }
-            windowMaxValue.setMaxOffsetLong(isLong);
-            result.put(key, windowMaxValue);
-        }
-        return result;
-    }
-
-    public Map<String, WindowMaxValue> queryOffsets(String name, Set<String> oriQueueIds) {
-        Map<String, WindowMaxValue> result = new HashMap<>();
-        for (String oriQueueId : oriQueueIds) {
-            String key = MapKeyUtil.createKey(name, splitId, oriQueueId);
-            WindowMaxValue windowMaxValue = windowOffsetMap.get(key);
-            result.put(key, windowMaxValue);
-        }
-        return result;
-    }
-
-    public Map<String, String> loadOffset(String name) {
-        Map<String, String> result = new HashMap<>();
-        if (window.isLocalStorageOnly()) {
-            return result;
-        }
-
-        String keyPrefix = MapKeyUtil.createKey(name, splitId);
-        String sql="select * from "+ ORMUtil.getTableName(WindowMaxValue.class)+ " where msg_key like '"+keyPrefix+"%'";
-        List<WindowMaxValue> windowMaxValues = ORMUtil.queryForList(sql, null, WindowMaxValue.class);
-        if (windowMaxValues == null || windowMaxValues.size() == 0) {
-            return result;
-        }
-
-        for (WindowMaxValue windowMaxValue : windowMaxValues) {
-            if (windowMaxValue != null && !windowMaxValue.getMaxOffset().equals("-1")) {
-                result.put(windowMaxValue.getMsgKey(), windowMaxValue.getMaxOffset());
-            }
-        }
-        return result;
-    }
-
-    public synchronized void removeKeyPrefixFromLocalCache() {
-        Map<String, WindowMaxValue> copy = new HashMap<>();
-        copy.putAll(this.windowOffsetMap);
-        for (String key : copy.keySet()) {
-            if (key.startsWith(this.splitId)) {
-                this.windowOffsetMap.remove(key);
-            }
-        }
-
-    }
-
-    /**
-     * 查询window的总计数器
-     *
-     * @return
-     */
-    protected WindowMaxValue queryOrCreateWindowOffset(String key, boolean onlyLocal) {
-        Set<String> keys = new HashSet<>();
-        keys.add(key);
-        Map<String, WindowMaxValue> windowMaxValueMap = queryOrCreateWindowOffset(keys, onlyLocal);
-        if (windowMaxValueMap == null) {
-            return null;
-        }
-        return windowMaxValueMap.values().iterator().next();
-    }
-
-    /**
-     * 查询window的总计数器
-     *
-     * @return
-     */
-    protected Map<String, WindowMaxValue> queryOrCreateWindowOffset(Set<String> keys, boolean onlyLocal) {
-        Map<String, WindowMaxValue> result = new HashMap<>();
-        if (keys == null) {
-            return result;
-        }
-        List<String> keyNotInLocal = new ArrayList<>();
-        for (String key : keys) {
-            WindowMaxValue windowMaxValue = windowOffsetMap.get(key);
-            if (windowMaxValue != null) {
-                result.put(key, windowMaxValue);
-            } else if (onlyLocal) {
-                windowMaxValue = create(key);
-                result.put(key, windowMaxValue);
-            } else {
-                keyNotInLocal.add(key);
-            }
-        }
-
-        if (onlyLocal) {
-            return result;
-        }
-        if (keyNotInLocal.size() == 0) {
-            return result;
-        }
-        synchronized (this) {
-            List<String> synchKeyNotInLocal = new ArrayList<>();
-            for (String key : keyNotInLocal) {
-                WindowMaxValue windowMaxValue = windowOffsetMap.get(key);
-                if (windowMaxValue != null) {
-                    result.put(key, windowMaxValue);
-                } else {
-                    synchKeyNotInLocal.add(key);
-                }
-            }
-            List<WindowMaxValue> windowMaxValues = null;
-            if (synchKeyNotInLocal.size() > 0) {
-                String sql = "select * from " + ORMUtil.getTableName(WindowMaxValue.class) + " where msg_key in (" + SQLUtil.createInSql(synchKeyNotInLocal) + ")";
-                windowMaxValues = ORMUtil.queryForList(sql, null, WindowMaxValue.class);
-
-            }
-            List<String> keysNotInDB = new ArrayList<>();
-            keysNotInDB.addAll(synchKeyNotInLocal);
-            if (windowMaxValues != null) {
-                for (WindowMaxValue windowMaxValue : windowMaxValues) {
-                    result.put(windowMaxValue.getMsgKey(), windowMaxValue);
-                    keysNotInDB.remove(windowMaxValue.getMsgKey());
-                    windowOffsetMap.put(windowMaxValue.getMsgKey(), windowMaxValue);
-                }
-            }
-            if (keysNotInDB != null && keysNotInDB.size() > 0) {
-                for (String key : keysNotInDB) {
-                    result.put(key, create(key));
-                }
-            }
-        }
-        return result;
-    }
-
-    protected String createSplitNumberKey(WindowInstance instance, String splitId) {
-        String key = MapKeyUtil.createKey(splitId, instance.getWindowInstanceKey());
-        return key;
-    }
-
-    protected WindowMaxValue create(String key) {
-        WindowMaxValue windowMaxValue = new WindowMaxValue();
-        windowMaxValue.setGmtCreate(new Date());
-        windowMaxValue.setGmtModified(new Date());
-        windowMaxValue.setMsgKey(key);
-        windowMaxValue.setMaxValue(MAX_VALUE_BASE_VALUE);
-        windowMaxValue.setMaxEventTime(null);
-        windowOffsetMap.put(key, windowMaxValue);
-        return windowMaxValue;
-    }
-
-    public void initMaxSplitNum(WindowInstance windowInstance, Long maxSplitNum) {
-        String key = createSplitNumberKey(windowInstance, splitId);
-        WindowMaxValue windowMaxValue = create(key);
-        if (maxSplitNum != null) {
-            windowMaxValue.setMaxValue(maxSplitNum);
-        }
-    }
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractShuffleWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractShuffleWindow.java
index 2655bb8..070a40f 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractShuffleWindow.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractShuffleWindow.java
@@ -16,27 +16,29 @@
  */
 package org.apache.rocketmq.streams.window.operator;
 
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.rocketmq.streams.common.channel.source.ISource;
 import org.apache.rocketmq.streams.common.context.AbstractContext;
 import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
 import org.apache.rocketmq.streams.window.model.WindowInstance;
 import org.apache.rocketmq.streams.window.shuffle.ShuffleChannel;
-import org.apache.rocketmq.streams.window.storage.WindowStorage;
+import org.apache.rocketmq.streams.window.storage.rocketmq.DefaultStorage;
+import org.apache.rocketmq.streams.window.storage.rocksdb.RocksdbStorage;
 import org.apache.rocketmq.streams.window.trigger.WindowTrigger;
 
-public abstract class AbstractShuffleWindow extends AbstractWindow {
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
 
+public abstract class AbstractShuffleWindow extends AbstractWindow {
+    private static final String PREFIX = "windowStates";
     protected transient ShuffleChannel shuffleChannel;
     protected transient AtomicBoolean hasCreated = new AtomicBoolean(false);
 
+
     @Override
     protected boolean initConfigurable() {
-        storage = new WindowStorage();
-        storage.setLocalStorageOnly(isLocalStorageOnly);
         return super.initConfigurable();
     }
 
@@ -50,9 +52,28 @@
             this.shuffleChannel.init();
             windowCache.setBatchSize(5000);
             windowCache.setShuffleChannel(shuffleChannel);
+
+            initStorage();
         }
     }
 
+    private void initStorage() {
+        ISource source = this.getFireReceiver().getPipeline().getSource();
+
+        String sourceTopic = source.getTopic();
+        String namesrvAddr = source.getNamesrvAddr();
+
+
+        String stateTopic = createStateTopic(PREFIX, sourceTopic);
+        String groupId = createStr(PREFIX);
+
+        int size = this.shuffleChannel.getQueueList().size();
+
+        RocksdbStorage rocksdbStorage = new RocksdbStorage();
+        this.storage = new DefaultStorage(stateTopic, groupId, namesrvAddr,
+                                            size, isLocalStorageOnly, rocksdbStorage);
+    }
+
     @Override
     public AbstractContext<IMessage> doMessage(IMessage message, AbstractContext context) {
         shuffleChannel.startChannel();
@@ -60,12 +81,12 @@
     }
 
     @Override
-    public int fireWindowInstance(WindowInstance windowInstance, Map<String, String> queueId2Offset) {
+    public int fireWindowInstance(WindowInstance windowInstance) {
         Set<String> splitIds = new HashSet<>();
         splitIds.add(windowInstance.getSplitId());
         shuffleChannel.flush(splitIds);
-        int fireCount = fireWindowInstance(windowInstance, windowInstance.getSplitId(), queueId2Offset);
-        return fireCount;
+
+        return doFireWindowInstance(windowInstance);
     }
 
     /**
@@ -81,8 +102,42 @@
      *
      * @param instance
      */
-    protected abstract int fireWindowInstance(WindowInstance instance, String queueId,
-        Map<String, String> queueId2Offset);
+    protected abstract int doFireWindowInstance(WindowInstance instance);
 
     public abstract void clearCache(String queueId);
+
+    private String createStateTopic(String prefix, String topic) {
+        StringBuilder builder = new StringBuilder();
+        builder.append(prefix);
+        builder.append("_");
+
+        builder.append(topic);
+        builder.append("_");
+
+        String namespace = this.getNameSpace().replaceAll("\\.", "_");
+        builder.append(namespace);
+        builder.append("_");
+
+        String configureName = this.getConfigureName().replaceAll("\\.", "_").replaceAll(";", "_");
+        builder.append(configureName);
+
+        return builder.toString();
+    }
+
+    private String createStr(String prefix) {
+        String temp = MapKeyUtil.createKey(this.getNameSpace(), this.getConfigureName(), this.getUpdateFlag() + "");
+        String result = temp.replaceAll("\\.", "_").replaceAll(";", "_");
+
+        StringBuilder builder = new StringBuilder();
+        builder.append(prefix);
+        builder.append("_");
+        builder.append(result);
+
+        return builder.toString();
+    }
+
+    public ShuffleChannel getShuffleChannel() {
+        return shuffleChannel;
+    }
+
 }
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractWindow.java
index 72028db..49d79c5 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractWindow.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/AbstractWindow.java
@@ -17,16 +17,6 @@
 package org.apache.rocketmq.streams.window.operator;
 
 import com.alibaba.fastjson.JSONObject;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.stream.Collectors;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -42,14 +32,7 @@
 import org.apache.rocketmq.streams.common.topology.model.IWindow;
 import org.apache.rocketmq.streams.common.topology.stages.WindowChainStage;
 import org.apache.rocketmq.streams.common.topology.stages.udf.IReducer;
-import org.apache.rocketmq.streams.common.utils.Base64Utils;
-import org.apache.rocketmq.streams.common.utils.CollectionUtil;
-import org.apache.rocketmq.streams.common.utils.DateUtil;
-import org.apache.rocketmq.streams.common.utils.InstantiationUtil;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.common.utils.TraceUtil;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
+import org.apache.rocketmq.streams.common.utils.*;
 import org.apache.rocketmq.streams.filter.builder.ExpressionBuilder;
 import org.apache.rocketmq.streams.script.operator.expression.ScriptExpression;
 import org.apache.rocketmq.streams.script.operator.impl.AggregationScript;
@@ -63,13 +46,15 @@
 import org.apache.rocketmq.streams.window.model.FunctionExecutor;
 import org.apache.rocketmq.streams.window.model.WindowCache;
 import org.apache.rocketmq.streams.window.model.WindowInstance;
-import org.apache.rocketmq.streams.window.offset.IWindowMaxValueManager;
-import org.apache.rocketmq.streams.window.offset.WindowMaxValueManager;
-import org.apache.rocketmq.streams.window.sqlcache.SQLCache;
 import org.apache.rocketmq.streams.window.state.impl.WindowValue;
-import org.apache.rocketmq.streams.window.storage.WindowStorage;
+import org.apache.rocketmq.streams.window.storage.IStorage;
 import org.apache.rocketmq.streams.window.trigger.WindowTrigger;
 
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
 /**
  * window definition in the pipeline, created by user's configure in WindowChainStage
  */
@@ -78,11 +63,6 @@
     protected static final Log LOG = LogFactory.getLog(AbstractWindow.class);
 
     /**
-     * tumble or hop window 目前不再使用了
-     */
-    protected String windowType;
-
-    /**
      * 用消息中的哪个字段做时间字段
      */
     protected String timeFieldName;
@@ -157,6 +137,8 @@
     protected boolean isLocalStorageOnly = true;//是否只用本地存储,可以提高性能,但不保证可靠性
     protected String reduceSerializeValue;//用户自定义的operator的序列化字节数组,做了base64解码
     protected transient IReducer reducer;
+
+    protected transient Long maxPartitionNum = 100000000L;
     /**
      * the computed column and it's process of computing
      */
@@ -182,22 +164,12 @@
      */
     protected transient String WINDOW_NAME;
 
-    /**
-     * 内部使用,定期检查窗口有没有触发
-     */
-    //protected transient ScheduledExecutorService fireWindowInstanceChecker =new ScheduledThreadPoolExecutor(3);
-
-    // protected transient ExecutorService deleteService = Executors.newSingleThreadExecutor();
 
     protected volatile transient WindowCache windowCache;
-    protected transient WindowStorage storage;
+    protected transient IStorage storage;
     protected transient WindowTrigger windowFireSource;
-    protected transient SQLCache sqlCache;
     protected transient EventTimeManager eventTimeManager;
 
-    //create and save window instacne max partitionNum and window max eventTime
-    protected transient IWindowMaxValueManager windowMaxValueManager;
-
     public AbstractWindow() {
         setType(IWindow.TYPE);
     }
@@ -205,13 +177,8 @@
     @Override
     protected boolean initConfigurable() {
         boolean success = super.initConfigurable();
-        /**
-         * 如果没有db配置,不开启远程存储服务
-         */
-        if (!ORMUtil.hasConfigueDB()) {
-            isLocalStorageOnly = true;
-        }
-        sqlCache = new SQLCache(isLocalStorageOnly);
+
+
         AbstractWindow window = this;
         windowCache = new WindowCache() {
 
@@ -233,7 +200,7 @@
             reducer = InstantiationUtil.deserializeObject(bytes);
         }
         eventTimeManager = new EventTimeManager();
-        windowMaxValueManager = new WindowMaxValueManager(this, sqlCache);
+
 
         return success;
     }
@@ -286,15 +253,13 @@
         windowInstance.setWindowInstanceName(createWindowInstanceName(startTime, endTime, fireTime));
         windowInstance.setWindowName(getConfigureName());
         windowInstance.setWindowNameSpace(getNameSpace());
-        String windowInstanceId = windowInstance.createWindowInstanceId();
-        String dbWindowInstanceId = StringUtil.createMD5Str(windowInstanceId);
-        windowInstance.setWindowInstanceKey(dbWindowInstanceId);
+        windowInstance.setWindowInstanceId(windowInstance.getWindowInstanceId());
         if (fireMode == 2) {
             windowInstance.setCanClearResource(false);
         } else {
             windowInstance.setCanClearResource(true);
         }
-        windowInstance.setWindowInstanceSplitName(StringUtil.createMD5Str(MapKeyUtil.createKey(getNameSpace(), getConfigureName(), splitId)));
+        windowInstance.setWindowInstanceSplitName(MapKeyUtil.createKey(getNameSpace(), getConfigureName(), splitId));
         windowInstance.setNewWindowInstance(true);
         return windowInstance;
     }
@@ -318,15 +283,22 @@
      * @param shuffleId
      * @return
      */
-
     public long incrementAndGetSplitNumber(WindowInstance instance, String shuffleId) {
-        long maxValue = windowMaxValueManager.incrementAndGetSplitNumber(instance, shuffleId);
-        return maxValue;
+        //instance.getWindowInstanceKey() 和shuffled组成key查询是否有相同,有的话+1,没有的话从100000000开始
+        Long maxPartitionNum = this.storage.getMaxPartitionNum(shuffleId, instance.getWindowInstanceId());
+        if (maxPartitionNum == null) {
+            maxPartitionNum = this.maxPartitionNum;
+        } else {
+            maxPartitionNum += 1;
+        }
+
+        this.storage.putMaxPartitionNum(shuffleId, instance.getWindowInstanceId(), maxPartitionNum);
+
+        return maxPartitionNum;
     }
 
-    public abstract Class getWindowBaseValueClass();
 
-    public abstract int fireWindowInstance(WindowInstance windowInstance, Map<String, String> queueId2Offset);
+    public abstract int fireWindowInstance(WindowInstance windowInstance);
 
     /**
      * 计算每条记录的group by值,对于groupby分组,里面任何字段不能为null值,如果为null值,这条记录会被忽略
@@ -338,15 +310,13 @@
         if (StringUtil.isEmpty(groupByFieldName)) {
             return null;
         }
+
         JSONObject msg = message.getMessageBody();
         String[] fieldNames = groupByFieldName.split(";");
         String[] values = new String[fieldNames.length];
-        boolean isFirst = true;
+
         int i = 0;
         for (String filedName : fieldNames) {
-            if (isFirst) {
-                isFirst = false;
-            }
             String value = msg.getString(filedName);
             values[i] = value;
             i++;
@@ -354,8 +324,6 @@
         return MapKeyUtil.createKey(values);
     }
 
-    public abstract void clearFireWindowInstance(WindowInstance windowInstance);
-
     public void clearFire(WindowInstance windowInstance) {
         if (windowInstance == null) {
             return;
@@ -363,6 +331,8 @@
         clearFireWindowInstance(windowInstance);
     }
 
+    public abstract void clearFireWindowInstance(WindowInstance windowInstance);
+
     /**
      * init the function executor TODO: 1) function executor may be parsed in parser module;
      */
@@ -576,13 +546,6 @@
         sizeVariable = variableName;
     }
 
-    public String getWindowType() {
-        return windowType;
-    }
-
-    public void setWindowType(String windowType) {
-        this.windowType = windowType;
-    }
 
     public String getTimeFieldName() {
         return timeFieldName;
@@ -743,7 +706,7 @@
     }
 
     public void removeInstanceFromMap(WindowInstance windowInstance) {
-        this.windowInstanceMap.remove(windowInstance.createWindowInstanceId());
+        this.windowInstanceMap.remove(windowInstance.getWindowInstanceId());
 
     }
 
@@ -757,7 +720,7 @@
         return windowCache;
     }
 
-    public WindowStorage getStorage() {
+    public IStorage getStorage() {
         return storage;
     }
 
@@ -765,9 +728,6 @@
         return windowFireSource;
     }
 
-    public IWindowMaxValueManager getWindowMaxValueManager() {
-        return windowMaxValueManager;
-    }
 
     public Long getMsgMaxGapSecond() {
         return msgMaxGapSecond;
@@ -781,16 +741,6 @@
         return eventTimeManager;
     }
 
-    public SQLCache getSqlCache() {
-        return sqlCache;
-    }
-
-    public void initWindowInstanceMaxSplitNum(WindowInstance instance) {
-        getWindowMaxValueManager().initMaxSplitNum(instance, queryWindowInstanceMaxSplitNum(instance));
-    }
-
-    protected abstract Long queryWindowInstanceMaxSplitNum(WindowInstance instance);
-
     public String getHavingExpression() {
         return havingExpression;
     }
@@ -823,5 +773,13 @@
         this.maxDelay = maxDelay;
     }
 
+    public Long getMaxPartitionNum() {
+        return maxPartitionNum;
+    }
+
+    public void setMaxPartitionNum(Long maxPartitionNum) {
+        this.maxPartitionNum = maxPartitionNum;
+    }
+
     public abstract boolean supportBatchMsgFinish();
 }
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java
index 7f0f982..fb9b4dc 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/OverWindow.java
@@ -18,13 +18,14 @@
 
 import java.util.Date;
 import java.util.List;
-import java.util.Map;
+
 import org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV;
 import org.apache.rocketmq.streams.common.context.AbstractContext;
 import org.apache.rocketmq.streams.common.context.IMessage;
 import org.apache.rocketmq.streams.common.utils.DateUtil;
 import org.apache.rocketmq.streams.window.model.WindowInstance;
 import org.apache.rocketmq.streams.window.operator.AbstractWindow;
+import org.apache.rocketmq.streams.window.storage.WindowType;
 
 /**
  * 只支持 时间去重的场景,日志是按系统时间顺序,所以不落盘。需要设置groupByFieldName和rowNumerName字段
@@ -132,9 +133,6 @@
         return true;
     }
 
-    @Override protected Long queryWindowInstanceMaxSplitNum(WindowInstance instance) {
-        return null;
-    }
 
     @Override public boolean supportBatchMsgFinish() {
         return false;
@@ -146,12 +144,7 @@
     }
 
     @Override
-    public Class getWindowBaseValueClass() {
-        return null;
-    }
-
-    @Override
-    public int fireWindowInstance(WindowInstance windowInstance, Map<String, String> queueId2Offsets) {
+    public int fireWindowInstance(WindowInstance windowInstance) {
         return 0;
     }
 
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java
index e9aa617..3b85e19 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/SessionOperator.java
@@ -29,6 +29,7 @@
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
+
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -36,13 +37,14 @@
 import org.apache.rocketmq.streams.common.utils.Base64Utils;
 import org.apache.rocketmq.streams.common.utils.CollectionUtil;
 import org.apache.rocketmq.streams.common.utils.DateUtil;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
 import org.apache.rocketmq.streams.common.utils.StringUtil;
 import org.apache.rocketmq.streams.common.utils.TraceUtil;
 import org.apache.rocketmq.streams.window.model.WindowInstance;
 import org.apache.rocketmq.streams.window.state.WindowBaseValue;
 import org.apache.rocketmq.streams.window.state.impl.WindowValue;
-import org.apache.rocketmq.streams.window.storage.WindowStorage.WindowBaseValueIterator;
+import org.apache.rocketmq.streams.window.storage.IteratorWrap;
+import org.apache.rocketmq.streams.window.storage.RocksdbIterator;
+import org.apache.rocketmq.streams.window.storage.WindowType;
 
 /**
  * an implementation of session window to save extra memory for different group by window instances
@@ -59,8 +61,6 @@
 
     private static final String SESSION_DATETIME_PATTERN = "yyyy-MM-dd HH:mm:ss";
 
-    private static final String ORDER_BY_FIRE_TIME_PREFIX = "_order_by_fire_time_";
-
     /**
      * 会话窗口的超时时间,时间单位时秒,默认10分钟
      */
@@ -76,17 +76,9 @@
         this.sessionTimeOut = Optional.ofNullable(timeout).orElse(sessionTimeOut);
     }
 
-    public int getSessionTimeOut() {
-        return sessionTimeOut;
-    }
-
-    public void setSessionTimeOut(int sessionTimeOut) {
-        this.sessionTimeOut = sessionTimeOut;
-    }
 
     @Override
     protected boolean initConfigurable() {
-        //
         return super.initConfigurable();
     }
 
@@ -100,7 +92,7 @@
     @Override
     public List<WindowInstance> queryOrCreateWindowInstance(IMessage message, String queueId) {
         WindowInstance instance = createWindowInstance(SESSION_WINDOW_BEGIN_TIME, SESSION_WINDOW_END_TIME, null, queueId);
-        String windowInstanceId = instance.createWindowInstanceId();
+        String windowInstanceId = instance.getWindowInstanceId();
         WindowInstance existWindowInstance = searchWindowInstance(windowInstanceId);
         if (existWindowInstance == null) {
             Pair<Date, Date> startEndPair = getSessionTime(message);
@@ -123,8 +115,9 @@
         }};
     }
 
-    @Override public WindowInstance registerWindowInstance(WindowInstance windowInstance) {
-        return super.registerWindowInstance(windowInstance.createWindowInstanceId(), windowInstance);
+    @Override
+    public WindowInstance registerWindowInstance(WindowInstance windowInstance) {
+        return super.registerWindowInstance(windowInstance.getWindowInstanceId(), windowInstance);
     }
 
     @Override
@@ -147,8 +140,36 @@
                 String storeKey = createStoreKey(queueId, groupValue, instance);
                 value2StoreMap.put(groupValue, storeKey);
             }
-            Map<String, List<WindowValue>> storeValueMap = storage.multiGetList(WindowValue.class, new ArrayList<>(value2StoreMap.values()));
-            //
+
+            List<WindowBaseValue> windowBaseValue =new ArrayList<>();
+
+            RocksdbIterator<WindowBaseValue> rocksdbIterator = storage.getWindowBaseValue(instance.getSplitId(),
+                    instance.getWindowInstanceId(), WindowType.SESSION_WINDOW, null);
+            while (rocksdbIterator.hasNext()) {
+                IteratorWrap<WindowBaseValue> next = rocksdbIterator.next();
+                windowBaseValue.add(next.getData());
+            }
+
+            //1、按照storeKey过滤
+            //2、将WindowBaseValue转化成WindowValue
+            List<String> storeKeys = new ArrayList<>();
+            for (String groupValue : groupBy.keySet()) {
+                String storeKey = createStoreKey(queueId, groupValue, instance);
+                storeKeys.add(storeKey);
+            }
+
+            Map<String, List<WindowValue>> storeValueMap = windowBaseValue.stream()
+                    .map((value) -> (WindowValue) value)
+                    .filter((value) -> {
+                        for (String storeKey : storeKeys) {
+                            if (storeKey.equalsIgnoreCase(value.getMsgKey())) {
+                                return true;
+                            }
+                        }
+                        return false;
+                    }).collect(Collectors.groupingBy(WindowValue::getMsgKey));
+
+
             Iterator<Map.Entry<String, List<IMessage>>> iterator = groupBy.entrySet().iterator();
             Map<String, List<WindowValue>> resultMap = new HashMap<>(groupSize);
             while (iterator.hasNext()) {
@@ -162,7 +183,6 @@
                     id2ValueMap.put(value.getPartitionNum(), value);
                 }
                 for (IMessage message : groupMessageList) {
-                    //
                     WindowValue windowValue = queryOrCreateWindowValue(instance, queueId, groupValue, message, valueList, storeKey);
                     windowValue.calculate(this, message);
                     //region trace
@@ -178,7 +198,7 @@
                     //endregion
                     id2ValueMap.put(windowValue.getPartitionNum(), windowValue);
                 }
-                //merge values
+                //merge values,
                 List<WindowValue> groupValueList = mergeWindowValue(new ArrayList<>(id2ValueMap.values()), instance, queueId);
                 resultMap.put(storeKey, groupValueList);
             }
@@ -200,7 +220,11 @@
      * @return
      */
     private WindowValue queryOrCreateWindowValue(WindowInstance windowInstance, String queueId, String groupByValue,
-        IMessage message, List<WindowValue> valueList, String storeKey) {
+                                                 IMessage message, List<WindowValue> valueList, String storeKey) {
+        //
+        if (CollectionUtil.isEmpty(valueList)) {
+            return createWindowValue(queueId, groupByValue, windowInstance, message, storeKey);
+        }
         //put keys to be deleted here and delete them at last
         List<String> deletePrefixKeyList = new ArrayList<>();
         //
@@ -210,35 +234,18 @@
             Pair<Date, Date> startEndPair = getSessionTime(message);
             Date messageBegin = startEndPair.getLeft();
             Date messageEnd = startEndPair.getRight();
-            if (messageBegin.compareTo(sessionBegin) < 0 && messageEnd.compareTo(sessionEnd) <= 0) {
-                sessionBegin = messageBegin;
-                value.setStartTime(DateUtil.format(sessionBegin, SESSION_DATETIME_PATTERN));
-                return value;
-            } else if (messageBegin.compareTo(sessionBegin) >= 0 && messageEnd.compareTo(sessionEnd) <= 0) {
-                return value;
-            } else if (messageBegin.compareTo(sessionBegin) >= 0 && messageBegin.compareTo(sessionEnd) < 0 && messageEnd.compareTo(sessionEnd) > 0) {
+            if (messageBegin.compareTo(sessionBegin) >= 0 && messageBegin.compareTo(sessionEnd) < 0) {
                 sessionEnd = messageEnd;
-                //clean older storage as sort field 'fireTime' changed
-                String existPartitionNumKey = createPrefixKey(value, windowInstance, queueId);
-                deletePrefixKeyList.add(existPartitionNumKey);
-                deletePrefixValue(deletePrefixKeyList);
-                //
                 Date sessionFire = DateUtil.addDate(TimeUnit.SECONDS, sessionEnd, waterMarkMinute * timeUnitAdjust);
                 value.setEndTime(DateUtil.format(sessionEnd, SESSION_DATETIME_PATTERN));
+                //clean order storage as sort field 'fireTime' changed
+                deleteMergeWindow(windowInstance.getWindowInstanceId(), value.getPartition(), value.getFireTime(), value.getPartitionNum(), value.getGroupBy());
+                //
                 value.setFireTime(DateUtil.format(sessionFire, SESSION_DATETIME_PATTERN));
                 return value;
-            } else if (messageBegin.compareTo(sessionBegin) < 0 && messageEnd.compareTo(sessionEnd) > 0) {
+            } else if (messageBegin.compareTo(sessionBegin) < 0 && messageEnd.compareTo(sessionBegin) > 0) {
                 sessionBegin = messageBegin;
-                sessionEnd = messageEnd;
-                //clean older storage as sort field 'fireTime' changed
-                String existPartitionNumKey = createPrefixKey(value, windowInstance, queueId);
-                deletePrefixKeyList.add(existPartitionNumKey);
-                deletePrefixValue(deletePrefixKeyList);
-                //
                 value.setStartTime(DateUtil.format(sessionBegin, SESSION_DATETIME_PATTERN));
-                value.setEndTime(DateUtil.format(sessionEnd, SESSION_DATETIME_PATTERN));
-                Date sessionFire = DateUtil.addDate(TimeUnit.SECONDS, sessionEnd, waterMarkMinute * timeUnitAdjust);
-                value.setFireTime(DateUtil.format(sessionFire, SESSION_DATETIME_PATTERN));
                 return value;
             }
         }
@@ -249,7 +256,7 @@
     }
 
     private List<WindowValue> mergeWindowValue(List<WindowValue> allValueList, WindowInstance windowInstance,
-        String queueId) {
+                                               String queueId) {
         if (allValueList.size() <= 1) {
             return allValueList;
         }
@@ -267,7 +274,7 @@
             WindowValue outValue = allValueList.get(outIndex);
             for (int inIndex = outIndex + 1; inIndex < allValueList.size(); inIndex++) {
                 WindowValue inValue = allValueList.get(inIndex);
-                if (inValue.getStartTime().compareTo(outValue.getEndTime()) <= 0) {
+                if (inValue.getFireTime().compareTo(outValue.getEndTime()) <= 0) {
                     deleteValueMap.put(inIndex, outIndex);
                     outValue.setEndTime(outValue.getEndTime().compareTo(inValue.getEndTime()) <= 0 ? inValue.getEndTime() : outValue.getEndTime());
                     outValue.setFireTime(outValue.getFireTime().compareTo(inValue.getFireTime()) <= 0 ? inValue.getFireTime() : outValue.getFireTime());
@@ -288,20 +295,34 @@
             theValue.setAggColumnResult(tempValue.getAggColumnResult());
             resultList.add(theValue);
         }
-        List<String> prefixKeyList = deleteValueMap.keySet().stream().map(index -> createPrefixKey(allValueList.get(index), windowInstance, queueId)).collect(Collectors.toList());
-        deletePrefixValue(prefixKeyList);
+
+        deleteValueMap.keySet().forEach(key -> {
+            WindowValue windowValue = allValueList.get(key);
+            deleteMergeWindow(windowInstance.getWindowInstanceId(), windowValue.getPartition(),
+                    windowValue.getFireTime(), windowValue.getPartitionNum(), windowValue.getGroupBy());
+        });
         return resultList;
     }
 
-    private void deletePrefixValue(List<String> keyList) {
-        if (CollectionUtil.isEmpty(keyList)) {
-            return;
-        }
-        storage.getLocalStorage().removeKeys(keyList);
-    }
 
-    private static String createPrefixKey(WindowValue windowValue, WindowInstance windowInstance, String queueId) {
-        return MapKeyUtil.createKey(getOrderBypPrefix() + queueId, windowInstance.createWindowInstanceId(), windowValue.getFireTime(), String.valueOf(windowValue.getPartitionNum()), windowValue.getGroupBy());
+    private void deleteMergeWindow(String windowInstanceId, String queueId, String fireTime, long partitionNum, String groupBy) {
+        RocksdbIterator<WindowValue> windowBaseValueWrap = storage.getWindowBaseValue(queueId, windowInstanceId, WindowType.SESSION_WINDOW, null);
+
+
+        while (windowBaseValueWrap.hasNext()) {
+            IteratorWrap<WindowValue> wrap = windowBaseValueWrap.next();
+            WindowValue windowValue = wrap.getData();
+            if (windowValue.getPartition().equals(queueId) && windowValue.getPartitionNum() == partitionNum
+                    && windowValue.getGroupBy().equals(groupBy) && windowValue.getFireTime().equals(fireTime)) {
+                windowBaseValueWrap.remove();
+            }
+        }
+
+        windowBaseValueWrap = storage.getWindowBaseValue(queueId, windowInstanceId, WindowType.SESSION_WINDOW, null);
+
+        if (windowBaseValueWrap.hasNext()) {
+            storage.putWindowBaseValueIterator(queueId, windowInstanceId, WindowType.SESSION_WINDOW, null, windowBaseValueWrap);
+        }
     }
 
     private Pair<Date, Date> getSessionTime(IMessage message) {
@@ -316,26 +337,19 @@
         return Pair.of(occurDate, endDate);
     }
 
-    protected void store(Map<String, List<WindowValue>> key2ValueMap, WindowInstance windowInstance,
-        String queueId) {
-        //
+    protected void store(Map<String, List<WindowValue>> key2ValueMap, WindowInstance windowInstance, String queueId) {
         if (CollectionUtil.isEmpty(key2ValueMap)) {
             return;
         }
-        //
-        storage.multiPutList(key2ValueMap, windowInstance.createWindowInstanceId(), queueId, sqlCache);
-        //
-        Map<String, WindowValue> allValueMap = new HashMap<>();
-        Iterator<Map.Entry<String, List<WindowValue>>> iterator = key2ValueMap.entrySet().iterator();
-        while (iterator.hasNext()) {
-            Map.Entry<String, List<WindowValue>> entry = iterator.next();
-            List<WindowValue> valueList = entry.getValue();
-            for (WindowValue value : valueList) {
-                String partitionNumKey = createPrefixKey(value, windowInstance, queueId);
-                allValueMap.put(partitionNumKey, value);
-            }
+
+        for (String storeKey : key2ValueMap.keySet()) {
+            List<WindowBaseValue> temp = key2ValueMap.get(storeKey).stream()
+                    .map(value -> (WindowBaseValue) value)
+                    .collect(Collectors.toList());
+
+            storage.putWindowBaseValue(queueId, windowInstance.getWindowInstanceId(), WindowType.SESSION_WINDOW, null, temp);
         }
-        storage.getLocalStorage().multiPut(allValueMap);
+
     }
 
     /**
@@ -347,7 +361,7 @@
      * @return
      */
     protected WindowValue createWindowValue(String queueId, String groupBy, WindowInstance instance, IMessage message,
-        String storeKey) {
+                                            String storeKey) {
         WindowValue value = new WindowValue();
         Pair<Date, Date> startEndPair = getSessionTime(message);
         String startTime = DateUtil.format(startEndPair.getLeft(), SESSION_DATETIME_PATTERN);
@@ -357,43 +371,46 @@
         value.setEndTime(endTime);
         value.setFireTime(fireTime);
         value.setGroupBy(groupBy);
-        value.setMsgKey(StringUtil.createMD5Str(storeKey));
+        value.setMsgKey(storeKey);
         //FIXME shuffleId vs queueId TODO delete assert
         String shuffleId = shuffleChannel.getChannelQueue(groupBy).getQueueId();
         assert shuffleId.equalsIgnoreCase(queueId);
-        value.setPartitionNum(createPartitionNum(value, queueId, instance));
+        value.setPartitionNum(createPartitionNum(queueId, instance));
         value.setPartition(shuffleId);
-        value.setWindowInstancePartitionId(instance.getWindowInstanceKey());
-        value.setWindowInstanceId(instance.getWindowInstanceKey());
+        value.setWindowInstanceId(instance.getWindowInstanceId());
         return value;
     }
 
-    protected static String getOrderBypPrefix() {
-        return ORDER_BY_FIRE_TIME_PREFIX;
-    }
 
-    /**
-     * update session's next fire time
-     *
-     * @param windowInstance
-     * @param queueId
-     * @param queueId2Offset
-     * @return
-     */
     @Override
-    public int fireWindowInstance(WindowInstance windowInstance, String queueId, Map<String, String> queueId2Offset) {
+    public int doFireWindowInstance(WindowInstance windowInstance) {
         synchronized (lock) {
-            //get iterator sorted by fire time
-            WindowBaseValueIterator<WindowValue> it = storage.loadWindowInstanceSplitData(getOrderBypPrefix(), queueId, windowInstance.createWindowInstanceId(), null, getWindowBaseValueClass());
-            //
+            String queueId = windowInstance.getSplitId();
+
+            RocksdbIterator<WindowBaseValue> windowBaseValue = storage.getWindowBaseValue(queueId,
+                    windowInstance.getWindowInstanceId(), WindowType.SESSION_WINDOW, null);
+
+
+            ArrayList<WindowBaseValue> baseValues = new ArrayList<>();
+
+            while (windowBaseValue.hasNext()) {
+                IteratorWrap<WindowBaseValue> next = windowBaseValue.next();
+                baseValues.add(next.getData());
+            }
+
+            baseValues.sort(Comparator.comparingLong(WindowBaseValue::getPartitionNum));
+
             Long currentFireTime = DateUtil.parse(windowInstance.getFireTime(), SESSION_DATETIME_PATTERN).getTime();
             Long nextFireTime = currentFireTime + 1000 * 60 * 1;
             List<WindowValue> toFireValueList = new ArrayList<>();
-            while (it.hasNext()) {
-                WindowValue windowValue = it.next();
+
+
+            for (WindowBaseValue baseValue : baseValues) {
+                WindowValue windowValue = (WindowValue) baseValue;
                 if (windowValue == null) {
                     continue;
                 }
+
                 if (checkFire(queueId, windowValue)) {
                     TraceUtil.debug(String.valueOf(windowValue.getPartitionNum()), "shuffle message fire", windowValue.getStartTime(), windowValue.getEndTime(), windowValue.getComputedColumnResult());
                     toFireValueList.add(windowValue);
@@ -404,19 +421,12 @@
                         break;
                     }
                 }
+
             }
-            doFire(queueId, windowInstance, toFireValueList);
-            //
-            if (!nextFireTime.equals(currentFireTime)) {
-                String instanceId = windowInstance.createWindowInstanceId();
-                WindowInstance existedWindowInstance = searchWindowInstance(instanceId);
-                if (existedWindowInstance != null) {
-                    existedWindowInstance.setFireTime(DateUtil.format(new Date(nextFireTime)));
-                    windowFireSource.registFireWindowInstanceIfNotExist(windowInstance, this);
-                } else {
-                    LOG.error("window instance lost, queueId: " + queueId + " ,fire time" + windowInstance.getFireTime());
-                }
-            }
+
+
+            doFire(queueId, windowInstance, toFireValueList, currentFireTime, nextFireTime);
+
             return toFireValueList.size();
         }
 
@@ -435,57 +445,84 @@
         return false;
     }
 
-    private void doFire(String queueId, WindowInstance instance, List<WindowValue> valueList) {
+    private void doFire(String queueId, WindowInstance instance, List<WindowValue> valueList, Long currentFireTime,
+                        Long nextFireTime) {
         if (CollectionUtil.isEmpty(valueList)) {
             return;
         }
         valueList.sort(Comparator.comparingLong(WindowBaseValue::getPartitionNum));
         sendFireMessage(valueList, queueId);
         clearWindowValues(valueList, queueId, instance);
+
+        if (!nextFireTime.equals(currentFireTime)) {
+            String instanceId = instance.getWindowInstanceId();
+            WindowInstance existedWindowInstance = searchWindowInstance(instanceId);
+            if (existedWindowInstance != null) {
+                existedWindowInstance.setFireTime(DateUtil.format(new Date(nextFireTime)));
+                windowFireSource.registFireWindowInstanceIfNotExist(instance, this);
+            } else {
+                LOG.error("window instance lost, queueId: " + queueId + " ,fire time" + instance.getFireTime());
+            }
+        }
     }
 
-    /**
-     * clear fired window value
-     *
-     * @param deleteValueList
-     * @param queueId
-     * @param instance
-     */
+
     protected void clearWindowValues(List<WindowValue> deleteValueList, String queueId, WindowInstance instance) {
         if (CollectionUtil.isEmpty(deleteValueList)) {
             return;
         }
+
         Set<String> storeKeySet = new HashSet<>(deleteValueList.size());
         Set<Long> valueIdSet = new HashSet<>(deleteValueList.size());
-        Set<String> prefixKeySet = new HashSet<>(deleteValueList.size());
+
         for (WindowValue windowValue : deleteValueList) {
             String storeKey = createStoreKey(queueId, windowValue.getGroupBy(), instance);
-            String prefixKey = createPrefixKey(windowValue, instance, queueId);
             Long valueId = windowValue.getPartitionNum();
             storeKeySet.add(storeKey);
             valueIdSet.add(valueId);
-            prefixKeySet.add(prefixKey);
+
         }
-        Map<String, List<WindowValue>> storeValueMap = storage.multiGetList(WindowValue.class, new ArrayList<>(storeKeySet));
+
+        RocksdbIterator<WindowBaseValue> rocksdbIterator = storage.getWindowBaseValue(queueId, instance.getWindowInstanceId(), WindowType.SESSION_WINDOW, null);
+        List<WindowBaseValue> temp = new ArrayList<>();
+        while (rocksdbIterator.hasNext()) {
+            IteratorWrap<WindowBaseValue> next = rocksdbIterator.next();
+            temp.add(next.getData());
+        }
+
+        Map<String, List<WindowValue>> storeValueMap = temp.stream()
+                .map((value) -> (WindowValue) value)
+                .filter((value) -> {
+                    for (String storeKey : storeKeySet) {
+                        if (storeKey.equalsIgnoreCase(value.getMsgKey())) {
+                            return true;
+                        }
+                    }
+                    return false;
+                }).collect(Collectors.groupingBy(WindowValue::getMsgKey));
+
         Map<String, List<WindowValue>> lastValueMap = new HashMap<>(storeValueMap.size());
-        Iterator<Map.Entry<String, List<WindowValue>>> iterator = storeValueMap.entrySet().iterator();
-        while (iterator.hasNext()) {
-            Map.Entry<String, List<WindowValue>> entry = iterator.next();
+        for (Map.Entry<String, List<WindowValue>> entry : storeValueMap.entrySet()) {
             String storeKey = entry.getKey();
             List<WindowValue> valueList = entry.getValue();
             valueList = valueList.stream().filter(value -> !valueIdSet.contains(value.getPartitionNum())).collect(Collectors.toList());
             lastValueMap.put(storeKey, valueList);
         }
-        storage.getLocalStorage().removeKeys(prefixKeySet);
+
+        for (WindowValue windowValue : deleteValueList) {
+            storage.deleteWindowBaseValue(queueId, instance.getWindowInstanceId(), WindowType.SESSION_WINDOW, null, windowValue.getMsgKey());
+        }
+
         store(lastValueMap, instance, queueId);
     }
 
+
     @Override
     public long incrementAndGetSplitNumber(WindowInstance instance, String shuffleId) {
-        long number = super.incrementAndGetSplitNumber(instance, shuffleId);
-        if (number > 900000000) {
-            this.getWindowMaxValueManager().resetSplitNum(instance, shuffleId);
+        long numer = super.incrementAndGetSplitNumber(instance, shuffleId);
+        if (numer > 900000000) {
+            this.storage.putMaxPartitionNum(shuffleId, instance.getWindowInstanceId(), numer);
         }
-        return number;
+        return numer;
     }
 }
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java
index 5c48044..65f326f 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/impl/WindowOperator.java
@@ -16,203 +16,137 @@
  */
 package org.apache.rocketmq.streams.window.operator.impl;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
 import org.apache.rocketmq.streams.common.context.IMessage;
 import org.apache.rocketmq.streams.common.context.MessageOffset;
-import org.apache.rocketmq.streams.common.utils.CollectionUtil;
 import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
 import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.db.driver.batchloader.IRowOperator;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
 import org.apache.rocketmq.streams.window.debug.DebugWriter;
 import org.apache.rocketmq.streams.window.model.WindowInstance;
 import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow;
-import org.apache.rocketmq.streams.window.operator.AbstractWindow;
-import org.apache.rocketmq.streams.window.sqlcache.impl.FiredNotifySQLElement;
 import org.apache.rocketmq.streams.window.state.WindowBaseValue;
 import org.apache.rocketmq.streams.window.state.impl.WindowValue;
-import org.apache.rocketmq.streams.window.storage.IWindowStorage;
-import org.apache.rocketmq.streams.window.storage.ShufflePartitionManager;
-import org.apache.rocketmq.streams.window.storage.WindowStorage.WindowBaseValueIterator;
+import org.apache.rocketmq.streams.window.storage.IteratorWrap;
+import org.apache.rocketmq.streams.window.storage.RocksdbIterator;
+import org.apache.rocketmq.streams.window.storage.WindowType;
+
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
 
 public class WindowOperator extends AbstractShuffleWindow {
-
-    private static final String ORDER_BY_SPLIT_NUM = "_order_by_split_num_";//key=_order;queueid,windowinstanceid,partitionNum
-
     public WindowOperator() {
         super();
     }
 
-    @Deprecated
-    public WindowOperator(String timeFieldName, int windowPeriodMinute) {
-        super();
-        super.timeFieldName = timeFieldName;
-        super.sizeInterval = windowPeriodMinute;
-    }
-
-    @Deprecated
-    public WindowOperator(String timeFieldName, int windowPeriodMinute, String calFieldName) {
-        super();
-        super.timeFieldName = timeFieldName;
-        super.sizeInterval = windowPeriodMinute;
-    }
-
-    public WindowOperator(int sizeInterval, String groupByFieldName, Map<String, String> select) {
-        this.sizeInterval = sizeInterval;
-        this.slideInterval = sizeInterval;
-        this.groupByFieldName = groupByFieldName;
-        this.setSelectMap(select);
-    }
-
-    protected transient AtomicInteger shuffleCount = new AtomicInteger(0);
-    protected transient AtomicInteger fireCountAccumulator = new AtomicInteger(0);
-
     @Override
-    public int fireWindowInstance(WindowInstance instance, String queueId, Map<String, String> queueId2Offset) {
-        List<WindowValue> windowValues = new ArrayList<>();
-        int fireCount = 0;
-        long startTime = System.currentTimeMillis();
-        int sendCost = 0;
-        int currentCount = 0;
-        //for(String queueId:currentQueueIds){
-        WindowBaseValueIterator<WindowBaseValue> it = storage.loadWindowInstanceSplitData(getOrderBypPrefix(), queueId, instance.createWindowInstanceId(), null, getWindowBaseValueClass());
-        if (queueId2Offset != null) {
-            String offset = queueId2Offset.get(queueId);
-            if (StringUtil.isNotEmpty(offset)) {
-                it.setPartitionNum(Long.valueOf(offset));
-            }
-        }
-        while (it.hasNext()) {
-            WindowBaseValue windowBaseValue = it.next();
-            if (windowBaseValue == null) {
-                continue;
-            }
-            WindowValue windowValue = (WindowValue) windowBaseValue;
+    public int doFireWindowInstance(WindowInstance instance) {
+        String windowInstanceId = instance.getWindowInstanceId();
+        String queueId = instance.getSplitId();
 
-            Integer currentValue = getValue(windowValue, "total");
+        RocksdbIterator<WindowBaseValue> rocksdbIterator = storage.getWindowBaseValue(queueId, windowInstanceId, WindowType.NORMAL_WINDOW, null);
 
-            fireCountAccumulator.addAndGet(currentValue);
-            windowValues.add((WindowValue) windowBaseValue);
-            if (windowValues.size() >= windowCache.getBatchSize()) {
-                long sendFireCost = System.currentTimeMillis();
-                sendFireMessage(windowValues, queueId);
-                sendCost += (System.currentTimeMillis() - sendFireCost);
-                fireCount += windowValues.size();
-                windowValues = new ArrayList<>();
-            }
+        ArrayList<WindowValue> windowValues = new ArrayList<>();
+        while (rocksdbIterator.hasNext()) {
+            IteratorWrap<WindowBaseValue> next = rocksdbIterator.next();
+            WindowValue data = (WindowValue)next.getData();
+            windowValues.add(data);
+        }
 
-        }
-        if (windowValues.size() > 0) {
-            long sendFireCost = System.currentTimeMillis();
-            sendFireMessage(windowValues, queueId);
-            sendCost += (System.currentTimeMillis() - sendFireCost);
-            fireCount += windowValues.size();
-        }
+        windowValues.sort(Comparator.comparingLong(WindowBaseValue::getPartitionNum));
+
+        int fireCount = sendBatch(windowValues, queueId, 0);
+
         clearFire(instance);
-        this.sqlCache.addCache(new FiredNotifySQLElement(queueId, instance.createWindowInstanceId()));
+
         return fireCount;
     }
 
-    protected transient Map<String, Integer> shuffleWindowInstanceId2MsgCount = new HashMap<>();
-    protected transient int windowvaluecount = 0;
+
+    private int sendBatch(List<WindowValue> windowValues, String queueId, int fireCount) {
+        if (windowValues == null || windowValues.size() == 0) {
+            return fireCount;
+        }
+
+        if (windowValues.size() <= windowCache.getBatchSize()) {
+            sendFireMessage(windowValues, queueId);
+
+            fireCount += windowValues.size();
+
+            return fireCount;
+        } else {
+            ArrayList<WindowValue> temp = new ArrayList<>();
+            for (int i = 0; i < windowCache.getBatchSize(); i++) {
+                temp.add(windowValues.remove(i));
+            }
+
+            sendFireMessage(temp, queueId);
+
+            return sendBatch(windowValues, queueId, fireCount + windowCache.getBatchSize());
+        }
+    }
+
 
     @Override
     public void shuffleCalculate(List<IMessage> messages, WindowInstance instance, String queueId) {
         DebugWriter.getDebugWriter(getConfigureName()).writeShuffleCalcultateReceveMessage(instance, messages, queueId);
+
         List<String> sortKeys = new ArrayList<>();
         Map<String, List<IMessage>> groupBy = groupByGroupName(messages, sortKeys);
-        Set<String> groupByKeys = groupBy.keySet();
-        List<String> storeKeys = new ArrayList<>();
-        for (String groupByKey : groupByKeys) {
-            String storeKey = createStoreKey(queueId, groupByKey, instance);
-            storeKeys.add(storeKey);
-        }
-        Map<String, WindowBaseValue> allWindowValues = new HashMap<>();
-        //从存储中,查找window value对象,value是对象的json格式
-        Map<String, WindowBaseValue> existWindowValues = storage.multiGet(getWindowBaseValueClass(), storeKeys, instance.createWindowInstanceId(), queueId);
-        //  Iterator<Entry<String, List<IMessage>>> it = groupBy.entrySet().iterator();
-        for (String groupByKey : sortKeys) {
 
+        RocksdbIterator<WindowBaseValue> windowBaseValue = storage.getWindowBaseValue(queueId, instance.getWindowInstanceId(), WindowType.NORMAL_WINDOW, null);
+
+        ArrayList<WindowBaseValue> windowValues = new ArrayList<>();
+        while (windowBaseValue.hasNext()) {
+            IteratorWrap<WindowBaseValue> next = windowBaseValue.next();
+            windowValues.add(next.getData());
+        }
+
+        Map<String, List<WindowValue>> temp = windowValues.stream().map((value) -> (WindowValue) value).collect(Collectors.groupingBy(WindowValue::getMsgKey));
+
+        Map<String, List<WindowValue>> groupByMsgKey = new HashMap<>(temp);
+
+        List<WindowValue> allWindowValues = new ArrayList<>();
+
+        //处理不同groupBy的message
+        for (String groupByKey : sortKeys) {
             List<IMessage> msgs = groupBy.get(groupByKey);
             String storeKey = createStoreKey(queueId, groupByKey, instance);
-            WindowValue windowValue = (WindowValue) existWindowValues.get(storeKey);
-            ;
-            if (windowValue == null) {
-                windowvaluecount++;
-                windowValue = createWindowValue(queueId, groupByKey, instance);
-                // windowValue.setOrigOffset(msgs.get(0).getHeader().getOffset());
-            }
-            allWindowValues.put(storeKey, windowValue);
-            windowValue.incrementUpdateVersion();
 
-            Integer origValue = getValue(windowValue, "total");
+            //msgKey 为唯一键
+            List<WindowValue> windowValueList = groupByMsgKey.get(storeKey);
+            WindowValue windowValue;
+            if (windowValueList == null || windowValueList.size() == 0) {
+                windowValue = createWindowValue(queueId, groupByKey, instance);
+            } else {
+                windowValue = windowValueList.get(0);
+            }
+
+            allWindowValues.add(windowValue);
+            windowValue.incrementUpdateVersion();
 
             if (msgs != null) {
                 for (IMessage message : msgs) {
                     calculateWindowValue(windowValue, message);
-
                 }
             }
-
-            Integer currentValue = getValue(windowValue, "total");
-
-            shuffleCount.addAndGet(-origValue);
-            shuffleCount.addAndGet(currentValue);
         }
+
         if (DebugWriter.getDebugWriter(this.getConfigureName()).isOpenDebug()) {
-            DebugWriter.getDebugWriter(this.getConfigureName()).writeWindowCalculate(this, new ArrayList(allWindowValues.values()), queueId);
+            DebugWriter.getDebugWriter(this.getConfigureName()).writeWindowCalculate(this, allWindowValues, queueId);
         }
 
-        saveStorage(allWindowValues, instance, queueId);
+        saveStorage(instance.getWindowInstanceId(), queueId, allWindowValues);
     }
 
-    private Integer getValue(WindowValue windowValue, String fieldName) {
-        Object value = windowValue.getComputedColumnResultByKey(fieldName);
-        if (value == null) {
-            return 0;
-        }
-        if (value instanceof Integer) {
-            return (Integer) value;
-        } else if (value instanceof String) {
-            String strValue = (String) value;
-            return Integer.valueOf(strValue);
-        }
-        throw new ClassCastException("value:[" + value + "] of fieldName:[" + fieldName + "] can not change to number.");
+
+    protected void saveStorage(String windowInstanceId, String queueId, List<WindowValue> allWindowValues) {
+        List<WindowBaseValue> temp = new ArrayList<>(allWindowValues);
+
+        storage.putWindowBaseValue(queueId, windowInstanceId, WindowType.NORMAL_WINDOW, null, temp);
     }
 
-    protected void saveStorage(Map<String, WindowBaseValue> allWindowValues, WindowInstance windowInstance,
-        String queueId) {
-        String windowInstanceId = windowInstance.createWindowInstanceId();
 
-        storage.multiPut(allWindowValues, windowInstanceId, queueId, sqlCache);
-        Map<String, WindowBaseValue> partionNumOrders = new HashMap<>();//需要基于key前缀排序partitionnum
-        for (WindowBaseValue windowBaseValue : allWindowValues.values()) {
-            WindowValue windowValue = (WindowValue) windowBaseValue;
-            String partitionNumKey = createStoreKey(getOrderBypPrefix() + queueId, MapKeyUtil.createKey(getOrderBypFieldName(windowValue), windowValue.getGroupBy()), windowInstance);
-            partionNumOrders.put(partitionNumKey, windowValue);
-        }
-        storage.getLocalStorage().multiPut(partionNumOrders);
-    }
 
-    @Override
-    public Class getWindowBaseValueClass() {
-        return WindowValue.class;
-    }
-
-    /**
-     * 按group name 进行分组
-     *
-     * @param messages
-     * @return
-     */
     protected Map<String, List<IMessage>> groupByGroupName(List<IMessage> messages, List<String> sortKeys) {
         if (messages == null || messages.size() == 0) {
             return new HashMap<>();
@@ -231,12 +165,13 @@
             } else {
                 if (minOffset.greateThan(message.getHeader().getOffset())) {
                     minOffset = message.getHeader().getMessageOffset();
-
                 }
             }
             minOffsets.put(groupByValue, minOffset);
             messageList.add(message);
         }
+
+
         List<Entry<String, MessageOffset>> sortByMinOffset = new ArrayList<>(minOffsets.entrySet());
         sortByMinOffset.sort((o1, o2) -> {
             if (o1.getValue().equals(o2.getValue())) {
@@ -255,10 +190,6 @@
         return groupBy;
     }
 
-    @Override
-    protected Long queryWindowInstanceMaxSplitNum(WindowInstance instance) {
-        return storage.getMaxSplitNum(instance, getWindowBaseValueClass());
-    }
 
     @Override
     public boolean supportBatchMsgFinish() {
@@ -270,201 +201,55 @@
 
     }
 
-    /**
-     * 创建新的window value对象
-     *
-     * @param groupBy
-     * @param instance
-     * @return
-     */
+
     protected WindowValue createWindowValue(String queueId, String groupBy, WindowInstance instance) {
         WindowValue windowValue = new WindowValue();
         windowValue.setStartTime(instance.getStartTime());
         windowValue.setEndTime(instance.getEndTime());
         windowValue.setFireTime(instance.getFireTime());
         windowValue.setGroupBy(groupBy == null ? "" : groupBy);
-        windowValue.setMsgKey(StringUtil.createMD5Str(MapKeyUtil.createKey(queueId, instance.createWindowInstanceId(), groupBy)));
+        windowValue.setMsgKey(MapKeyUtil.createKey(queueId, instance.getWindowInstanceId(), groupBy));
         String shuffleId = shuffleChannel.getChannelQueue(groupBy).getQueueId();
-        windowValue.setPartitionNum(createPartitionNum(windowValue, queueId, instance));
+        windowValue.setPartitionNum(createPartitionNum(queueId, instance));
         windowValue.setPartition(shuffleId);
-        windowValue.setWindowInstancePartitionId(instance.getWindowInstanceKey());
-        windowValue.setWindowInstanceId(instance.getWindowInstanceKey());
+        windowValue.setWindowInstanceId(instance.getWindowInstanceId());
 
         return windowValue;
-
     }
 
-    protected long createPartitionNum(WindowValue windowValue, String shuffleId, WindowInstance instance) {
+    protected long createPartitionNum(String shuffleId, WindowInstance instance) {
         return incrementAndGetSplitNumber(instance, shuffleId);
     }
 
-    /**
-     * 创建存储key
-     *
-     * @param groupByKey
-     * @param windowInstance
-     * @return
-     */
+
     protected static String createStoreKey(String shuffleId, String groupByKey, WindowInstance windowInstance) {
-        return MapKeyUtil.createKey(shuffleId, windowInstance.createWindowInstanceId(), groupByKey);
+        return MapKeyUtil.createKey(shuffleId, windowInstance.getWindowInstanceId(), groupByKey);
     }
 
-    /**
-     * 需要排序的前缀
-     *
-     * @return
-     */
-    protected static String getOrderBypPrefix() {
-        return ORDER_BY_SPLIT_NUM;
-    }
 
-    /**
-     * 需要排序的字段值
-     *
-     * @return
-     */
-    protected static String getOrderBypFieldName(WindowValue windowValue) {
-        return windowValue.getPartitionNum() + "";
-    }
-
-    /**
-     * 删除掉触发过的数据
-     *
-     * @param windowInstance
-     */
     @Override
     public void clearFireWindowInstance(WindowInstance windowInstance) {
-        String partitionNum = (getOrderBypPrefix() + windowInstance.getSplitId());
-
         boolean canClear = windowInstance.isCanClearResource();
-        //        if(fireMode!=2){
-        //            canClear=true;
-        //        }
 
         if (canClear) {
             logoutWindowInstance(windowInstance.createWindowInstanceTriggerId());
-            windowMaxValueManager.deleteSplitNum(windowInstance, windowInstance.getSplitId());
-            ShufflePartitionManager.getInstance().clearWindowInstance(windowInstance.createWindowInstanceId());
-            storage.delete(windowInstance.createWindowInstanceId(), windowInstance.getSplitId(), getWindowBaseValueClass(), sqlCache);
-            storage.getLocalStorage().delete(windowInstance.createWindowInstanceId(), partitionNum, getWindowBaseValueClass());
-            if (!isLocalStorageOnly) {
-                WindowInstance.clearInstance(windowInstance, sqlCache);
-            }
+
+            //清理MaxPartitionNum
+            storage.deleteMaxPartitionNum(windowInstance.getSplitId(), windowInstance.getWindowInstanceId());
+
+            //清理WindowInstance
+            storage.deleteWindowInstance(windowInstance.getSplitId(), this.getNameSpace(), this.getConfigureName(), windowInstance.getWindowInstanceId());
+
+            //清理WindowValue
+            storage.deleteWindowBaseValue(windowInstance.getSplitId(), windowInstance.getWindowInstanceId(), WindowType.NORMAL_WINDOW, null);
         }
 
     }
 
     @Override
     public void clearCache(String queueId) {
-        getStorage().clearCache(shuffleChannel.getChannelQueue(queueId), getWindowBaseValueClass());
-        getStorage().clearCache(getOrderByQueue(queueId, getOrderBypPrefix()), getWindowBaseValueClass());
-        ShufflePartitionManager.getInstance().clearSplit(queueId);
+        storage.clearCache(queueId);
     }
 
-    public ISplit getOrderByQueue(String key, String prefix) {
-        int index = shuffleChannel.hash(key);
-        ISplit targetQueue = shuffleChannel.getQueueList().get(index);
-        return new ISplit() {
-            @Override
-            public String getQueueId() {
-                return prefix + targetQueue.getQueueId();
-            }
-
-            @Override
-            public Object getQueue() {
-                return targetQueue.getQueue();
-            }
-
-            @Override
-            public int compareTo(Object o) {
-                return targetQueue.compareTo(o);
-            }
-
-            @Override
-            public String toJson() {
-                return targetQueue.toJson();
-            }
-
-            @Override
-            public void toObject(String jsonString) {
-                targetQueue.toObject(jsonString);
-            }
-        };
-    }
-
-    public static void compareAndSet(WindowInstance windowInstance, IWindowStorage storage,
-        List<WindowValue> windowValues) {
-        if (windowValues == null || storage == null) {
-            return;
-        }
-        synchronized (storage) {
-            List<String> storeKeys = new ArrayList<>();
-            Map<String, WindowValue> windowValueMap = new HashMap<>();
-            for (WindowValue windowValue : windowValues) {
-                String storeKey = createStoreKey(windowValue.getPartition(), windowValue.getGroupBy(), windowInstance);
-                storeKeys.add(storeKey);
-                windowValueMap.put(storeKey, windowValue);
-                String storeOrderKey = createStoreKey(windowValue.getPartition(), windowValue.getPartitionNum() + "", windowInstance);
-                windowValueMap.put(storeOrderKey, windowValue);
-            }
-            Map<String, WindowBaseValue> valueMap = storage.multiGet(WindowValue.class, storeKeys);
-            if (valueMap == null || valueMap.size() == 0) {
-                storage.multiPut(windowValueMap);
-                return;
-            }
-            Iterator<Entry<String, WindowBaseValue>> it = valueMap.entrySet().iterator();
-
-            while (it.hasNext()) {
-                Entry<String, WindowBaseValue> entry = it.next();
-                String storeKey = entry.getKey();
-                WindowBaseValue localValue = entry.getValue();
-                WindowValue windowValue = windowValueMap.get(storeKey);
-                if (windowValue.getUpdateVersion() <= localValue.getUpdateVersion()) {
-                    windowValueMap.remove(storeKey);
-                }
-            }
-            if (CollectionUtil.isNotEmpty(windowValueMap)) {
-                storage.multiPut(windowValueMap);
-            }
-        }
-    }
-
-    public static class WindowRowOperator implements IRowOperator {
-
-        protected WindowInstance windowInstance;
-        protected String spiltId;
-        protected AbstractWindow window;
-
-        public WindowRowOperator(WindowInstance windowInstance, String spiltId, AbstractWindow window) {
-            this.windowInstance = windowInstance;
-            this.spiltId = spiltId;
-            this.window = window;
-        }
-
-        @Override
-        public synchronized void doProcess(Map<String, Object> row) {
-            WindowValue windowValue = ORMUtil.convert(row, WindowValue.class);
-            List<String> keys = new ArrayList<>();
-            String storeKey = createStoreKey(spiltId, windowValue.getGroupBy(), windowInstance);
-            keys.add(storeKey);
-            String storeOrderKey = createStoreKey(getOrderBypPrefix() + windowValue.getPartition(), MapKeyUtil.createKey(getOrderBypFieldName(windowValue), windowValue.getGroupBy()), windowInstance);
-            Map<String, WindowBaseValue> valueMap = window.getStorage().getLocalStorage().multiGet(WindowValue.class, keys);
-            if (CollectionUtil.isEmpty(valueMap)) {
-                Map<String, WindowBaseValue> map = new HashMap<>(4);
-
-                map.put(storeKey, windowValue);
-                map.put(storeOrderKey, windowValue);
-                window.getStorage().getLocalStorage().multiPut(map);
-                return;
-            }
-            WindowValue localValue = (WindowValue) valueMap.values().iterator().next();
-            if (windowValue.getUpdateVersion() > localValue.getUpdateVersion()) {
-                Map<String, WindowBaseValue> map = new HashMap<>();
-                map.put(storeKey, windowValue);
-                map.put(storeOrderKey, windowValue);
-                window.getStorage().getLocalStorage().multiPut(map);
-            }
-        }
-    }
 
 }
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/DBOperator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/DBOperator.java
deleted file mode 100644
index e6923ee..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/DBOperator.java
+++ /dev/null
@@ -1,450 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.operator.join;
-
-import com.alibaba.fastjson.JSONArray;
-import com.alibaba.fastjson.JSONObject;
-import java.security.MessageDigest;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.context.IMessage;
-import org.apache.rocketmq.streams.common.context.Message;
-import org.apache.rocketmq.streams.common.context.MessageHeader;
-import org.apache.rocketmq.streams.common.utils.DateUtil;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
-import org.apache.rocketmq.streams.window.state.impl.JoinLeftState;
-import org.apache.rocketmq.streams.window.state.impl.JoinRightState;
-import org.apache.rocketmq.streams.window.state.impl.JoinState;
-
-public class DBOperator implements Operator {
-
-    private static final Log LOG = LogFactory.getLog(DBOperator.class);
-
-    /**
-     * 根据join流对原始数据进行处理并入库
-     *
-     * @param messageList
-     * @param joinType
-     */
-    public void addMessage(List<IMessage> messageList, String joinType) {
-        List<JoinLeftState> joinLeftStates = new ArrayList<JoinLeftState>();
-        List<JoinRightState> joinRightStates = new ArrayList<JoinRightState>();
-        for (IMessage message : messageList) {
-            String routeLabel = message.getHeader().getMsgRouteFromLable();
-            JoinState state = dealMessge(message, routeLabel);
-            if ("left".equalsIgnoreCase(routeLabel)) {
-                joinLeftStates.add((JoinLeftState) state);
-            } else if ("right".equalsIgnoreCase(routeLabel)) {
-                joinRightStates.add((JoinRightState) state);
-            }
-            //            joinStates.add(state);
-        }
-
-        if (joinLeftStates.size() > 0) {
-            ORMUtil.batchReplaceInto(joinLeftStates);
-        }
-        if (joinRightStates.size() > 0) {
-            ORMUtil.batchReplaceInto(joinRightStates);
-        }
-
-    }
-
-    /**
-     * 生成joinstate对象
-     *
-     * @param message
-     * @param routeLabel
-     * @return
-     */
-    public JoinState dealMessge(IMessage message, String routeLabel) {
-
-        JSONObject window = message.getMessageBody().getJSONObject("Window");
-        String windowName = window.getString("configureName");
-        String windowNameSpace = window.getString("nameSpace");
-        MessageHeader header = message.getHeader();
-        String queueId = "_Dipper";
-        String offset = System.nanoTime() + "";
-        if (header.getQueueId() != null) {
-            queueId = header.getQueueId();
-        }
-        if (header.isEmptyOffset()) {
-            header.setOffset(offset);
-            offset = header.getOffset();
-        }
-
-        String messageId = windowNameSpace + "_" + windowName + "_" + queueId + "_" + offset;
-
-        List<String> leftJoinFieldNames = window.getJSONArray("leftJoinFieldNames") != null ?
-            toJavaList(window.getJSONArray("leftJoinFieldNames")) :
-            new ArrayList<>();
-        List<String> rightJoinFieldNames = window.getJSONArray("rightJoinFieldNames") != null ?
-            toJavaList(window.getJSONArray("rightJoinFieldNames")) :
-            new ArrayList<>();
-
-        String messageKey = generateKey(message.getMessageBody(), routeLabel, leftJoinFieldNames, rightJoinFieldNames);
-        JSONObject messageBody = (JSONObject) message.getMessageBody().clone();
-        messageBody.remove("WindowInstance");
-        messageBody.remove("Window");
-
-        JoinState state = null;
-        if ("left".equalsIgnoreCase(routeLabel)) {
-            state = new JoinLeftState();
-        } else if ("right".equalsIgnoreCase(routeLabel)) {
-            state = new JoinRightState();
-        }
-
-        state.setGmtCreate(new Date());
-        state.setGmtModified(new Date());
-        state.setWindowName(windowName);
-        state.setWindowNameSpace(windowNameSpace);
-        state.setMessageId(messageId);
-        state.setMessageKey(messageKey);
-        state.setMessageTime(new Date());
-        state.setMessageBody(messageBody.toJSONString());
-
-        return state;
-    }
-
-    private List<String> toJavaList(JSONArray jsonArray) {
-        List<String> list = new ArrayList<String>(jsonArray.size());
-
-        for (Object item : jsonArray) {
-            if (item == null) {
-                list.add(null);
-            } else if (String.class.isInstance(item)) {
-                list.add((String) item);
-            } else {
-                list.add(item.toString());
-            }
-
-        }
-
-        return list;
-    }
-
-    /**
-     * 根据join条件生成消息比对key值
-     *
-     * @param messageBody
-     * @param joinLabel
-     * @param leftJoinFieldNames
-     * @param rightJoinFieldNames
-     * @return
-     */
-    public static String generateKey(JSONObject messageBody, String joinLabel, List<String> leftJoinFieldNames,
-        List<String> rightJoinFieldNames) {
-        StringBuffer buffer = new StringBuffer();
-        if ("left".equalsIgnoreCase(joinLabel)) {
-            for (String field : leftJoinFieldNames) {
-                String value = messageBody.getString(field);
-                buffer.append(value).append("_");
-            }
-        } else {
-            for (String field : rightJoinFieldNames) {
-                String[] rightFields = field.split("\\.");
-                if (rightFields.length > 1) {
-                    field = rightFields[1];
-                }
-                String value = messageBody.getString(field);
-                buffer.append(value).append("_");
-            }
-        }
-
-        return MD5(buffer.toString());
-    }
-
-    public static String MD5(String s) {
-        char hexDigits[] = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F'};
-
-        try {
-            byte[] btInput = s.getBytes();
-            // 获得MD5摘要算法的 MessageDigest 对象
-            MessageDigest mdInst = MessageDigest.getInstance("MD5");
-            // 使用指定的字节更新摘要
-            mdInst.update(btInput);
-            // 获得密文
-            byte[] md = mdInst.digest();
-            // 把密文转换成十六进制的字符串形式
-            int j = md.length;
-            char str[] = new char[j * 2];
-            int k = 0;
-            for (int i = 0; i < j; i++) {
-                byte byte0 = md[i];
-                str[k++] = hexDigits[byte0 >>> 4 & 0xf];
-                str[k++] = hexDigits[byte0 & 0xf];
-            }
-            return new String(str);
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new RuntimeException(e.getMessage(), e);
-        }
-    }
-
-    public List<Map<String, Object>> dealJoin(IMessage message) {
-        List<Map<String, Object>> result = new ArrayList<>();
-        JSONObject messageBody = message.getMessageBody();
-        JSONObject msg = messageBody.getJSONObject("msg");
-        String routeLabel = messageBody.getString("routeLabel");
-        JSONArray windowInstances = msg.getJSONArray("WindowInstance");
-        JSONObject windowInstance = null;
-        if (windowInstances != null && windowInstances.size() > 0) {
-            windowInstance = windowInstances.getJSONObject(0);
-        } else {
-            return result;
-        }
-
-        Integer retainWindowCount = messageBody.getInteger("retainWindowCount");
-        Integer sizeInterval = messageBody.getInteger("sizeInterval");
-
-        List<JSONObject> tmp = new ArrayList<>();
-        if ("left".equalsIgnoreCase(routeLabel)) {
-            String endTime = windowInstance.getString("endTime");
-            String startTime = addTime(windowInstance.getString("startTime"), TimeUnit.MINUTES, -retainWindowCount * sizeInterval);
-            String tableName = "join_right_state";
-            String messageKey = messageBody.getString("messageKey");
-            String windowName = windowInstance.getString("windowName");
-            String windowNameSpace = windowInstance.getString("windowNameSpace");
-            tmp = getJoinData(tableName, messageKey, windowName, windowNameSpace, startTime, endTime);
-
-        } else if ("right".equalsIgnoreCase(routeLabel)) {
-            //            String startTime = addTime(windowInstance.getString("startTime"), TimeUnit.MINUTES, -sizeInterval);
-            String startTime = addTime(windowInstance.getString("startTime"), TimeUnit.MINUTES, -retainWindowCount * sizeInterval);
-            String endTime = addTime(windowInstance.getString("endTime"), TimeUnit.MINUTES, -sizeInterval);
-            String tableName = "join_left_state";
-            String messageKey = messageBody.getString("messageKey");
-            String windowName = windowInstance.getString("windowName");
-            String windowNameSpace = windowInstance.getString("windowNameSpace");
-            tmp = getJoinData(tableName, messageKey, windowName, windowNameSpace, startTime, endTime);
-        }
-
-        result = converToMapFromJson(tmp);
-        return result;
-
-    }
-
-    public List<JSONObject> connectJoin(IMessage message, List<Map<String, Object>> rows, String joinType,
-        String rightAsName) {
-        List<JSONObject> result = new ArrayList<>();
-        if (rows.size() <= 0) {
-            return result;
-        }
-        if ("inner".equalsIgnoreCase(joinType)) {
-            result = connectInnerJoin(message, rows, rightAsName);
-        } else if ("left".equalsIgnoreCase(joinType)) {
-            result = connectLeftJoin(message, rows, rightAsName);
-        }
-        return result;
-    }
-
-    /**
-     * inner join 将匹配messageKey的各行与message进行连接
-     *
-     * @param message
-     * @param rows
-     * @return
-     */
-    public List<JSONObject> connectInnerJoin(IMessage message, List<Map<String, Object>> rows, String rightAsName) {
-        List<JSONObject> result = new ArrayList<>();
-        String routeLabel = message.getMessageBody().getString("routeLabel");
-        if ("left".equalsIgnoreCase(routeLabel)) {
-            JSONObject messageBody = message.getMessageBody().getJSONObject("msg");
-            for (Map<String, Object> raw : rows) {
-                //                addAsName(raw, rightAsName);
-                JSONObject object = (JSONObject) messageBody.clone();
-                object.fluentPutAll(addAsName(raw, rightAsName));
-                result.add(object);
-            }
-        } else {
-            JSONObject messageBody = message.getMessageBody().getJSONObject("msg");
-            messageBody = (JSONObject) addAsName(messageBody, rightAsName);
-            for (Map<String, Object> raw : rows) {
-                JSONObject object = (JSONObject) messageBody.clone();
-                object.fluentPutAll(raw);
-                result.add(object);
-            }
-        }
-
-        return result;
-    }
-
-    private Map<String, Object> addAsName(Map<String, Object> raw, String rightAsName) {
-        Map<String, Object> asName = new HashMap<>();
-        for (Map.Entry<String, Object> tmp : raw.entrySet()) {
-            asName.put(rightAsName + "." + tmp.getKey(), tmp.getValue());
-            //            raw.remove(tmp.getKey());
-        }
-        return asName;
-    }
-
-    public List<JSONObject> connectLeftJoin(IMessage message, List<Map<String, Object>> rows, String rightAsName) {
-        List<JSONObject> result = new ArrayList<>();
-        String routeLabel = message.getMessageBody().getString("routeLabel");
-        JSONObject messageBody = message.getMessageBody().getJSONObject("msg");
-        if ("left".equalsIgnoreCase(routeLabel)) {
-            if (rows != null && rows.size() > 0) {
-                for (Map<String, Object> raw : rows) {
-                    //                    raw = addAsName(raw, rightAsName);
-                    JSONObject object = (JSONObject) messageBody.clone();
-                    object.fluentPutAll(addAsName(raw, rightAsName));
-                    result.add(object);
-                }
-                return result;
-            } else {
-                result.add(messageBody);
-            }
-
-        } else {
-            if (rows != null && rows.size() > 0) {
-                messageBody = (JSONObject) addAsName(messageBody, rightAsName);
-                for (Map<String, Object> raw : rows) {
-                    JSONObject object = (JSONObject) messageBody.clone();
-                    object.fluentPutAll(raw);
-                    result.add(object);
-                }
-                return result;
-            }
-        }
-
-        return result;
-
-    }
-
-    private List<Map<String, Object>> converToMapFromJson(List<JSONObject> list) {
-        List<Map<String, Object>> mapList = new ArrayList<>();
-        if (list != null && list.size() > 0) {
-            for (JSONObject object : list) {
-                Map<String, Object> tmp = object;
-                mapList.add(tmp);
-            }
-        }
-        return mapList;
-    }
-
-    public List<JSONObject> getJoinData(String tableName, String messageKey, String windowName, String windowNameSpace,
-        String startTime, String endTime) {
-        Map<String, Object> paras = new HashMap<>();
-        paras.put("messageKey", messageKey);
-        paras.put("startTime", startTime);
-        paras.put("endTime", endTime);
-        paras.put("windowName", windowName);
-        paras.put("windowNameSpace", windowNameSpace);
-        //        paras.put("tableName", tableName);
-        List<JoinState> result = new ArrayList<>();
-        if ("join_right_state".equalsIgnoreCase(tableName)) {
-            result = ORMUtil.queryForList("select message_body from join_right_state where message_key = #{messageKey} and window_name = #{windowName}" +
-                "and window_name_space = #{windowNameSpace} and gmt_create >= #{startTime} and gmt_create < #{endTime}", paras, JoinState.class);
-        } else if ("join_left_state".equalsIgnoreCase(tableName)) {
-            result = ORMUtil.queryForList("select message_body from join_left_state where message_key = #{messageKey} and window_name = #{windowName} " +
-                "and window_name_space = #{windowNameSpace} and gmt_create >= #{startTime} and gmt_create < #{endTime}", paras, JoinState.class);
-        }
-
-        List<JSONObject> bodys = new ArrayList<>();
-        for (JoinState tmp : result) {
-            try {
-                bodys.add(Message.parseObject(tmp.getMessageBody()));
-            } catch (Exception e) {
-                LOG.error("json parase error:", e);
-            }
-
-        }
-        return bodys;
-    }
-
-    public String addTime(String time, TimeUnit unit, int value) {
-        Date date = DateUtil.parseTime(time);
-        date = DateUtil.addDate(unit, date, value);
-        return DateUtil.format(date);
-    }
-
-    /**
-     * 根据window去除过期消息数据,消息去除时间为starttime加上窗口
-     *
-     * @param windowNameSpace
-     * @param windowName
-     * @param retainWindowCount
-     * @param sizeInterval
-     * @param startTime
-     */
-    public void cleanMessage(String windowNameSpace, String windowName, int retainWindowCount, int sizeInterval,
-        String startTime) {
-        Map<String, Object> params = new HashMap<>();
-        String start = addTime(startTime, TimeUnit.MINUTES, -retainWindowCount * sizeInterval);
-        params.put("startTime", start);
-        params.put("windowNameSpace", windowNameSpace);
-        params.put("windowName", windowName);
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("dboperata delete param is " + JSONObject.toJSONString(params));
-        }
-
-        List<JoinLeftState> joinLeftStates = ORMUtil.queryForList("select id from join_left_state where window_name_space = #{windowNameSpace} and " +
-            "window_name = #{windowName} and gmt_create < #{startTime}", params, JoinLeftState.class);
-        if (joinLeftStates != null && joinLeftStates.size() > 0) {
-            List<String> deleteIds = this.getDeleteIds(joinLeftStates);
-            for (String ids : deleteIds) {
-                //                params.put("ids", ids);
-                ORMUtil.executeSQL("delete from join_left_state where id in (" + ids + ")", null);
-            }
-
-        }
-
-        List<JoinRightState> joinRightStates = ORMUtil.queryForList("select id from join_right_state where window_name_space = #{windowNameSpace} and " +
-            "window_name = #{windowName} and gmt_create < #{startTime}", params, JoinRightState.class);
-        if (joinRightStates != null && joinRightStates.size() > 0) {
-            List<String> deleteIds = this.getDeleteIds(joinRightStates);
-            for (String ids : deleteIds) {
-                //                params.put("ids", ids);
-                ORMUtil.executeSQL("delete from join_right_state where id in (" + ids + ")", null);
-            }
-
-        }
-
-    }
-
-    private List<String> getDeleteIds(List<? extends JoinState> instances) {
-        List<String> deteleIds = new ArrayList<>();
-        if (instances == null || instances.size() == 0) {
-            return deteleIds;
-        }
-        int count = 1;
-
-        StringBuilder builder = new StringBuilder();
-        for (; count <= instances.size(); count++) {
-            builder.append(instances.get(count - 1).getId());
-            if (count % 1000 == 0) {
-                deteleIds.add(builder.toString());
-                builder = new StringBuilder();
-            } else {
-                if (count == instances.size()) {
-                    deteleIds.add(builder.toString());
-                } else {
-                    builder.append(",");
-                }
-
-            }
-            //            count++;
-        }
-
-        return deteleIds;
-    }
-
-}
\ No newline at end of file
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java
index 97d1c14..9c1c0af 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/JoinWindow.java
@@ -17,13 +17,15 @@
 package org.apache.rocketmq.streams.window.operator.join;
 
 import com.alibaba.fastjson.JSONObject;
+
 import java.util.ArrayList;
 import java.util.Date;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+
 import org.apache.rocketmq.streams.common.component.ComponentCreator;
 import org.apache.rocketmq.streams.common.context.AbstractContext;
 import org.apache.rocketmq.streams.common.context.Context;
@@ -42,18 +44,14 @@
 import org.apache.rocketmq.streams.window.state.impl.JoinLeftState;
 import org.apache.rocketmq.streams.window.state.impl.JoinRightState;
 import org.apache.rocketmq.streams.window.state.impl.JoinState;
-import org.apache.rocketmq.streams.window.storage.ShufflePartitionManager;
+import org.apache.rocketmq.streams.window.storage.IteratorWrap;
+import org.apache.rocketmq.streams.window.storage.RocksdbIterator;
+import org.apache.rocketmq.streams.window.storage.WindowJoinType;
+import org.apache.rocketmq.streams.window.storage.WindowType;
 
 import static org.apache.rocketmq.streams.window.shuffle.ShuffleChannel.SHUFFLE_OFFSET;
 
 public class JoinWindow extends AbstractShuffleWindow {
-
-    public static final String JOIN_KEY = "JOIN_KEY";
-
-    public static final String LABEL_LEFT = "left";
-
-    public static final String LABEL_RIGHT = "right";
-
     //保存多少个周期的数据。比如window的滚动周期是5分钟,join需要1个小时数据,则retainWindowCount=12
     protected int retainWindowCount = 4;
     protected List<String> leftJoinFieldNames;//join等值条件中,左流的字段列表
@@ -62,58 +60,49 @@
 
     protected String joinType;//join类型,值为INNER,LEFT
     protected String expression;//条件表达式。在存在非等值比较时使用
-    protected transient DBOperator joinOperator = new DBOperator();
 
-    //    @Override
-    //    protected void addPropertyToMessage(IMessage oriMessage, JSONObject oriJson){
-    //        oriJson.put("AbstractWindow", this);
-    //
-    //    }
 
     @Override
-    protected int fireWindowInstance(WindowInstance instance, String shuffleId, Map<String, String> queueId2Offsets) {
+    protected int doFireWindowInstance(WindowInstance instance) {
+        //todo 只是清理吗?
         clearFire(instance);
         return 0;
     }
 
     @Override
     public void clearCache(String queueId) {
-        getStorage().clearCache(shuffleChannel.getChannelQueue(queueId), getWindowBaseValueClass());
-        ShufflePartitionManager.getInstance().clearSplit(queueId);
+        storage.clearCache(queueId);
     }
 
     @Override
     public void shuffleCalculate(List<IMessage> messages, WindowInstance instance, String queueId) {
+        String windowInstanceId = instance.getWindowInstanceId();
 
         for (IMessage msg : messages) {
-            Map<String, WindowBaseValue> joinLeftStates = new HashMap<>();
-            Map<String, WindowBaseValue> joinRightStates = new HashMap<>();
             MessageHeader header = JSONObject.parseObject(msg.getMessageBody().getString(WindowCache.ORIGIN_MESSAGE_HEADER), MessageHeader.class);
             msg.setHeader(header);
             String routeLabel = header.getMsgRouteFromLable();
-            String storeKey = createStoreKey(msg, routeLabel, instance);
+
             JoinState state = createJoinState(msg, instance, routeLabel);
-            if ("left".equalsIgnoreCase(routeLabel)) {
-                joinLeftStates.put(storeKey, state);
-            } else if ("right".equalsIgnoreCase(routeLabel)) {
-                joinRightStates.put(storeKey, state);
-            }
-            if (joinLeftStates.size() > 0) {
-                storage.multiPut(joinLeftStates);
-            }
-            if (joinRightStates.size() > 0) {
-                storage.multiPut(joinRightStates);
+            List<WindowBaseValue> temp = new ArrayList<>();
+            temp.add(state);
+
+            if (WindowJoinType.left.name().equalsIgnoreCase(routeLabel)) {
+                storage.putWindowBaseValue(queueId, windowInstanceId, WindowType.JOIN_WINDOW, WindowJoinType.left, temp);
+
+            } else if (WindowJoinType.right.name().equalsIgnoreCase(routeLabel)) {
+                storage.putWindowBaseValue(queueId, windowInstanceId, WindowType.JOIN_WINDOW, WindowJoinType.right, temp);
+            } else {
+                throw new RuntimeException("param routeLabel: [" + routeLabel + "] error.");
             }
 
-            routeLabel = msg.getHeader().getMsgRouteFromLable();
-            String storeKeyPrefix = "";
-            Iterator<WindowBaseValue> iterator = null;
-            if (LABEL_LEFT.equalsIgnoreCase(routeLabel)) {
-                storeKeyPrefix = createStoreKeyPrefix(msg, LABEL_RIGHT, instance);
-                iterator = getMessageIterator(queueId, instance, msg, storeKeyPrefix, JoinRightState.class);
-            } else if (LABEL_RIGHT.equalsIgnoreCase(routeLabel)) {
-                storeKeyPrefix = createStoreKeyPrefix(msg, LABEL_LEFT, instance);
-                iterator = getMessageIterator(queueId, instance, msg, storeKeyPrefix, JoinLeftState.class);
+            Iterator<WindowBaseValue> iterator;
+            if (WindowJoinType.left.name().equalsIgnoreCase(routeLabel)) {
+                iterator = getMessageIterator(queueId, WindowJoinType.right);
+            } else if (WindowJoinType.right.name().equalsIgnoreCase(routeLabel)) {
+                iterator = getMessageIterator(queueId, WindowJoinType.left);
+            } else {
+                throw new RuntimeException("param routeLabel: [" + routeLabel + "] error.");
             }
 
             List<WindowBaseValue> tmpMessages = new ArrayList<>();
@@ -136,8 +125,8 @@
 
     }
 
-    private Iterator<WindowBaseValue> getMessageIterator(String queueId, WindowInstance instance, IMessage msg,
-        String keyPrefix, Class<? extends WindowBaseValue> clazz) {
+
+    private Iterator<WindowBaseValue> getMessageIterator(String queueId, WindowJoinType joinType) {
 
         List<WindowInstance> instances = new ArrayList<>();
         for (Map.Entry<String, WindowInstance> entry : this.windowInstanceMap.entrySet()) {
@@ -147,7 +136,7 @@
         }
         Iterator<WindowInstance> windowInstanceIter = instances.iterator();
         return new Iterator<WindowBaseValue>() {
-            private Iterator<WindowBaseValue> iterator = null;
+            private RocksdbIterator<WindowBaseValue> iterator = null;
 
             @Override
             public boolean hasNext() {
@@ -156,7 +145,7 @@
                 }
                 if (windowInstanceIter.hasNext()) {
                     WindowInstance instance = windowInstanceIter.next();
-                    iterator = storage.loadWindowInstanceSplitData(null, null, instance.createWindowInstanceId(), keyPrefix, clazz);
+                    iterator = storage.getWindowBaseValue(instance.getSplitId(), instance.getWindowInstanceId(), WindowType.JOIN_WINDOW, joinType);
                     if (iterator != null && iterator.hasNext()) {
                         return true;
                     }
@@ -166,50 +155,15 @@
 
             @Override
             public WindowBaseValue next() {
-                return iterator.next();
+                return iterator.next().getData();
             }
         };
 
     }
 
-    private Iterator<WindowBaseValue> getIterator(String queueId, String keyPrefix, WindowInstance instance,
-        Class<? extends WindowBaseValue> clazz) {
-
-        List<WindowInstance> instances = new ArrayList<>();
-        for (Map.Entry<String, WindowInstance> entry : this.windowInstanceMap.entrySet()) {
-            instances.add(entry.getValue());
-        }
-        Iterator<WindowInstance> windowInstanceIter = instances.iterator();
-        return new Iterator<WindowBaseValue>() {
-            private Iterator<WindowBaseValue> iterator = null;
-
-            @Override
-            public boolean hasNext() {
-                if (iterator != null && iterator.hasNext()) {
-                    return true;
-                }
-                if (windowInstanceIter.hasNext()) {
-                    WindowInstance instance = windowInstanceIter.next();
-                    iterator = storage.loadWindowInstanceSplitData(null, queueId, instance.createWindowInstanceId(), keyPrefix, clazz);
-                    if (iterator != null && iterator.hasNext()) {
-                        return true;
-                    } else {
-                        return false;
-                    }
-                }
-                return false;
-            }
-
-            @Override
-            public WindowBaseValue next() {
-                return iterator.next();
-            }
-        };
-
-    }
 
     public List<JSONObject> connectJoin(IMessage message, List<Map<String, Object>> rows, String joinType,
-        String rightAsName) {
+                                        String rightAsName) {
         List<JSONObject> result = new ArrayList<>();
 
         if ("inner".equalsIgnoreCase(joinType)) {
@@ -230,7 +184,7 @@
         JSONObject messageBody = message.getMessageBody();
         String traceId = message.getHeader().getTraceId();
         int index = 1;
-        if (LABEL_LEFT.equalsIgnoreCase(routeLabel) && rows.size() > 0) {
+        if (WindowJoinType.left.name().equalsIgnoreCase(routeLabel) && rows.size() > 0) {
             for (Map<String, Object> raw : rows) {
                 JSONObject object = (JSONObject) messageBody.clone();
                 object.fluentPutAll(addAsName(raw, rightAsName));
@@ -238,11 +192,11 @@
                 index++;
                 result.add(object);
             }
-        } else if (LABEL_LEFT.equalsIgnoreCase(routeLabel) && rows.size() <= 0) {
+        } else if (WindowJoinType.left.name().equalsIgnoreCase(routeLabel) && rows.size() <= 0) {
             JSONObject object = (JSONObject) messageBody.clone();
             object.put(TraceUtil.TRACE_ID_FLAG, traceId + "-" + index);
             result.add(object);
-        } else if (LABEL_RIGHT.equalsIgnoreCase(routeLabel) && rows.size() > 0) {
+        } else if (WindowJoinType.right.name().equalsIgnoreCase(routeLabel) && rows.size() > 0) {
             messageBody = addAsName(messageBody, rightAsName);
             for (Map<String, Object> raw : rows) {
                 JSONObject object = (JSONObject) messageBody.clone();
@@ -267,7 +221,7 @@
         String routeLabel = message.getHeader().getMsgRouteFromLable();
         String traceId = message.getHeader().getTraceId();
         int index = 1;
-        if (LABEL_LEFT.equalsIgnoreCase(routeLabel)) {
+        if (WindowJoinType.left.name().equalsIgnoreCase(routeLabel)) {
             JSONObject messageBody = message.getMessageBody();
             for (Map<String, Object> raw : rows) {
                 JSONObject object = (JSONObject) messageBody.clone();
@@ -298,7 +252,6 @@
         }
         for (Map.Entry<String, Object> tmp : raw.entrySet()) {
             object.put(rightAsName + "." + tmp.getKey(), tmp.getValue());
-            //            raw.remove(tmp.getKey());
         }
         return object;
     }
@@ -315,15 +268,10 @@
         String shuffleKey = message.getMessageBody().getString(WindowCache.SHUFFLE_KEY);
         String orginQueueId = message.getMessageBody().getString(WindowCache.ORIGIN_QUEUE_ID);
         String originOffset = message.getMessageBody().getString(WindowCache.ORIGIN_OFFSET);
-        String storeKey = MapKeyUtil.createKey(windowInstance.createWindowInstanceId(), shuffleKey, routeLabel, orginQueueId, originOffset);
+        String storeKey = MapKeyUtil.createKey(windowInstance.getWindowInstanceId(), shuffleKey, routeLabel, orginQueueId, originOffset);
         return storeKey;
     }
 
-    protected String createStoreKeyPrefix(IMessage message, String routeLabel, WindowInstance windowInstance) {
-        String shuffleKey = message.getMessageBody().getString(WindowCache.SHUFFLE_KEY);
-        String storeKey = MapKeyUtil.createKey(shuffleKey, routeLabel);
-        return storeKey;
-    }
 
     /**
      * 根据左右流标志对原始消息进行封装
@@ -355,9 +303,9 @@
         messageBody.remove("MessageHeader");
 
         JoinState state = null;
-        if ("left".equalsIgnoreCase(routeLabel)) {
+        if (WindowJoinType.left.name().equalsIgnoreCase(routeLabel)) {
             state = new JoinLeftState();
-        } else if ("right".equalsIgnoreCase(routeLabel)) {
+        } else if (WindowJoinType.right.name().equalsIgnoreCase(routeLabel)) {
             state = new JoinRightState();
         }
 
@@ -373,9 +321,8 @@
         String shuffleKey = message.getMessageBody().getString(WindowCache.SHUFFLE_KEY);
         String shuffleId = shuffleChannel.getChannelQueue(shuffleKey).getQueueId();
         state.setPartition(shuffleId);
-        state.setWindowInstanceId(instance.getWindowInstanceKey());
+        state.setWindowInstanceId(instance.getWindowInstanceId());
         state.setPartitionNum(incrementAndGetSplitNumber(instance, shuffleId));
-        state.setWindowInstancePartitionId(instance.getWindowInstanceKey());
 
         return state;
     }
@@ -390,9 +337,9 @@
      * @return
      */
     public static String generateKey(JSONObject messageBody, String joinLabel, List<String> leftJoinFieldNames,
-        List<String> rightJoinFieldNames) {
+                                     List<String> rightJoinFieldNames) {
         StringBuffer buffer = new StringBuffer();
-        if ("left".equalsIgnoreCase(joinLabel)) {
+        if (WindowJoinType.left.name().equalsIgnoreCase(joinLabel)) {
             for (String field : leftJoinFieldNames) {
                 String value = messageBody.getString(field);
                 buffer.append(value).append("_");
@@ -408,7 +355,7 @@
             }
         }
 
-        return StringUtil.createMD5Str(buffer.toString());
+        return buffer.toString();
     }
 
     @Override
@@ -418,16 +365,7 @@
         return messageKey;
     }
 
-    @Override
-    public Class getWindowBaseValueClass() {
-        return JoinState.class;
-    }
 
-    /**
-     * window触发后的清理工作
-     *
-     * @param windowInstance
-     */
     @Override
     public synchronized void clearFireWindowInstance(WindowInstance windowInstance) {
         List<WindowInstance> removeInstances = new ArrayList<>();
@@ -444,16 +382,43 @@
         }
 
         for (WindowInstance instance : removeInstances) {
+            //清理MaxPartitionNum
+            storage.deleteMaxPartitionNum(instance.getSplitId(), instance.getWindowInstanceId());
 
-            windowMaxValueManager.deleteSplitNum(instance, instance.getSplitId());
-            ShufflePartitionManager.getInstance().clearWindowInstance(instance.createWindowInstanceId());
-            storage.delete(instance.createWindowInstanceId(), null, WindowBaseValue.class, sqlCache);
-            if (!isLocalStorageOnly) {
-                WindowInstance.clearInstance(instance, sqlCache);
-                joinOperator.cleanMessage(instance.getWindowNameSpace(), instance.getWindowName(), this.getRetainWindowCount(), this.getSizeInterval(), windowInstance.getStartTime());
+            //从windowInstance表中删除
+            storage.deleteWindowInstance(instance.getSplitId(), this.getNameSpace(), this.getConfigureName(), instance.getWindowInstanceId());
+
+
+            //从JoinState表中删除
+            deleteFromJoinState(instance, WindowJoinType.right);
+            deleteFromJoinState(instance, WindowJoinType.left);
+        }
+    }
+
+    private void deleteFromJoinState(WindowInstance instance, WindowJoinType windowJoinType) {
+
+        RocksdbIterator<JoinState> joinStates = storage.getWindowBaseValue(instance.getSplitId(), instance.getWindowInstanceId(), WindowType.JOIN_WINDOW, windowJoinType);
+        while (joinStates.hasNext()) {
+            IteratorWrap<JoinState> next = joinStates.next();
+
+            JoinState joinState = next.getData();
+            Date start = addTime(instance.getStartTime(), TimeUnit.MINUTES, -retainWindowCount * sizeInterval);
+
+            if (canDelete(instance, joinState, start)) {
+                storage.deleteWindowBaseValue(instance.getSplitId(), instance.getWindowInstanceId(), WindowType.JOIN_WINDOW, windowJoinType);
             }
         }
+    }
 
+    private boolean canDelete(WindowInstance instance, JoinState joinState, Date start) {
+        return instance.getWindowNameSpace().equals(joinState.getWindowNameSpace())
+                && instance.getWindowName().equals(joinState.getWindowName())
+                && instance.getGmtCreate().getTime() < start.getTime();
+    }
+
+    private Date addTime(String time, TimeUnit unit, int value) {
+        Date date = DateUtil.parseTime(time);
+        return DateUtil.addDate(unit, date, value);
     }
 
     protected List<Map<String, Object>> matchRows(JSONObject msg, List<Map<String, Object>> rows) {
@@ -540,24 +505,6 @@
         }
     }
 
-    @Override
-    protected Long queryWindowInstanceMaxSplitNum(WindowInstance instance) {
-        Long leftMaxSplitNum = storage.getMaxSplitNum(instance, JoinLeftState.class);
-        Long rigthMaxSplitNum = storage.getMaxSplitNum(instance, JoinRightState.class);
-        if (leftMaxSplitNum == null) {
-            return rigthMaxSplitNum;
-        }
-        if (rigthMaxSplitNum == null) {
-            return leftMaxSplitNum;
-        }
-        if (leftMaxSplitNum >= rigthMaxSplitNum) {
-            return leftMaxSplitNum;
-        }
-        if (leftMaxSplitNum < rigthMaxSplitNum) {
-            return rigthMaxSplitNum;
-        }
-        return null;
-    }
 
     @Override
     public boolean supportBatchMsgFinish() {
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/Operator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/Operator.java
deleted file mode 100644
index a50d612..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/Operator.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.operator.join;
-
-import com.alibaba.fastjson.JSONObject;
-import java.security.MessageDigest;
-import java.util.List;
-
-public interface Operator {
-
-    public static String generateKey(JSONObject messageBody, String joinLabel, List<String> leftJoinFieldNames,
-        List<String> rightJoinFieldNames) {
-        StringBuffer buffer = new StringBuffer();
-        if ("left".equalsIgnoreCase(joinLabel)) {
-            for (String field : leftJoinFieldNames) {
-                String value = messageBody.getString(field);
-                buffer.append(value).append("_");
-            }
-        } else {
-            for (String field : rightJoinFieldNames) {
-                String[] rightFields = field.split(".");
-                if (rightFields.length > 1) {
-                    field = rightFields[1];
-                }
-                String value = messageBody.getString(field);
-                buffer.append(value).append("_");
-            }
-        }
-
-        buffer.charAt(buffer.length() - 1);
-
-        return MD5(buffer.toString());
-    }
-
-    public static String MD5(String s) {
-        char hexDigits[] = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F'};
-
-        try {
-            byte[] btInput = s.getBytes();
-            // 获得MD5摘要算法的 MessageDigest 对象
-            MessageDigest mdInst = MessageDigest.getInstance("MD5");
-            // 使用指定的字节更新摘要
-            mdInst.update(btInput);
-            // 获得密文
-            byte[] md = mdInst.digest();
-            // 把密文转换成十六进制的字符串形式
-            int j = md.length;
-            char str[] = new char[j * 2];
-            int k = 0;
-            for (int i = 0; i < j; i++) {
-                byte byte0 = md[i];
-                str[k++] = hexDigits[byte0 >>> 4 & 0xf];
-                str[k++] = hexDigits[byte0 & 0xf];
-            }
-            return new String(str);
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new RuntimeException(e.getMessage(), e);
-        }
-    }
-
-}
\ No newline at end of file
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java
index 9e90997..6c4659b 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/AbstractSystemChannel.java
@@ -57,7 +57,6 @@
     protected ISource consumer;
     protected AbstractSupportShuffleSink producer;
     protected Map<String, String> channelConfig = new HashMap<>();
-    ;
     protected boolean hasCreateShuffleChannel = false;
 
     public void startChannel() {
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java
index a52e556..348cefb 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleCache.java
@@ -22,52 +22,104 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.Future;
+
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.rocketmq.streams.common.context.IMessage;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
+import org.apache.rocketmq.streams.common.context.MessageOffset;
 import org.apache.rocketmq.streams.window.debug.DebugWriter;
 import org.apache.rocketmq.streams.window.model.WindowCache;
 import org.apache.rocketmq.streams.window.model.WindowInstance;
-import org.apache.rocketmq.streams.window.offset.WindowMaxValue;
 import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow;
-import org.apache.rocketmq.streams.window.sqlcache.impl.SplitSQLElement;
+import org.apache.rocketmq.streams.window.storage.IStorage;
 
 /**
  * save receiver messages into cachefilter when checkpoint/autoflush/flush, process cachefilter message
  */
 public class ShuffleCache extends WindowCache {
     protected AbstractShuffleWindow window;
+    private HashMap<String, Boolean> hasLoad = new HashMap<>();
 
     public ShuffleCache(AbstractShuffleWindow window) {
         this.window = window;
     }
 
+    /**
+     * 调用时机:ShuffleChannel从上游读到shuffle数据,加入缓存后,
+     * 满足条件: 定时/条数大于特定值/checkpoint/开始接收批量消息 时触发此方法
+     *
+     * @param messageList
+     * @return
+     */
     @Override
     protected boolean batchInsert(List<IMessage> messageList) {
-        Map<Pair<String, String>, List<IMessage>> instance2Messages = new HashMap<>();
-        Map<String, WindowInstance> windowInstanceMap = new HashMap<>();
+        Map<Pair<String/*queueId*/, String/*windowInstanceId*/>, List<IMessage>> instance2Messages = new HashMap<>();
+        Map<String/*windowInstanceId*/, WindowInstance> windowInstanceMap = new HashMap<>();
+
         groupByWindowInstanceAndQueueId(messageList, instance2Messages, windowInstanceMap);
+
         List<Pair<String, String>> keys = new ArrayList<>(instance2Messages.keySet());
         Collections.sort(keys);
+
         for (Pair<String, String> queueIdAndInstanceKey : keys) {
             String queueId = queueIdAndInstanceKey.getLeft();
             String windowInstanceId = queueIdAndInstanceKey.getRight();
+
             List<IMessage> messages = instance2Messages.get(queueIdAndInstanceKey);
+
             WindowInstance windowInstance = windowInstanceMap.get(windowInstanceId);
+
             DebugWriter.getDebugWriter(window.getConfigureName()).writeShuffleReceive(window, messages, windowInstance);
+
+            stateMustLoad(queueId);
+
             window.shuffleCalculate(messages, windowInstance, queueId);
+
+            //保存处理进度
             saveSplitProgress(queueId, messages);
         }
         return true;
     }
 
+    private void stateMustLoad(String queueId) {
+        Boolean load = this.hasLoad.get(queueId);
+        if (load != null && load) {
+            return;
+        }
+
+        //在计算之前需要异步加载状态完成
+        HashMap<String, Future<?>> loadResult = this.window.getShuffleChannel().getLoadResult();
+        Future<?> future = loadResult.get(queueId);
+
+        if (future == null) {
+            return;
+        }
+
+        try {
+            long before = System.currentTimeMillis();
+            future.get();
+            long after = System.currentTimeMillis();
+
+            System.out.println("message wait before state recover:[" + (after - before) + "] ms, queueId=" + queueId);
+
+            hasLoad.put(queueId, true);
+        } catch (Throwable t) {
+            throw new RuntimeException("check remote with queueId:" + queueId + ",error", t);
+        }
+    }
+
     /**
-     * save consumer progress(offset)for groupby  source queueId
+     * save consumer progress(offset)for groupby  source shuffleId
+     * window configName: name_window_10001
+     * shuffleId: shuffle_NormalTestTopic_namespace_name_broker-a_001
+     * oriQueueId: NormalTestTopic2_broker-a_000
      *
-     * @param queueId
+     * @param shuffleId
      * @param messages
      */
-    protected void saveSplitProgress(String queueId, List<IMessage> messages) {
+    protected void saveSplitProgress(String shuffleId, List<IMessage> messages) {
+        IStorage delegator = this.window.getStorage();
+
         Map<String, String> queueId2OrigOffset = new HashMap<>();
         Boolean isLong = false;
         for (IMessage message : messages) {
@@ -76,11 +128,19 @@
             String oriOffset = message.getMessageBody().getString(WindowCache.ORIGIN_OFFSET);
             queueId2OrigOffset.put(oriQueueId, oriOffset);
         }
-        Map<String, WindowMaxValue> windowMaxValueMap = window.getWindowMaxValueManager().saveMaxOffset(isLong, window.getConfigureName(), queueId, queueId2OrigOffset);
-        window.getSqlCache().addCache(new SplitSQLElement(queueId, ORMUtil.createBatchReplacetSQL(new ArrayList<>(windowMaxValueMap.values()))));
 
+        for (String oriQueueId : queueId2OrigOffset.keySet()) {
+            String currentOffset = queueId2OrigOffset.get(oriQueueId);
+
+            String remoteMaxOffset = delegator.getMaxOffset(shuffleId, window.getConfigureName(), oriQueueId);
+
+            if (remoteMaxOffset == null || MessageOffset.greateThan(currentOffset, remoteMaxOffset, isLong)) {
+                delegator.putMaxOffset(shuffleId, window.getConfigureName(), oriQueueId, currentOffset);
+            }
+        }
     }
 
+
     @Override
     protected String generateShuffleKey(IMessage message) {
         return null;
@@ -94,7 +154,7 @@
      * @param windowInstanceMap
      */
     protected void groupByWindowInstanceAndQueueId(List<IMessage> messageList,
-        Map<Pair<String, String>, List<IMessage>> instance2Messages, Map<String, WindowInstance> windowInstanceMap) {
+                                                   Map<Pair<String, String>, List<IMessage>> instance2Messages, Map<String, WindowInstance> windowInstanceMap) {
         for (IMessage message : messageList) {
             //the queueId will be replace below, so get first here!
             String queueId = message.getHeader().getQueueId();
@@ -106,13 +166,13 @@
             message.getHeader().setOffsetIsLong(isLong);
             List<WindowInstance> windowInstances = (List<WindowInstance>) message.getMessageBody().get(WindowInstance.class.getSimpleName());
             for (WindowInstance windowInstance : windowInstances) {
-                String windowInstanceId = windowInstance.createWindowInstanceId();
+                String windowInstanceId = windowInstance.getWindowInstanceId();
                 Pair<String, String> queueIdAndInstanceKey = Pair.of(queueId, windowInstanceId);
                 List<IMessage> messages = instance2Messages.computeIfAbsent(queueIdAndInstanceKey, k -> new ArrayList<>());
                 //in case of changing message concurrently in hop window
                 IMessage cloneMessage = message.deepCopy();
                 //bring window instance id into accumulator computation
-                cloneMessage.getMessageBody().put("HIT_WINDOW_INSTANCE_ID", windowInstance.createWindowInstanceId());
+                cloneMessage.getMessageBody().put("HIT_WINDOW_INSTANCE_ID", windowInstance.getWindowInstanceId());
                 messages.add(cloneMessage);
                 windowInstanceMap.put(windowInstanceId, windowInstance);
             }
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleChannel.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleChannel.java
index 26556db..b938ac9 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleChannel.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/shuffle/ShuffleChannel.java
@@ -18,16 +18,6 @@
 
 import com.alibaba.fastjson.JSONArray;
 import com.alibaba.fastjson.JSONObject;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -48,21 +38,26 @@
 import org.apache.rocketmq.streams.common.interfaces.ISystemMessage;
 import org.apache.rocketmq.streams.common.topology.ChainPipeline;
 import org.apache.rocketmq.streams.common.topology.model.Pipeline;
-import org.apache.rocketmq.streams.common.utils.CollectionUtil;
 import org.apache.rocketmq.streams.common.utils.CompressUtil;
-import org.apache.rocketmq.streams.common.utils.DateUtil;
 import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
 import org.apache.rocketmq.streams.common.utils.StringUtil;
 import org.apache.rocketmq.streams.common.utils.TraceUtil;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
 import org.apache.rocketmq.streams.window.debug.DebugWriter;
 import org.apache.rocketmq.streams.window.model.WindowCache;
 import org.apache.rocketmq.streams.window.model.WindowInstance;
 import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow;
 import org.apache.rocketmq.streams.window.operator.AbstractWindow;
-import org.apache.rocketmq.streams.window.operator.impl.WindowOperator.WindowRowOperator;
-import org.apache.rocketmq.streams.window.sqlcache.impl.SQLElement;
-import org.apache.rocketmq.streams.window.storage.ShufflePartitionManager;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
 
 import static org.apache.rocketmq.streams.window.model.WindowCache.ORIGIN_MESSAGE_TRACE_ID;
 
@@ -87,15 +82,14 @@
 
     // protected NotifyChannel notfiyChannel;//负责做分片的通知管理
     protected AbstractShuffleWindow window;
-    private Set<String> currentQueueIds;//当前管理的分片
 
     protected transient boolean isWindowTest = false;
 
+    private transient HashMap<String, Future<?>> loadResult = new HashMap<>();
     /**
      * 每个分片,已经确定处理的最大offset
      */
-    protected transient Map<String, String> split2MaxOffsets = new HashMap<>();
-
+//    protected transient Map<String, String> split2MaxOffsets = new HashMap<>();
     public ShuffleChannel(AbstractShuffleWindow window) {
         this.window = window;
         channelConfig = new HashMap<>();
@@ -143,7 +137,7 @@
 
             this.queueMap = tmp;
         }
-        isWindowTest = ComponentCreator.getPropertyBooleanValue("window.fire.isTest");
+//        isWindowTest = ComponentCreator.getPropertyBooleanValue("window.fire.isTest");
     }
 
     /**
@@ -191,7 +185,7 @@
             message.getHeader().setQueueId(queueId);
             message.getMessageBody().put(SHUFFLE_OFFSET, oriMessage.getHeader().getOffset());
             window.updateMaxEventTime(message);
-            if (isRepeateMessage(message, queueId)) {
+            if (isRepeatMessage(message, queueId)) {
                 continue;
             }
             List<WindowInstance> windowInstances = window.queryOrCreateWindowInstance(message, queueId);
@@ -200,12 +194,9 @@
                 continue;
             }
             for (WindowInstance windowInstance : windowInstances) {
-                String windowInstanceId = windowInstance.createWindowInstanceId();
                 //new instance, not need load data from remote
                 if (windowInstance.isNewWindowInstance()) {
-                    window.getSqlCache().addCache(new SQLElement(windowInstance.getSplitId(), windowInstanceId, ORMUtil.createBatchReplacetSQL(windowInstance)));
                     windowInstance.setNewWindowInstance(false);
-                    ShufflePartitionManager.getInstance().setWindowInstanceFinished(windowInstance.createWindowInstanceId());
                 }
             }
 
@@ -225,63 +216,34 @@
         }
         if (isWindowTest) {
             long count = COUNT.addAndGet(messages.size());
-            System.out.println(window.getConfigureName() + " receive shuffle msg count is " + count);
+            System.out.println(window.getConfigureName() + " receive total shuffle msg count is " + count + ",this batch size: " + messages.size());
         }
 
         return null;
     }
 
+    //加载状态数据到本地存储
     @Override
     public void addNewSplit(IMessage message, AbstractContext context, NewSplitMessage newSplitMessage) {
-        this.currentQueueIds = newSplitMessage.getCurrentSplitIds();
-        loadSplitProgress(newSplitMessage);
+        Set<String> splitIds = newSplitMessage.getSplitIds();
 
-        List<WindowInstance> allWindowInstances = WindowInstance.queryAllWindowInstance(DateUtil.getCurrentTimeString(), window, newSplitMessage.getSplitIds());
-        if (CollectionUtil.isNotEmpty(allWindowInstances)) {
-            Map<String, Set<WindowInstance>> queueId2WindowInstances = new HashMap<>();
-            for (WindowInstance windowInstance : allWindowInstances) {
-                windowInstance.setNewWindowInstance(false);
-                window.registerWindowInstance(windowInstance);
-                window.getWindowFireSource().registFireWindowInstanceIfNotExist(windowInstance, window);
-                String queueId = windowInstance.getSplitId();
-                window.getStorage().loadSplitData2Local(queueId, windowInstance.createWindowInstanceId(), window.getWindowBaseValueClass(), new WindowRowOperator(windowInstance, queueId, window));
-                window.initWindowInstanceMaxSplitNum(windowInstance);
-            }
-
-        } else {
-            for (String queueId : newSplitMessage.getSplitIds()) {
-                ShufflePartitionManager.getInstance().setSplitFinished(queueId);
-            }
+        for (String splitId : splitIds) {
+            Future<?> future = this.window.getStorage().load(splitId);
+            this.loadResult.put(splitId, future);
         }
+
         window.getFireReceiver().doMessage(message, context);
     }
 
-    /**
-     * load ori split consume offset
-     *
-     * @param newSplitMessage
-     */
-    protected void loadSplitProgress(NewSplitMessage newSplitMessage) {
-        for (String queueId : newSplitMessage.getSplitIds()) {
-            Map<String, String> result = window.getWindowMaxValueManager().loadOffsets(window.getConfigureName(), queueId);
-            if (result != null) {
-                this.split2MaxOffsets.putAll(result);
-            }
-        }
-    }
 
     @Override
     public void removeSplit(IMessage message, AbstractContext context, RemoveSplitMessage removeSplitMessage) {
-        this.currentQueueIds = removeSplitMessage.getCurrentSplitIds();
+        //去掉缓存中的
         Set<String> queueIds = removeSplitMessage.getSplitIds();
         if (queueIds != null) {
             for (String queueId : queueIds) {
-                ShufflePartitionManager.getInstance().setSplitInValidate(queueId);
                 window.clearCache(queueId);
-
             }
-            window.getWindowMaxValueManager().removeKeyPrefixFromLocalCache(queueIds);
-            //window.getWindowFireSource().removeSplit(queueIds);
         }
         window.getFireReceiver().doMessage(message, context);
     }
@@ -290,19 +252,14 @@
     public void checkpoint(IMessage message, AbstractContext context, CheckPointMessage checkPointMessage) {
         if (message.getHeader().isNeedFlush()) {
             this.flush(message.getHeader().getCheckpointQueueIds());
-            window.getSqlCache().flush(message.getHeader().getCheckpointQueueIds());
+            window.getStorage().flush(new ArrayList<>(message.getHeader().getCheckpointQueueIds()));
         }
         CheckPointState checkPointState = new CheckPointState();
         checkPointState.setQueueIdAndOffset(this.shuffleCache.getFinishedQueueIdAndOffsets(checkPointMessage));
         checkPointMessage.reply(checkPointState);
     }
 
-    /**
-     * do system message
-     *
-     * @param oriMessage
-     * @param context
-     */
+
     protected void doSystemMessage(IMessage oriMessage, AbstractContext context) {
         ISystemMessage systemMessage = oriMessage.getSystemMessage();
         if (systemMessage instanceof CheckPointMessage) {
@@ -312,9 +269,6 @@
         } else if (systemMessage instanceof RemoveSplitMessage) {
             this.removeSplit(oriMessage, context, (RemoveSplitMessage) systemMessage);
         } else if (systemMessage instanceof BatchFinishMessage) {
-            //            if(COUNT.get()!=88121){
-            //                throw new RuntimeException("fired before receiver");
-            //            }
             System.out.println("start fire window by fininsh flag " + oriMessage.getHeader().getQueueId());
             this.batchMessageFinish(oriMessage, context, (BatchFinishMessage) systemMessage);
         } else {
@@ -330,15 +284,16 @@
      * @param queueId
      * @return
      */
-    protected boolean isRepeateMessage(IMessage message, String queueId) {
+    protected boolean isRepeatMessage(IMessage message, String queueId) {
         boolean isOrigOffsetLong = message.getMessageBody().getBoolean(WindowCache.ORIGIN_QUEUE_IS_LONG);
         String oriQueueId = message.getMessageBody().getString(WindowCache.ORIGIN_QUEUE_ID);
         String oriOffset = message.getMessageBody().getString(WindowCache.ORIGIN_OFFSET);
-        String key = MapKeyUtil.createKey(window.getConfigureName(), queueId, oriQueueId);
-        String offset = this.split2MaxOffsets.get(key);
-        if (offset != null) {
-            MessageOffset messageOffset = new MessageOffset(oriOffset, isOrigOffsetLong);
-            if (!messageOffset.greateThan(offset)) {
+
+        //由storage统一缓存,方便管理一致性
+        String maxOffset = this.window.getStorage().getMaxOffset(queueId, window.getConfigureName(), oriQueueId);
+
+        if (maxOffset != null) {
+            if (!MessageOffset.greateThan(oriOffset, maxOffset, isOrigOffsetLong)) {
                 System.out.println("the message offset is old, the message is discard ");
                 return true;
             }
@@ -372,7 +327,7 @@
     @Override
     protected String createShuffleTopic(String topic, ChainPipeline pipeline) {
         return "shuffle_" + topic + "_" + pipeline.getSource().getNameSpace().replaceAll("\\.", "_") + "_" + pipeline
-            .getConfigureName().replaceAll("\\.", "_").replaceAll(";", "_");
+                .getConfigureName().replaceAll("\\.", "_").replaceAll(";", "_");
     }
 
     /**
@@ -449,8 +404,7 @@
 
     public ISplit getChannelQueue(String key) {
         int index = hash(key);
-        ISplit targetQueue = queueList.get(index);
-        return targetQueue;
+        return queueList.get(index);
     }
 
     public int hash(Object key) {
@@ -514,9 +468,6 @@
         return splitNum > 0 ? splitNum : 32;
     }
 
-    public Set<String> getCurrentQueueIds() {
-        return currentQueueIds;
-    }
 
     public List<ISplit> getQueueList() {
         return queueList;
@@ -530,13 +481,20 @@
     public void batchMessageFinish(IMessage message, AbstractContext context, BatchFinishMessage batchFinishMessage) {
         if (window.supportBatchMsgFinish()) {
             shuffleCache.flush(message.getHeader().getQueueId());
-            Set<String> queueIds = new HashSet();
+            List<String> queueIds = new ArrayList<>();
             queueIds.add(message.getHeader().getQueueId());
-            window.getSqlCache().flush(queueIds);
+
+
+            window.getStorage().flush(queueIds);
+
             window.getWindowFireSource().fireWindowInstance(message.getHeader().getQueueId());
             IMessage cpMsg = batchFinishMessage.getMsg().copy();
             window.getFireReceiver().doMessage(cpMsg, context);
         }
 
     }
+
+    public HashMap<String, Future<?>> getLoadResult() {
+        return loadResult;
+    }
 }
\ No newline at end of file
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/ISQLElement.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/ISQLElement.java
deleted file mode 100644
index a6a18ff..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/ISQLElement.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.streams.window.sqlcache;
-
-public interface ISQLElement {
-
-    boolean isWindowInstanceSQL();
-
-    /**
-     * window max value for max offset
-     *
-     * @return
-     */
-    boolean isSplitSQL();
-
-    /**
-     * fire message, can cancel not commit sqls which owned the windowinstance
-     *
-     * @return
-     */
-    boolean isFireNotify();
-
-    String getQueueId();
-
-    String getWindowInstanceId();
-
-    String getSQL();
-
-    Integer getIndex();
-
-    void setIndex(int index);
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java
deleted file mode 100644
index bfe7a2b..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/SQLCache.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.streams.window.sqlcache;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.rocketmq.streams.common.channel.sinkcache.IMessageFlushCallBack;
-import org.apache.rocketmq.streams.common.channel.sinkcache.impl.AbstractMultiSplitMessageCache;
-import org.apache.rocketmq.streams.db.driver.DriverBuilder;
-import org.apache.rocketmq.streams.db.driver.JDBCDriver;
-
-/**
- * cache sql, async and batch commit
- */
-
-public class SQLCache extends AbstractMultiSplitMessageCache<ISQLElement> {
-    protected Boolean isOpenCache = true;//if false,then execute sql when receive sql
-    protected Set<String> firedWindowInstances = new HashSet<>();//fired window instance ,if the owned sqls have not commit, can cancel the sqls
-    protected Map<String, Integer> windowInstance2Index = new HashMap<>();//set index to ISQLElement group by window instance
-
-    protected boolean isLocalOnly;
-
-    public SQLCache(boolean isLocalOnly) {
-        super(null);
-        this.isLocalOnly = isLocalOnly;
-        this.flushCallBack = new MessageFlushCallBack(new SQLCacheCallback());
-        this.setBatchSize(1000);
-        this.setAutoFlushTimeGap(10 * 1000);
-        this.setAutoFlushSize(100);
-        this.openAutoFlush();
-    }
-
-    @Override
-    public int addCache(ISQLElement isqlElement) {
-        if (isLocalOnly) {
-            return 0;
-        }
-        if (isOpenCache == false) {
-            DriverBuilder.createDriver().execute(isqlElement.getSQL());
-            return 1;
-        }
-        if (isqlElement.isFireNotify()) {
-            firedWindowInstances.add(isqlElement.getWindowInstanceId());
-        } else if (isqlElement.isWindowInstanceSQL()) {
-            Integer index = windowInstance2Index.get(isqlElement.getWindowInstanceId());
-            if (index == null) {
-                index = 0;
-            }
-            index++;
-            isqlElement.setIndex(index);
-            windowInstance2Index.put(isqlElement.getWindowInstanceId(), index);
-        }
-
-        return super.addCache(isqlElement);
-    }
-
-    @Override
-    protected String createSplitId(ISQLElement msg) {
-        return msg.getQueueId();
-    }
-
-    protected AtomicInteger executeSQLCount = new AtomicInteger(0);
-    protected AtomicInteger cancelQLCount = new AtomicInteger(0);
-
-    protected class SQLCacheCallback implements IMessageFlushCallBack<ISQLElement> {
-
-        @Override
-        public boolean flushMessage(List<ISQLElement> messages) {
-            List<String> sqls = new ArrayList<>();
-
-            for (ISQLElement isqlElement : messages) {
-                if (isqlElement.isSplitSQL()) {
-                    sqls.add(isqlElement.getSQL());
-                } else if (isqlElement.isWindowInstanceSQL()) {
-                    sqls.add(isqlElement.getSQL());
-                } else if (isqlElement.isFireNotify()) {
-                    windowInstance2Index.remove(isqlElement.getWindowInstanceId());
-                    firedWindowInstances.remove(isqlElement.getWindowInstanceId());
-
-                }
-            }
-            if (sqls.size() == 0) {
-                return true;
-            }
-            JDBCDriver dataSource = DriverBuilder.createDriver();
-            try {
-                executeSQLCount.addAndGet(sqls.size());
-                dataSource.executSqls(sqls);
-                System.out.println("execute sql count is " + executeSQLCount.get() + ";  cancel sql count is " + cancelQLCount.get());
-            } catch (Exception e) {
-                e.printStackTrace();
-                throw new RuntimeException(e);
-            } finally {
-                if (dataSource != null) {
-                    dataSource.destroy();
-                }
-            }
-            return true;
-        }
-    }
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/FiredNotifySQLElement.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/FiredNotifySQLElement.java
deleted file mode 100644
index b06b691..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/FiredNotifySQLElement.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.streams.window.sqlcache.impl;
-
-import org.apache.rocketmq.streams.window.sqlcache.ISQLElement;
-
-public class FiredNotifySQLElement implements ISQLElement {
-    protected String queueId;
-    protected String windowInstanceId;
-    public FiredNotifySQLElement(String splitId,String windowInstanceId){
-        this.queueId=splitId;
-        this.windowInstanceId=windowInstanceId;
-    }
-
-    @Override public boolean isWindowInstanceSQL() {
-        return false;
-    }
-
-    @Override public boolean isSplitSQL() {
-        return false;
-    }
-
-    @Override public boolean isFireNotify() {
-        return true;
-    }
-
-    @Override
-    public String getQueueId() {
-        return queueId;
-    }
-
-    @Override
-    public String getWindowInstanceId() {
-        return windowInstanceId;
-    }
-
-    @Override
-    public String getSQL() {
-        throw new RuntimeException("can not support this method");
-    }
-
-    @Override public Integer getIndex() {
-        return null;
-    }
-
-    @Override
-    public void setIndex(int index) {
-
-    }
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/SQLElement.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/SQLElement.java
deleted file mode 100644
index 94abbf9..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/SQLElement.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.streams.window.sqlcache.impl;
-
-import org.apache.rocketmq.streams.window.sqlcache.ISQLElement;
-
-public class SQLElement implements ISQLElement {
-    protected String queueId;
-    protected String windowInstanceId;
-    protected String sql;
-    protected Integer index;
-    public SQLElement(String splitId,String windowInstanceId,String sql){
-        this.queueId=splitId;
-        this.windowInstanceId=windowInstanceId;
-        this.sql=sql;
-    }
-
-    @Override public boolean isWindowInstanceSQL() {
-        return true;
-    }
-
-    @Override public boolean isSplitSQL() {
-        return false;
-    }
-
-    @Override public boolean isFireNotify() {
-        return false;
-    }
-
-    @Override
-    public String getQueueId() {
-        return queueId;
-    }
-
-    @Override
-    public String getWindowInstanceId() {
-        return windowInstanceId;
-    }
-
-    @Override
-    public String getSQL() {
-        return sql;
-    }
-
-    @Override public Integer getIndex() {
-        return index;
-    }
-
-    @Override
-    public void setIndex(int index) {
-        this.index = index;
-    }
-
-
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/SplitSQLElement.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/SplitSQLElement.java
deleted file mode 100644
index ddf5e2f..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/sqlcache/impl/SplitSQLElement.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.streams.window.sqlcache.impl;
-
-import org.apache.rocketmq.streams.window.sqlcache.ISQLElement;
-
-public class SplitSQLElement implements ISQLElement {
-    protected String queueId;
-    protected String sql;
-    public SplitSQLElement(String splitId,String sql){
-        this.queueId=splitId;
-        this.sql=sql;
-    }
-
-
-    @Override public boolean isWindowInstanceSQL() {
-        return false;
-    }
-
-    @Override public boolean isSplitSQL() {
-        return true;
-    }
-
-    @Override public boolean isFireNotify() {
-        return false;
-    }
-
-    @Override public String getQueueId() {
-        return queueId;
-    }
-
-    @Override public String getWindowInstanceId() {
-        throw new RuntimeException("can not support this method");
-    }
-
-    @Override public String getSQL() {
-        return sql;
-    }
-
-    @Override public Integer getIndex() {
-        return null;
-    }
-
-    @Override
-    public void setIndex(int index) {
-
-    }
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/WindowBaseValue.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/WindowBaseValue.java
index 0129553..2aeb3c4 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/WindowBaseValue.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/WindowBaseValue.java
@@ -32,12 +32,14 @@
     private static final long serialVersionUID = -4985883726971532986L;
 
     /**
-     * 唯一标识一个groupBy的窗口计算数据 创建唯一主键 内存及磁盘里使用(partition,windowNamespace,windowName,startTime,endOrFireTime,groupByValue)的拼接 数据库里用上面拼接字符串的MD5
+     * 唯一标识
+     * queueId + windowInstanceId + groupBy
      */
     protected String msgKey;
 
     /**
-     * 唯一标识一个窗口 内存及磁盘使用(patitionId,windowNamespace,windowName,startTime,endOrFireTime) 数据库里使用上面字符串的MD5
+     * 属于那个windowInstance
+     * splitId, windowNameSpace, windowName, windowInstanceName, startTime, endTime
      */
     protected String windowInstanceId;
 
@@ -52,11 +54,6 @@
     protected long partitionNum;
 
     /**
-     * 标识一个分片同一个窗口 内存及磁盘使用(partition,windowNamespace,windowName,windowinstanceName,startTime,endTime,partition) 数据库里使用上面字符串的MD5值
-     */
-    protected String windowInstancePartitionId;
-
-    /**
      * 窗口实例的开始时间
      */
     protected String startTime;
@@ -166,13 +163,7 @@
         this.updateVersion.set(updateVersion);
     }
 
-    public String getWindowInstancePartitionId() {
-        return windowInstancePartitionId;
-    }
 
-    public void setWindowInstancePartitionId(String windowInstancePartitionId) {
-        this.windowInstancePartitionId = windowInstancePartitionId;
-    }
 
     @Override
     public WindowBaseValue clone() {
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/WindowValue.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/WindowValue.java
index 2de397d..5aeb0c1 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/WindowValue.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/state/impl/WindowValue.java
@@ -242,7 +242,9 @@
                 this.maxOffset.put(queueId, offset);
             } else {
                 //如果比最大的offset 小或等于,则直接丢弃掉消息
-                System.out.println("!!!!!!!!!!!!!!!!!!! has outOfOrder data " + maxOffsetOfQueue + " " + message.getHeader().getOffset());
+                System.out.println("!!!!!!!!!!!!!!!!!!! has outOfOrder data."
+                        + "queueId= " + queueId + ",maxOffsetOfQueue="+ maxOffsetOfQueue
+                        + ",messageOffset=" + message.getHeader().getOffset());
                 return false;
             }
         }
@@ -318,58 +320,6 @@
         }
     }
 
-    /**
-     * merge different window values into one window value which have the same group by value
-     *
-     * @param window          the window definition
-     * @param windowInstances all window instance which belong to same window and have different group by value
-     * @return
-     */
-    public static List<WindowValue> mergeWindowValues(AbstractWindow window, List<WindowInstance> windowInstances) {
-        if (windowInstances == null || windowInstances.size() == 0) {
-            return new ArrayList<>();
-        }
-        StringBuilder sb = new StringBuilder();
-        boolean isFirst = true;
-        String name = MapKeyUtil.createKey(window.getNameSpace(), window.getConfigureName());
-        for (WindowInstance windowInstance : windowInstances) {
-            if (isFirst) {
-                isFirst = false;
-            } else {
-                sb.append(",");
-            }
-            sb.append("('" + name + "','" + windowInstance.getStartTime() + "','" + windowInstance.getEndTime() + "')");
-        }
-        String inSQL = sb.toString();
-        /**
-         * 分批,内存撑暴 todo
-         */
-        String sql = "select * from " + ORMUtil
-            .getTableName(WindowValue.class) + " where status > 0 && (name, start_time, end_time) in (" + inSQL + ")";
-        Map<String, Object> paras = new HashMap<>(4);
-        List<WindowValue> windowValueList = ORMUtil.queryForList(sql, paras, WindowValue.class);
-        return queryMergeWindowValues(window, windowValueList);
-    }
-
-    public static List<WindowValue> queryMergeWindowValues(AbstractWindow window, List<WindowValue> windowValueList) {
-        Map<String, List<WindowValue>> groupWindowMap = new HashMap<>(64);
-        for (WindowValue value : windowValueList) {
-            String key = MapKeyUtil.createKeyBySign(value.getStartTime(), value.getEndTime(),
-                value.getGroupBy());
-            if (groupWindowMap.containsKey(key)) {
-                groupWindowMap.get(key).add(value);
-            } else {
-                groupWindowMap.put(key, new ArrayList<WindowValue>() {{
-                    add(value);
-                }});
-            }
-        }
-        List<WindowValue> mergedValueList = new ArrayList<>();
-        for (Entry<String, List<WindowValue>> entry : groupWindowMap.entrySet()) {
-            mergedValueList.add(mergeWindowValue(window, entry.getValue()));
-        }
-        return mergedValueList;
-    }
 
     /**
      * merge the group which has the same group by value and different split id
@@ -475,7 +425,6 @@
         clonedValue.setMsgKey(msgKey);
         clonedValue.setAggColumnMap(aggColumnResult);
         clonedValue.setMaxOffset(getMaxOffset());
-        clonedValue.setWindowInstancePartitionId(windowInstancePartitionId);
         clonedValue.setWindowInstanceId(windowInstanceId);
         clonedValue.setPartition(partition);
         clonedValue.setPartitionNum(partitionNum);
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/AbstractStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/AbstractStorage.java
new file mode 100644
index 0000000..e3f8472
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/AbstractStorage.java
@@ -0,0 +1,111 @@
+package org.apache.rocketmq.streams.window.storage;
+/*
+ * 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.
+ */
+
+import org.apache.rocketmq.streams.window.model.WindowInstance;
+import org.apache.rocketmq.streams.window.state.WindowBaseValue;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+public abstract class AbstractStorage implements IStorage {
+
+    @Override
+    public Future<?> load(String shuffleId) {
+        return new NullFuture();
+    }
+
+    @Override
+    public int flush(List<String> queueId) {
+        return 0;
+    }
+
+    @Override
+    public void clearCache(String queueId) {
+    }
+
+    protected String getCurrentTimestamp() {
+        long l = System.currentTimeMillis();
+
+        return String.valueOf(l);
+    }
+
+    protected String merge(String... args) {
+        if (args == null || args.length == 0) {
+            return null;
+        }
+
+        StringBuilder sb = new StringBuilder();
+        for (String arg : args) {
+            sb.append(arg);
+            sb.append(IStorage.SEPARATOR);
+        }
+
+        return sb.substring(0, sb.lastIndexOf(IStorage.SEPARATOR));
+    }
+
+    protected List<String> split(String str) {
+        String[] split = str.split(IStorage.SEPARATOR);
+        return new ArrayList<>(Arrays.asList(split));
+    }
+
+    protected long getTimestamp(Object target) {
+        if (target instanceof WindowInstance) {
+            return ((WindowInstance) target).getLastMaxUpdateTime();
+        } else if (target instanceof WindowBaseValue) {
+            return ((WindowBaseValue) target).getUpdateVersion();
+        } else if (target instanceof String) {
+            String time = ((String) target).split(IStorage.SEPARATOR)[0];
+            return Long.parseLong(time);
+        } else {
+            throw new RuntimeException();
+        }
+    }
+
+
+    static class NullFuture implements Future<Object> {
+        @Override
+        public boolean cancel(boolean mayInterruptIfRunning) {
+            return false;
+        }
+
+        @Override
+        public boolean isCancelled() {
+            return false;
+        }
+
+        @Override
+        public boolean isDone() {
+            return true;
+        }
+
+        @Override
+        public Object get() throws InterruptedException, ExecutionException {
+            return null;
+        }
+
+        @Override
+        public Object get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
+            return null;
+        }
+    }
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/AbstractWindowStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/AbstractWindowStorage.java
deleted file mode 100644
index b29a3b8..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/AbstractWindowStorage.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.storage;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.db.driver.batchloader.BatchRowLoader;
-import org.apache.rocketmq.streams.db.driver.batchloader.IRowOperator;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
-import org.apache.rocketmq.streams.window.state.WindowBaseValue;
-
-public abstract class AbstractWindowStorage<T extends WindowBaseValue> implements IWindowStorage<T> {
-    protected boolean isLocalStorageOnly = false;
-    protected transient ExecutorService dataLoaderExecutor = new ThreadPoolExecutor(10, 10,
-        0L, TimeUnit.MILLISECONDS,
-        new LinkedBlockingQueue<Runnable>());
-    ;
-
-
-    @Override
-    public void multiPut(Map<String, T> map, String windowInstanceId, String queueId) {
-        throw new RuntimeException("can not support this method");
-    }
-
-    @Override
-    public Map<String, T> multiGet(Class<T> clazz, List<String> keys, String windowInstanceId, String queueId) {
-        throw new RuntimeException("can not support this method");
-    }
-
-    @Override
-    public void loadSplitData2Local(String queueId, String windowInstanceId, Class<T> clazz, IRowOperator processor) {
-        if (isLocalStorageOnly) {
-            return;
-        }
-        String windowInstancePartitionId = StringUtil.createMD5Str(windowInstanceId);
-        dataLoaderExecutor.execute(new Runnable() {
-            @Override
-            public void run() {
-                long start = System.currentTimeMillis();
-                BatchRowLoader batchRowLoader = new BatchRowLoader("partition_num",
-                    "select * from " + ORMUtil.getTableName(clazz) + "  where window_instance_partition_id ='"
-                        + windowInstancePartitionId + "'", processor);
-                batchRowLoader.startLoadData();
-                ShufflePartitionManager.getInstance().setWindowInstanceFinished(windowInstanceId);
-                System.out.println(System.currentTimeMillis() - start);
-                System.out.println("");
-            }
-        });
-
-    }
-
-    @Override
-    public void put(String key, T value) {
-        Map<String, T> map = new HashMap<>();
-        map.put(key, value);
-        multiPut(map);
-    }
-
-    @Override
-    public T get(Class<T> clazz, String key) {
-        Map<String, T> result = multiGet(clazz, key);
-        if (result == null) {
-            return null;
-        }
-        return result.values().iterator().next();
-    }
-
-    @Override
-    public Map<String, T> multiGet(Class<T> clazz, String... keys) {
-        List<String> keyList = new ArrayList<>();
-        for (String key : keys) {
-            keyList.add(key);
-        }
-        return multiGet(clazz, keyList);
-    }
-
-    public boolean isLocalStorageOnly() {
-        return isLocalStorageOnly;
-    }
-
-    public void setLocalStorageOnly(boolean localStorageOnly) {
-        isLocalStorageOnly = localStorageOnly;
-    }
-
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IKeyGenerator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/DataType.java
similarity index 72%
rename from rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IKeyGenerator.java
rename to rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/DataType.java
index cb01eff..caa1d7d 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IKeyGenerator.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/DataType.java
@@ -1,3 +1,4 @@
+package org.apache.rocketmq.streams.window.storage;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -14,16 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.streams.window.storage;
 
-/**
- * 为了适配不同存储,可以通过key generator的实现来做非kv系统的映射
- */
-public interface IKeyGenerator {
+public enum DataType {
+    WINDOW_INSTANCE("windowInstance"),
+    WINDOW_BASE_VALUE("windowBaseValue"),
+    MAX_OFFSET("maxOffset"),
+    MAX_PARTITION_NUM("maxPartitionNum");
 
-    /**
-     * @param
-     * @return
-     */
-    String createKey();
+    private final String value;
+
+    DataType(String value) {
+        this.value = value;
+    }
+
+    public String getValue() {
+        return value;
+    }
 }
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ICommonStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ICommonStorage.java
deleted file mode 100644
index 44a83fe..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ICommonStorage.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.storage;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-/**
- * 对存储的统一抽象,最底层的抽象 T必须是可识别的对象
- */
-public interface ICommonStorage<T> {
-
-    /**
-     * 支持单个key value的存储
-     *
-     * @param key
-     * @param value
-     */
-    void put(String key, T value);
-
-    //多组key value批量存储
-    void multiPut(Map<String, T> map);
-
-    /**
-     * put <key,list> into the storage
-     *
-     * @param elements
-     */
-    void multiPutList(Map<String, List<T>> elements);
-
-    //获取多个key的值
-    Map<String, T> multiGet(Class<T> clazz, List<String> keys);
-
-    /**
-     * get list from storage according by key
-     *
-     * @param clazz
-     * @param keys
-     * @return
-     */
-    Map<String, List<T>> multiGetList(Class<T> clazz, List<String> keys);
-
-    //获取单个key的值
-    T get(Class<T> clazz, String key);
-
-    //获取多个key的值
-    Map<String, T> multiGet(Class<T> clazz, String... keys);
-
-    void removeKeys(Collection<String> keys);
-
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IRemoteStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IRemoteStorage.java
deleted file mode 100644
index f7e9362..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IRemoteStorage.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.streams.window.storage;
-
-import java.util.List;
-import java.util.Map;
-
-public interface IRemoteStorage<T> extends ICommonStorage<T> {
-
-    //多组key value批量存储
-    String multiPutSQL(Map<String, T> values);
-
-    /**
-     * generate sql sentence for session window
-     * @param infoMap
-     * @return
-     */
-    String multiPutListSQL(Map<String, List<T>> infoMap);
-
-    String deleteSQL(String windowInstanceId, String queueId, Class<T> clazz);
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IShufflePartitionManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IShufflePartitionManager.java
deleted file mode 100644
index eb68753..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IShufflePartitionManager.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.storage;
-
-import java.util.Collection;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-
-public interface IShufflePartitionManager {
-
-    /**
-     * 这个分片是否可用本地存储
-     *
-     * @param shuffleId
-     * @return
-     */
-    boolean isLocalStorage(String shuffleId, String windowInstanceId);
-
-    void setLocalStorageInvalid(ISplit channelQueue);
-
-    void setLocalStorageInvalid(ISplit channelQueue, String windowInstanceId);
-
-    /**
-     * setLocalStorageInvalid 如果 shuffle id不存在,且
-     *
-     * @param shuffleId
-     */
-    boolean setLocalStorageValdateIfNotExist(String shuffleId, String windowInstanceId);
-
-    /**
-     * 当窗口实例触发后,通过这个方法,回收资源
-     *
-     * @param windowInstanceId
-     * @param queueIds
-     */
-    void clearWindowInstanceStorageStatus(String windowInstanceId, Collection<String> queueIds);
-
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IStorage.java
index 074770d..ea435d5 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IStorage.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IStorage.java
@@ -1,3 +1,4 @@
+package org.apache.rocketmq.streams.window.storage;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -13,68 +14,61 @@
  * 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.streams.window.storage;
+ */
 
-import java.util.Collection;
-import java.util.Iterator;
+import org.apache.rocketmq.streams.window.model.WindowInstance;
+import org.apache.rocketmq.streams.window.state.WindowBaseValue;
+
 import java.util.List;
-import java.util.Map;
-import org.apache.rocketmq.streams.db.driver.batchloader.IRowOperator;
+import java.util.concurrent.Future;
 
-public interface IStorage<T> {
+public interface IStorage {
+    String SEPARATOR = "@";
 
+    Future<?> load(String shuffleId);
+
+
+    void putWindowInstance(String shuffleId, String windowNamespace, String windowConfigureName, WindowInstance windowInstance);
+
+    <T> RocksdbIterator<T> getWindowInstance(String shuffleId, String windowNamespace, String windowConfigureName);
 
     /**
-     * 支持单个key value的存储
-     * @param key
-     * @param value
+     * WindowInstance的唯一索引字段
+     *
+     * @param windowInstanceId
      */
-    void put(String key, T value);
+    void deleteWindowInstance(String shuffleId, String windowNamespace, String windowConfigureName, String windowInstanceId);
 
-    //多组key value批量存储
-    void mutilPut(Map<String, T> keyValue);
+    void putWindowBaseValue(String shuffleId, String windowInstanceId,
+                            WindowType windowType, WindowJoinType joinType,
+                            List<WindowBaseValue> windowBaseValue);
 
-    //获取单个key的值
-    T get(Class<T> clazz, IKeyGenerator keyGenerator, String key);
+    void putWindowBaseValueIterator(String shuffleId, String windowInstanceId,
+                                    WindowType windowType, WindowJoinType joinType,
+                                    RocksdbIterator<? extends WindowBaseValue> windowBaseValueIterator);
 
-    //获取多个key的值
-    Map<String,T> mutilGet(Class<T> clazz, IKeyGenerator keyGenerator, String... keyValues);
-    //获取多个key的值
-    Map<String,T> mutilGet(Class<T> clazz, IKeyGenerator keyGenerator, List<String> keys);
-
-    /**
-     * remove keys
-     * @param keys
-     */
-    void removeKeys(IKeyGenerator keyGenerator, Collection<String> keys);
-
-    /**
-     * remove keys by prefix
-     * @param keyPrefix
-     */
-    void removeKeyPrefix(IKeyGenerator keyGenerator, String keyPrefix);
-
-    /*
-        create Iterator by key prefix
-     */
-    Iterator<T> iterateByPrefix(IKeyGenerator keyGenerator, String keyPrefix, Class<T> clazz);
+    <T> RocksdbIterator<T> getWindowBaseValue(String shuffleId, String windowInstanceId, WindowType windowType, WindowJoinType joinType);
 
 
-    T putIfAbsent(T t, Class<T> clazz);
+    //用windowInstanceId删除所有WindowBaseValue【包括WindowValue、JoinState】
+    void deleteWindowBaseValue(String shuffleId, String windowInstanceId, WindowType windowType, WindowJoinType joinType);
+
+    void deleteWindowBaseValue(String shuffleId, String windowInstanceId, WindowType windowType, WindowJoinType joinType, String msgKey);
+
+    String getMaxOffset(String shuffleId, String windowConfigureName, String oriQueueId);
+
+    void putMaxOffset(String shuffleId, String windowConfigureName, String oriQueueId, String offset);
+
+    void deleteMaxOffset(String shuffleId, String windowConfigureName, String oriQueueId);
 
 
-    int count(IKeyGenerator keyGenerator, String key);
+    void putMaxPartitionNum(String shuffleId, String windowInstanceId, long maxPartitionNum);
 
-    int incrementAndGet(IKeyGenerator keyGenerator, String key);
+    Long getMaxPartitionNum(String shuffleId, String windowInstanceId);
 
+    void deleteMaxPartitionNum(String shuffleId, String windowInstanceId);
 
-    Iterator<T> queryByPrefixBetweenOrderByValue(IKeyGenerator keyGenerator, String keyPrefix, Object startIndexValue,
-        Object endIndexValue, Class<T> clazz);
+    int flush(List<String> queueId);
 
-
-
-    void loadByPrefixBetweenOrderByValue(IKeyGenerator keyGenerator, String keyPrefix, Object startIndexValue,
-        Object endIndexValue,
-        IRowOperator rowOperator, Class<T> clazz);
-
+    void clearCache(String queueId);
 }
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IWindowStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IWindowStorage.java
deleted file mode 100644
index ea0ec4b..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IWindowStorage.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.storage;
-
-import java.util.List;
-import java.util.Map;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-import org.apache.rocketmq.streams.db.driver.batchloader.IRowOperator;
-import org.apache.rocketmq.streams.window.model.WindowInstance;
-import org.apache.rocketmq.streams.window.state.WindowBaseValue;
-import org.apache.rocketmq.streams.window.storage.WindowStorage.WindowBaseValueIterator;
-
-public interface IWindowStorage<T extends WindowBaseValue> extends ICommonStorage<T> {
-
-    //多组key value批量存储
-    void multiPut(Map<String, T> map, String windowInstanceId, String queueId);
-
-    //获取多个key的值
-    Map<String, T> multiGet(Class<T> clazz, List<String> keys, String windowInstanceId, String queueId);
-
-    /***
-     * 把queueId 前缀的数据全部失效掉
-     * @param channelQueue 必须6位,64001  1280001 128分片总数,001第一个分片
-     */
-
-    void clearCache(ISplit channelQueue, Class<T> clazz);
-
-    /**
-     * 删除一个窗口实例的数据,包括远程和本地存储
-     */
-    void delete(String windowInstanceId, String queueId, Class<T> clazz);
-
-    /**
-     * 加载一个窗口实例的数据到本地存储
-     */
-    WindowBaseValueIterator<T> loadWindowInstanceSplitData(String localStorePrefix, String queueId,
-        String windowInstanceId, String keyPrefix,
-        Class<T> clazz);
-
-    /**
-     * 这个窗口实例,最大的分片序列号,主要是通过db获取
-     *
-     * @return
-     */
-    Long getMaxSplitNum(WindowInstance windowInstance, Class<T> clazz);
-
-    /**
-     * 批量加载数据,放入本地缓存
-     *
-     * @param splitNumer
-     * @param rowOperator
-     */
-    void loadSplitData2Local(String splitNumer, String windowInstanceId, Class<T> clazz, IRowOperator rowOperator);
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IteratorWrap.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IteratorWrap.java
new file mode 100644
index 0000000..ee43af0
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IteratorWrap.java
@@ -0,0 +1,53 @@
+package org.apache.rocketmq.streams.window.storage;
+/*
+ * 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.
+ */
+
+public class IteratorWrap<T> {
+    private String key;
+    private T data;
+    private byte[] raw;
+
+    public IteratorWrap(String key, T data, byte[] raw) {
+        this.key = key;
+        this.data = data;
+        this.raw = raw;
+    }
+
+    public String getKey() {
+        return key;
+    }
+
+    public void setKey(String key) {
+        this.key = key;
+    }
+
+    public T getData() {
+        return data;
+    }
+
+    public void setData(T data) {
+        this.data = data;
+    }
+
+    public byte[] getRaw() {
+        return raw;
+    }
+
+    public void setRaw(byte[] raw) {
+        this.raw = raw;
+    }
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/RocksdbIterator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/RocksdbIterator.java
new file mode 100644
index 0000000..9b84c19
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/RocksdbIterator.java
@@ -0,0 +1,63 @@
+package org.apache.rocketmq.streams.window.storage;
+/*
+ * 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.
+ */
+
+import org.apache.rocketmq.streams.common.utils.SerializeUtil;
+import org.rocksdb.ReadOptions;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksIterator;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+
+public class RocksdbIterator<T> implements Iterator<IteratorWrap<T>> {
+    private String keyPrefix;
+    private ReadOptions readOptions = new ReadOptions();
+    private RocksIterator rocksIterator;
+
+
+
+    public RocksdbIterator(String keyPrefix, RocksDB rocksDB) {
+        this.keyPrefix = keyPrefix;
+        this.rocksIterator = rocksDB.newIterator(readOptions);
+        this.rocksIterator.seek(keyPrefix.getBytes(StandardCharsets.UTF_8));
+    }
+
+    @Override
+    public boolean hasNext() {
+        boolean hasNext = rocksIterator.isValid();
+        String key = new String(rocksIterator.key());
+
+        if (!key.startsWith(keyPrefix)) {
+            hasNext = false;
+        }
+        return hasNext;
+    }
+
+    @Override
+    public IteratorWrap<T> next() {
+        String key = new String(rocksIterator.key());
+
+        T data = SerializeUtil.deserialize(rocksIterator.value());
+        IteratorWrap<T> result = new IteratorWrap<>(key, data, rocksIterator.value());
+
+        rocksIterator.next();
+        return result;
+    }
+}
+
+
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ShufflePartitionManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ShufflePartitionManager.java
deleted file mode 100644
index 93a0b2a..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/ShufflePartitionManager.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.storage;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-public class ShufflePartitionManager {
-    private static ShufflePartitionManager instance = new ShufflePartitionManager();
-    protected Map<String, Boolean> splitId2AllWindowInstanceFinishInit = new HashMap<>();//split是否有效,这个分片下所有的window instacne都完成了初始化
-    protected Map<String, Boolean> windowInstanceId2FinishInit = new HashMap<>();//window instance 是否完成初始化
-    private ExecutorService executorService;
-
-    private ShufflePartitionManager() {
-        executorService = new ThreadPoolExecutor(10, 10,
-            0L, TimeUnit.MILLISECONDS,
-            new LinkedBlockingQueue<Runnable>());
-
-    }
-
-    public static ShufflePartitionManager getInstance() {
-        return instance;
-    }
-
-    /**
-     * if window instance finished init, return true else return false
-     *
-     * @param splitId
-     * @param windowInstanceId
-     * @return
-     */
-    public boolean isWindowInstanceFinishInit(String splitId, String windowInstanceId) {
-        Boolean allSpliltFinish = splitId2AllWindowInstanceFinishInit.get(splitId);
-        if (allSpliltFinish != null && allSpliltFinish) {
-            return true;
-        }
-        Boolean windowInstanceInitFinished = windowInstanceId2FinishInit.get(windowInstanceId);
-        if (windowInstanceInitFinished != null && windowInstanceInitFinished) {
-            return true;
-        }
-        return false;
-    }
-
-    public synchronized void setSplitFinished(String splitId) {
-        splitId2AllWindowInstanceFinishInit.put(splitId, true);
-    }
-
-    public synchronized void setSplitInValidate(String splitId) {
-        splitId2AllWindowInstanceFinishInit.put(splitId, false);
-    }
-
-    public synchronized void setWindowInstanceFinished(String windowInstanceId) {
-        windowInstanceId2FinishInit.put(windowInstanceId, true);
-    }
-
-    public synchronized void clearWindowInstance(String windowInstanceId) {
-        windowInstanceId2FinishInit.remove(windowInstanceId);
-    }
-
-    public synchronized void clearSplit(String queueId) {
-        splitId2AllWindowInstanceFinishInit.remove(queueId);
-        Map<String, Boolean> map=new HashMap<>(this.windowInstanceId2FinishInit);
-        for (String windowInstanceId : map.keySet()) {
-            if (windowInstanceId.startsWith(queueId)) {
-                this.windowInstanceId2FinishInit.remove(windowInstanceId);
-            }
-        }
-    }
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/StorageManager.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/StorageManager.java
deleted file mode 100644
index a78c848..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/StorageManager.java
+++ /dev/null
@@ -1,307 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.storage;
-
-/**
- * 存储管理,根据分片本地存储是否有效,来选择对应的存储完成计算
- */
-public class StorageManager {
-    //private ExecutorService executorService= Executors.newFixedThreadPool(10);
-    //
-    //private ExecutorService dbService = Executors.newSingleThreadExecutor();
-    //
-    //private static StorageManager storageManager=new StorageManager();
-    //private StorageManager(){}
-    //protected static IStorage rocksDBStorage=new RocksdbStorage();//本地存储
-    //protected static DBStorage dbStorage=new DBStorage();//jdbc 存储
-    ////0/null:false;1:true;2加载中
-    ////
-    //protected transient boolean isLocalOnly=false;//只支持本地存储
-    //protected ConcurrentHashMap<String,Integer> shuffleIdAndWindowInstance2IsLocal=new ConcurrentHashMap<>();//某个分片是否本地存储有效
-    //public static StorageManager getStorageManager(){
-    //   return storageManager;
-    //}
-    //
-    //public static IShufflePartitionManager getShufflePartitionManager(){
-    //    return storageManager;
-    //}
-    //
-    //public static IStorage getLocalStorage(){
-    //    return rocksDBStorage;
-    //}
-    //
-    //public static IStorage getRemoteStorage(){
-    //    return dbStorage;
-    //}
-    //
-    //@Override
-    //public void put(Map<String, WindowBaseValue> values, boolean onlyLocal) {
-    //    if(onlyLocal){
-    //        rocksDBStorage.put(values, true);
-    //        return;
-    //    }
-    //    Map<String,WindowBaseValue> notLocalWindowBaseValues=new HashMap<>();
-    //    Map<String,WindowBaseValue> localWindowBaseValues=new HashMap<>();
-    //    Iterator<Entry<String, WindowBaseValue>> it = values.entrySet().iterator();
-    //    while (it.hasNext()){
-    //        Entry<String, WindowBaseValue>entry=it.next();
-    //        boolean isLocal=isLocalStorage(entry.getValue().getPartition(),entry.getValue().getWindowInstanceId());
-    //        if(isLocal){
-    //            localWindowBaseValues.put(entry.getKey(),entry.getValue());
-    //        }else {
-    //            notLocalWindowBaseValues.put(entry.getKey(),entry.getValue());
-    //        }
-    //    }
-    //    rocksDBStorage.put(values, false);
-    //    if(isLocalOnly){
-    //        return;
-    //    }
-    //    /**
-    //     *
-    //     */
-    //    if (!CollectionUtil.isEmpty(localWindowBaseValues)) {
-    //        //如果本地可用,可以用异步的方式写,提高写性能
-    //        dbService.execute(new Runnable() {
-    //            @Override
-    //            public void run() {
-    //                dbStorage.put(getMd5Value(localWindowBaseValues), false);
-    //            }
-    //        });
-    //    } else if (!CollectionUtil.isEmpty(notLocalWindowBaseValues)) {
-    //        //如果本地不可用,必须同步写
-    //        dbStorage.put(getMd5Value(notLocalWindowBaseValues), false);
-    //    }
-    //}
-    //
-    ///**
-    // * DB存储时用MD5,TODO 考虑使用aop
-    // */
-    //private Map<String, WindowBaseValue> getMd5Value(Map<String, WindowBaseValue> originMap) {
-    //    Map<String, WindowBaseValue> valueMap = new HashMap<>(originMap.size());
-    //    Iterator<Entry<String, WindowBaseValue>> iterator = originMap.entrySet().iterator();
-    //    while (iterator.hasNext()) {
-    //        Entry<String, WindowBaseValue> entry = iterator.next();
-    //        WindowBaseValue value = entry.getValue();
-    //        if (value instanceof WindowValue) {
-    //            WindowValue md5Value = (WindowValue)entry.getValue();
-    //            valueMap.put(entry.getKey(), md5Value.toMd5Value());
-    //        } else {
-    //            //TODO join的MD5计算逻辑
-    //            valueMap.put(entry.getKey(), entry.getValue());
-    //        }
-    //    }
-    //    return valueMap;
-    //}
-    //
-    //@Override
-    //public Map<String, WindowBaseValue> get(Collection<String> keys, Class<? extends WindowBaseValue> clazz) {
-    //    Map<String, WindowBaseValue> result=new HashMap<>();
-    //    if(isLocalOnly){
-    //        result.putAll(rocksDBStorage.get(keys,clazz));
-    //        return result;
-    //    }
-    //
-    //    List<String> notLocalKeys=new ArrayList<>();
-    //    List<String> localKeys=new ArrayList<>();
-    //    for(String key:keys){
-    //        String[] values=MapKeyUtil.spliteKey(key);
-    //        String shuffleId=values[0];
-    //        boolean isLocal = isLocalStorage(shuffleId, WindowInstance.createWindowInstanceId(key));
-    //        if(isLocal){
-    //            localKeys.add(key);
-    //        }else {
-    //            notLocalKeys.add(key);
-    //        }
-    //    }
-    //
-    //    result.putAll(rocksDBStorage.get(localKeys,clazz));
-    //    result.putAll(dbStorage.get(notLocalKeys,clazz));
-    //    return result;
-    //}
-    //
-    //@Override
-    //public void delete(String windowNameSpace, String windowName, String startTime, String endOrFireTime,
-    //    Class<? extends WindowBaseValue> clazz) {
-    //    executorService.execute(new Runnable() {
-    //
-    //        @Override
-    //        public void run() {
-    //            rocksDBStorage.delete(windowNameSpace, windowName, startTime, endOrFireTime, clazz);
-    //            if(!isLocalOnly){
-    //                dbStorage.delete(windowNameSpace, windowName, startTime, endOrFireTime, clazz);
-    //            }
-    //
-    //        }
-    //    });
-    //
-    //}
-    //
-    //@Override
-    //public void clearCache(ISplit channelQueue) {
-    //    rocksDBStorage.clearCache(channelQueue);
-    //}
-    //
-    //@Override
-    //public Iterator<WindowBaseValue> loadWindowInstanceSplitData(String queueId, String windowNameSpace,
-    //    String windowName, String startTime, String endOrFireTime, String key, Class<? extends WindowBaseValue> clazz) {
-    //    boolean isLocal = isLocalStorage(queueId,
-    //        WindowInstance.getWindowInstanceId(windowNameSpace, windowName, startTime, endOrFireTime));
-    //    if(isLocal){
-    //        return rocksDBStorage.loadWindowInstanceSplitData(queueId,windowNameSpace,windowName,startTime,endOrFireTime,key, clazz);
-    //    }else {
-    //        return dbStorage.loadWindowInstanceSplitData(queueId,windowNameSpace,windowName,startTime,endOrFireTime,key, clazz);
-    //    }
-    //}
-    //
-    //@Override
-    //public long getMaxShuffleId(String queueId, String windowNameSpace, String windowName, String startTime,
-    //    String endOrFireTime, Class<? extends WindowBaseValue> clazz) {
-    //    if(isLocalOnly){
-    //        return 0;
-    //    }
-    //    return dbStorage.getMaxShuffleId(queueId,windowNameSpace,windowName,startTime,endOrFireTime,clazz);
-    //}
-    //
-    //@Override
-    //public void loadSplitData2Local(String splitNumer, String windowNameSpace, String windowName,
-    //    String startTime, String endOrFireTime, Class<? extends WindowBaseValue> clazz, IRowOperator rowOperator) {
-    //    if(isLocalOnly){
-    //        return;
-    //    }
-    //    String windowInstanceId = WindowInstance.getWindowInstanceId(windowNameSpace, windowName, startTime, endOrFireTime);
-    //    Integer value=this.shuffleIdAndWindowInstance2IsLocal.get(MapKeyUtil.createKey(splitNumer,windowInstanceId));
-    //    if(value!=null&&value!=0){
-    //        return;
-    //    }
-    //    synchronized (this){
-    //        value=this.shuffleIdAndWindowInstance2IsLocal.get(MapKeyUtil.createKey(splitNumer,windowInstanceId));
-    //        if(value!=null&&value!=0){
-    //            return;
-    //        }
-    //        shuffleIdAndWindowInstance2IsLocal.put(MapKeyUtil.createKey(splitNumer,windowInstanceId),2);
-    //        executorService.execute(new Runnable() {
-    //            @Override
-    //            public void run() {
-    //                if (rowOperator == null) {
-    //                    dbStorage.loadSplitData2Local(splitNumer, windowNameSpace, windowName, startTime, endOrFireTime,
-    //                        clazz,
-    //                        new IRowOperator() {
-    //                            @Override
-    //                            public void doProcess(Map<String, Object> row) {
-    //                                WindowBaseValue theValue = ORMUtil.convert(row, clazz);
-    //                                List<String> keys = new ArrayList<>();
-    //                                keys.add(theValue.getMsgKey());
-    //                                WindowBaseValue windowBaseValue = (WindowBaseValue)rocksDBStorage.get(keys, clazz);
-    //                                if (windowBaseValue == null) {
-    //                                    Map<String, WindowBaseValue> map = new HashMap<>();
-    //                                    map.put(theValue.getMsgKey(), theValue);
-    //                                    rocksDBStorage.put(map, true);
-    //                                    return;
-    //                                }
-    //                                if (theValue.getUpdateVersion() > windowBaseValue.getUpdateVersion()) {
-    //                                    Map<String, WindowBaseValue> map = new HashMap<>();
-    //                                    map.put(theValue.getMsgKey(), theValue);
-    //                                    rocksDBStorage.put(map, true);
-    //                                }
-    //                            }
-    //                        });
-    //                } else {
-    //                    dbStorage.loadSplitData2Local(splitNumer, windowNameSpace, windowName, startTime, endOrFireTime,
-    //                        clazz, rowOperator);
-    //                }
-    //                shuffleIdAndWindowInstance2IsLocal.put(MapKeyUtil.createKey(splitNumer, windowInstanceId), 1);
-    //            }
-    //        });
-    //    }
-    //
-    //
-    //}
-    //
-    //
-    //@Override
-    //public boolean isLocalStorage(String shuffleId,String windowInstanceId) {
-    //    Integer value=this.shuffleIdAndWindowInstance2IsLocal.get(MapKeyUtil.createKey(shuffleId,windowInstanceId));
-    //    return isLocalStorage(value)||isLocalOnly;
-    //}
-    //@Override
-    //public void setLocalStorageInvalid(ISplit channelQueue,String windowInstanceId) {
-    //    this.shuffleIdAndWindowInstance2IsLocal.remove(MapKeyUtil.createKey(channelQueue.getQueueId(),windowInstanceId),false);
-    //}
-    //@Override
-    //public void setLocalStorageInvalid(ISplit channelQueue) {
-    //    Iterator<Entry<String, Integer>> it = this.shuffleIdAndWindowInstance2IsLocal.entrySet().iterator();
-    //    List<String> keys=new ArrayList<>();
-    //    while (it.hasNext()){
-    //        Entry<String, Integer> entry=it.next();
-    //        String key=entry.getKey();
-    //        if(key.startsWith(channelQueue.getQueueId())){
-    //            this.shuffleIdAndWindowInstance2IsLocal.put(key,0);
-    //            keys.add(entry.getKey());
-    //        }
-    //
-    //    }
-    //    executorService.execute(new Runnable() {
-    //        @Override
-    //        public void run() {
-    //            clearCache(channelQueue);
-    //            for(String key:keys){
-    //                shuffleIdAndWindowInstance2IsLocal.remove(key);
-    //            }
-    //        }
-    //    });
-    //}
-    //
-    //@Override
-    //public boolean setLocalStorageValdateIfNotExist(String shuffleId,String windowInstanceId) {
-    //    Integer value = this.shuffleIdAndWindowInstance2IsLocal.get(
-    //        MapKeyUtil.createKey(shuffleId, windowInstanceId));
-    //    if (value != null) {
-    //        return isLocalStorage(value);
-    //    }
-    //    this.shuffleIdAndWindowInstance2IsLocal.put( MapKeyUtil.createKey(shuffleId, windowInstanceId), 1);
-    //    return true;
-    //}
-    //
-    //
-    //@Override
-    //public void clearWindowInstanceStorageStatus(String windowInstanceId,Collection<String> queueIds){
-    //    if(queueIds!=null){
-    //        for(String queueId:queueIds){
-    //            this.shuffleIdAndWindowInstance2IsLocal.remove(MapKeyUtil.createKey(queueId,windowInstanceId));
-    //        }
-    //    }
-    //}
-    //
-    //
-    //public boolean isLocalStorage(Integer value){
-    //    if(value!=null&&value==1){
-    //        return true;
-    //    }
-    //    if(isLocalOnly){
-    //        return true;
-    //    }
-    //    return false;
-    //}
-    //
-    //public boolean isLocalOnly() {
-    //    return isLocalOnly;
-    //}
-    //
-    //public void setLocalOnly(boolean localOnly) {
-    //    isLocalOnly = localOnly;
-    //}
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IKeyGenerator.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowInstanceIterator.java
similarity index 72%
copy from rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IKeyGenerator.java
copy to rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowInstanceIterator.java
index cb01eff..96e68b0 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/IKeyGenerator.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowInstanceIterator.java
@@ -1,3 +1,4 @@
+package org.apache.rocketmq.streams.window.storage;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -14,16 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.streams.window.storage;
 
-/**
- * 为了适配不同存储,可以通过key generator的实现来做非kv系统的映射
- */
-public interface IKeyGenerator {
+import org.apache.rocketmq.streams.window.model.WindowInstance;
 
-    /**
-     * @param
-     * @return
-     */
-    String createKey();
+import java.util.Iterator;
+
+public class WindowInstanceIterator implements Iterator<WindowInstance> {
+
+
+
+
+
+
+    @Override
+    public boolean hasNext() {
+        return false;
+    }
+
+    @Override
+    public WindowInstance next() {
+        return null;
+    }
 }
diff --git a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBType.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowJoinType.java
similarity index 85%
rename from rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBType.java
rename to rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowJoinType.java
index 83e7978..01278c3 100644
--- a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBType.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowJoinType.java
@@ -1,3 +1,4 @@
+package org.apache.rocketmq.streams.window.storage;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -15,9 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.streams.dbinit.mysql.delegate;
-
-public class DBType {
-
-    public static final String DB_MYSQL = "MYSQL";
+public enum WindowJoinType {
+    left,
+    right
 }
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowStorage.java
deleted file mode 100644
index 82fef5c..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowStorage.java
+++ /dev/null
@@ -1,255 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.storage;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-import org.apache.rocketmq.streams.common.utils.CollectionUtil;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.common.utils.SQLUtil;
-import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
-import org.apache.rocketmq.streams.window.model.WindowInstance;
-import org.apache.rocketmq.streams.window.sqlcache.SQLCache;
-import org.apache.rocketmq.streams.window.sqlcache.impl.SQLElement;
-import org.apache.rocketmq.streams.window.state.WindowBaseValue;
-import org.apache.rocketmq.streams.window.state.impl.WindowValue;
-import org.apache.rocketmq.streams.window.storage.db.DBStorage;
-import org.apache.rocketmq.streams.window.storage.rocksdb.RocksdbStorage;
-
-public class WindowStorage<T extends WindowBaseValue> extends AbstractWindowStorage<T> {
-    protected transient ShufflePartitionManager shufflePartitionManager = ShufflePartitionManager.getInstance();
-    protected IWindowStorage localStorage;
-    protected IWindowStorage remoteStorage;
-
-    public WindowStorage(boolean isLoaclStorageOnly) {
-        this();
-        this.isLocalStorageOnly = isLoaclStorageOnly;
-    }
-
-    public WindowStorage() {
-        localStorage = new RocksdbStorage();
-        remoteStorage = new DBStorage();
-    }
-
-    @Override
-    public WindowBaseValueIterator<T> loadWindowInstanceSplitData(String localStorePrefix, String queueId,
-        String windowInstanceId, String keyPrefix, Class<T> clazz) {
-        if (isLocalStorageOnly) {
-            return localStorage.loadWindowInstanceSplitData(localStorePrefix, queueId, windowInstanceId, keyPrefix, clazz);
-        }
-        if (shufflePartitionManager.isWindowInstanceFinishInit(queueId, windowInstanceId)) {
-            return localStorage.loadWindowInstanceSplitData(localStorePrefix, queueId, windowInstanceId, keyPrefix, clazz);
-        }
-        return remoteStorage.loadWindowInstanceSplitData(localStorePrefix, queueId, windowInstanceId, keyPrefix
-            , clazz);
-    }
-
-    @Override
-    public void multiPut(Map<String, T> values, String windowInstanceId, String queueId) {
-        multiPut(values, windowInstanceId, queueId, null);
-    }
-
-    public void multiPut(Map<String, T> values, String windowInstanceId, String queueId, SQLCache sqlCache) {
-        localStorage.multiPut(values);
-        if (isLocalStorageOnly) {
-            return;
-        }
-        if (shufflePartitionManager.isWindowInstanceFinishInit(queueId, windowInstanceId)) {
-            //可以考虑异步
-            if (sqlCache != null) {
-                sqlCache.addCache(new SQLElement(queueId, windowInstanceId, ((IRemoteStorage) this.remoteStorage).multiPutSQL(values)));
-            } else {
-                remoteStorage.multiPut(values);
-            }
-
-            return;
-        }
-        remoteStorage.multiPut(values);
-    }
-
-    /**
-     * used in session window only
-     *
-     * @param values
-     * @param windowInstanceId
-     * @param queueId
-     * @param sqlCache
-     */
-    public void multiPutList(Map<String, List<T>> values, String windowInstanceId, String queueId, SQLCache sqlCache) {
-        localStorage.multiPutList(values);
-        if (!isLocalStorageOnly) {
-            //delete all values first
-            deleteRemoteValue(values.keySet());
-            //
-            if (shufflePartitionManager.isWindowInstanceFinishInit(queueId, windowInstanceId)) {
-                if (sqlCache != null) {
-                    sqlCache.addCache(new SQLElement(queueId, windowInstanceId, ((IRemoteStorage) this.remoteStorage).multiPutListSQL(values)));
-                } else {
-                    remoteStorage.multiPutList(values);
-                }
-                return;
-            }
-            remoteStorage.multiPutList(values);
-        }
-    }
-
-    private void deleteRemoteValue(Set<String> storeKeyList) {
-        if (CollectionUtil.isEmpty(storeKeyList)) {
-            return;
-        }
-        String sql = "delete from " + ORMUtil.getTableName(WindowValue.class) + " where " + SQLUtil.createLikeSql(storeKeyList.stream().map(key -> Pair.of("msg_key", StringUtil.createMD5Str(key))).collect(Collectors.toList()));
-        ORMUtil.executeSQL(sql, new HashMap<>(4));
-    }
-
-    @Override
-    public Long getMaxSplitNum(WindowInstance windowInstance, Class<T> clazz) {
-        if (isLocalStorageOnly) {
-            return null;
-        }
-        return remoteStorage.getMaxSplitNum(windowInstance, clazz);
-    }
-
-    @Override
-    public Map<String, T> multiGet(Class<T> clazz, List<String> keys, String windowInstanceId, String queueId) {
-        if (isLocalStorageOnly || shufflePartitionManager.isWindowInstanceFinishInit(queueId, windowInstanceId)) {
-            return localStorage.multiGet(clazz, keys);
-        }
-        return remoteStorage.multiGet(clazz, keys);
-    }
-
-    @Override
-    public void multiPutList(Map<String, List<T>> elements) {
-        if (!isLocalStorageOnly) {
-            remoteStorage.multiPutList(elements);
-        }
-        localStorage.multiPutList(elements);
-    }
-
-    @Override
-    public Map<String, List<T>> multiGetList(Class<T> clazz, List<String> keys) {
-        if (isLocalStorageOnly) {
-            return localStorage.multiGetList(clazz, keys);
-        }
-        Map<String, List<T>> resultMap = new HashMap<>(keys.size());
-        Pair<List<String>, List<String>> pair = getStorageKeys(keys);
-        resultMap.putAll(localStorage.multiGetList(clazz, pair.getLeft()));
-        resultMap.putAll(remoteStorage.multiGetList(clazz, pair.getRight()));
-        return resultMap;
-    }
-
-    private Pair<List<String>, List<String>> getStorageKeys(List<String> allKeys) {
-        List<String> remoteKeys = new ArrayList<>();
-        List<String> localKeys = new ArrayList<>();
-        for (String key : allKeys) {
-            String[] values = MapKeyUtil.splitKey(key);
-            String shuffleId = values[0];
-            boolean isLocal = shufflePartitionManager.isWindowInstanceFinishInit(shuffleId, createWindowInstanceId(key));
-            if (isLocal) {
-                localKeys.add(key);
-            } else {
-                remoteKeys.add(key);
-            }
-        }
-        return Pair.of(localKeys, remoteKeys);
-    }
-
-    @Override
-    public void multiPut(Map<String, T> values) {
-        localStorage.multiPut(values);
-        if (isLocalStorageOnly) {
-            return;
-        }
-        remoteStorage.multiPut(values);
-
-    }
-
-    @Override
-    public Map<String, T> multiGet(Class<T> clazz, List<String> keys) {
-        Map<String, T> result = new HashMap<>();
-        if (isLocalStorageOnly) {
-            result.putAll(localStorage.multiGet(clazz, keys));
-            return result;
-        }
-        Pair<List<String>, List<String>> pair = getStorageKeys(keys);
-        result.putAll(localStorage.multiGet(clazz, pair.getLeft()));
-        result.putAll(remoteStorage.multiGet(clazz, pair.getRight()));
-        return result;
-    }
-
-    @Override
-    public void removeKeys(Collection<String> keys) {
-        localStorage.removeKeys(keys);
-    }
-
-    /**
-     * refer to: WindowMessageProcessor.createStoreKey
-     */
-    public static String createWindowInstanceId(String msgKey) {
-        String[] values = MapKeyUtil.splitKey(msgKey);
-        String[] lastValues = Arrays.copyOfRange(values, 1, values.length - 2);
-        //values[4]: endTime or fireTime
-        return MapKeyUtil.createKey(lastValues);
-    }
-
-    @Override
-    public void delete(String windowInstanceId, String queueId, Class<T> clazz) {
-        this.delete(windowInstanceId, queueId, clazz, null);
-    }
-
-    public void delete(String windowInstanceId, String queueId, Class<T> clazz, SQLCache sqlCache) {
-        localStorage.delete(windowInstanceId, queueId, clazz);
-        if (!isLocalStorageOnly) {
-            if (sqlCache != null) {
-                sqlCache.addCache(new SQLElement(queueId, windowInstanceId, ((IRemoteStorage) this.remoteStorage).deleteSQL(windowInstanceId, queueId, clazz)));
-            } else {
-                remoteStorage.delete(windowInstanceId, queueId, clazz);
-            }
-
-        }
-    }
-
-    public static abstract class WindowBaseValueIterator<T extends WindowBaseValue> implements Iterator<T> {
-        protected long partitionNum = -1;
-
-        public void setPartitionNum(long partitionNum) {
-            this.partitionNum = partitionNum;
-        }
-    }
-
-    @Override
-    public void clearCache(ISplit split, Class<T> clazz) {
-        localStorage.clearCache(split, clazz);
-    }
-
-    public IWindowStorage getLocalStorage() {
-        return localStorage;
-    }
-
-    public IWindowStorage getRemoteStorage() {
-        return remoteStorage;
-    }
-}
diff --git a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBType.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowType.java
similarity index 85%
copy from rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBType.java
copy to rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowType.java
index 83e7978..74029cf 100644
--- a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBType.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowType.java
@@ -1,3 +1,4 @@
+package org.apache.rocketmq.streams.window.storage;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -15,9 +16,8 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.streams.dbinit.mysql.delegate;
-
-public class DBType {
-
-    public static final String DB_MYSQL = "MYSQL";
+public enum WindowType {
+    NORMAL_WINDOW,
+    JOIN_WINDOW,
+    SESSION_WINDOW
 }
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/db/DBStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/db/DBStorage.java
deleted file mode 100644
index afb1819..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/db/DBStorage.java
+++ /dev/null
@@ -1,282 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.storage.db;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-import org.apache.rocketmq.streams.common.utils.CollectionUtil;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.common.utils.SQLUtil;
-import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
-import org.apache.rocketmq.streams.window.model.WindowInstance;
-import org.apache.rocketmq.streams.window.state.WindowBaseValue;
-import org.apache.rocketmq.streams.window.storage.AbstractWindowStorage;
-import org.apache.rocketmq.streams.window.storage.IRemoteStorage;
-import org.apache.rocketmq.streams.window.storage.WindowStorage.WindowBaseValueIterator;
-
-/**
- * database storage
- */
-public class DBStorage<T extends WindowBaseValue> extends AbstractWindowStorage<T> implements IRemoteStorage<T> {
-
-    @Override public String multiPutSQL(Map<String, T> values) {
-        if (CollectionUtil.isEmpty(values)) {
-            return null;
-        }
-        String sql = ORMUtil.createBatchReplacetSQL(new ArrayList<>(values.values()));
-        return sql;
-    }
-
-    @Override public String multiPutListSQL(Map<String, List<T>> infoMap) {
-        if (CollectionUtil.isNotEmpty(infoMap)) {
-            List<T> valueList = duplicate(infoMap);
-            return ORMUtil.createBatchReplacetSQL(valueList);
-        }
-        return null;
-    }
-
-    /**
-     * the list value has the same store key, add suffix for session window
-     *
-     * @param infoMap
-     * @return
-     */
-    private List<T> duplicate(Map<String, List<T>> infoMap) {
-        List<T> resultList = new ArrayList<>();
-        Iterator<Map.Entry<String, List<T>>> iterator = infoMap.entrySet().iterator();
-        while (iterator.hasNext()) {
-            Map.Entry<String, List<T>> entry = iterator.next();
-            List<T> valueList = entry.getValue();
-            for (int index = 0; index < valueList.size(); index++) {
-                //TODO 是否要进行clone
-                T value = valueList.get(index);
-                value.setMsgKey(value.getMsgKey() + "_" + index);
-                resultList.add(value);
-            }
-        }
-        return resultList;
-    }
-
-    @Override
-    public void multiPut(Map<String, T> values) {
-        if (CollectionUtil.isEmpty(values)) {
-            return;
-        }
-        ORMUtil.batchReplaceInto(values.values());
-    }
-
-    @Override
-    public Map<String, T> multiGet(Class<T> clazz, List<String> keys) {
-        if (CollectionUtil.isEmpty(keys)) {
-            return new HashMap<>(4);
-        }
-        Map<String, String> md5Key2Keys = new HashMap<>();
-        List<String> md5Keys = new ArrayList<>();
-        for (String key : keys) {
-            String md5Key = StringUtil.createMD5Str(key);
-            md5Keys.add(md5Key);
-            md5Key2Keys.put(md5Key, key);
-        }
-        List<T> values = ORMUtil.queryForList("select * from " + ORMUtil.getTableName(clazz) +
-            " where msg_key in (" + SQLUtil.createInSql(md5Keys) + " )", new HashMap<>(4), clazz);
-        Map<String, T> map = new HashMap<>(keys.size());
-        for (T value : values) {
-            String key = md5Key2Keys.get(value.getMsgKey());
-            map.put(key, value);
-        }
-        return map;
-    }
-
-    @Override public void multiPutList(Map<String, List<T>> elements) {
-        if (CollectionUtil.isEmpty(elements)) {
-            return;
-        }
-        List<T> valueList = duplicate(elements);
-        ORMUtil.batchReplaceInto(valueList);
-    }
-
-    /**
-     * the key in db is md5(key)_index
-     *
-     * @param clazz
-     * @param keys
-     * @return
-     */
-    @Override public Map<String, List<T>> multiGetList(Class<T> clazz, List<String> keys) {
-        if (CollectionUtil.isEmpty(keys)) {
-            return new HashMap<>(4);
-        }
-        Map<String, String> recordMap = new HashMap<>(keys.size());
-        List<String> dbKeyList = new ArrayList<>(keys.size());
-        List<Pair<String, String>> variableAndValue = new ArrayList<>(keys.size());
-        for (String key : keys) {
-            String md5Value = StringUtil.createMD5Str(key);
-            dbKeyList.add(md5Value);
-            recordMap.put(md5Value, key);
-            variableAndValue.add(Pair.of("msg_key", md5Value + "%"));
-        }
-        List<T> values = ORMUtil.queryForList("select * from " + ORMUtil.getTableName(clazz) +
-            " where " + SQLUtil.createLikeSql(variableAndValue), new HashMap<>(4), clazz);
-        Map<String, List<T>> resultMap = new HashMap<>(keys.size());
-        for (T value : values) {
-            String dbKeyWithoutSuffix = value.getMsgKey().substring(0, 24);
-            value.setMsgKey(dbKeyWithoutSuffix);
-            String key = recordMap.get(dbKeyWithoutSuffix);
-            List<T> valueList = resultMap.getOrDefault(key, null);
-            if (valueList == null) {
-                valueList = new ArrayList<>();
-                resultMap.put(key, valueList);
-            }
-            valueList.add(value);
-        }
-        return resultMap;
-    }
-
-    @Override
-    public void removeKeys(Collection<String> keys) {
-
-    }
-
-    @Override
-    public WindowBaseValueIterator<T> loadWindowInstanceSplitData(String localStorePrefix, String queueId,
-        String windowInstanceId, String keyPrex, Class<T> clazz) {
-
-        //search max partition number in case of inserting fresh data [min,max)
-        long maxPartitionIndex = getPartitionNum(windowInstanceId, clazz, true) + 1;
-        long mimPartitionIndex = getPartitionNum(windowInstanceId, clazz, false) - 1;
-        if (maxPartitionIndex <= 1) {
-            return new WindowBaseValueIterator<T>() {
-                @Override
-                public boolean hasNext() {
-                    return false;
-                }
-
-                @Override
-                public T next() {
-                    return null;
-                }
-            };
-        }
-
-        DBIterator dbIterator = new DBIterator<T>(queueId, windowInstanceId, keyPrex, clazz, maxPartitionIndex);
-        dbIterator.setPartitionNum(mimPartitionIndex);
-        return dbIterator;
-    }
-
-    @Override public Long getMaxSplitNum(WindowInstance windowInstance, Class<T> clazz) {
-        return getPartitionNum(windowInstance.createWindowInstanceId(), clazz, true);
-    }
-
-    @Override
-    public void clearCache(ISplit channelQueue, Class<T> clazz) {
-        throw new RuntimeException("can not support this method");
-    }
-
-    @Override
-    public void delete(String windowInstanceId, String queueId, Class<T> clazz) {
-
-        ORMUtil.executeSQL(
-            deleteSQL(windowInstanceId, queueId, clazz),
-            new HashMap<>(4));
-    }
-
-    @Override public String deleteSQL(String windowInstanceId, String queueId, Class<T> clazz) {
-        String sql = "delete from " + ORMUtil.getTableName(clazz) + " where window_instance_id = '" + StringUtil.createMD5Str(windowInstanceId) + "'";
-        return sql;
-    }
-
-    public static class DBIterator<T extends WindowBaseValue> extends WindowBaseValueIterator<T> {
-        private LinkedList<T> container = new LinkedList<>();
-        int batchSize = 1000;
-        private boolean exist = true;
-
-        private long maxPartitionIndex;
-        private Class<T> clazz;
-
-        String sql;
-
-        public DBIterator(String queueId, String windowInstanceId, String keyPrex, Class<T> clazz,
-            long maxPartitionIndex) {
-            String windowInstancePartitionId = StringUtil.createMD5Str(windowInstanceId);
-
-            if (StringUtil.isEmpty(keyPrex)) {
-                sql = "select * from " + ORMUtil.getTableName(clazz)
-                    + " where window_instance_partition_id = '" + windowInstancePartitionId
-                    + "' and partition_num > #{partitionNum} order by window_instance_partition_id, partition_num limit "
-                    + batchSize;
-            } else {
-                //join usage(different clazz)
-                String prefix = MapKeyUtil.createKey(queueId, windowInstanceId, keyPrex);
-                sql = "select * from " + ORMUtil.getTableName(clazz) + " where window_instance_partition_id ='"
-                    + windowInstancePartitionId + "' " +
-                    "and msg_key like '" + prefix
-                    + "%' and  partition_num > #{partitionNum} order by window_instance_partition_id, partition_num  limit "
-                    + batchSize;
-            }
-            this.maxPartitionIndex = maxPartitionIndex;
-            this.clazz = clazz;
-        }
-
-        @Override
-        public boolean hasNext() {
-            if (!container.isEmpty()) {
-                return true;
-            } else if (!exist) {
-                return false;
-            } else {
-                Map<String, Long> parameter = new HashMap<>(4);
-                parameter.put("partitionNum", partitionNum);
-                exist = partitionNum + batchSize <= maxPartitionIndex;
-                List<T> batchResult = ORMUtil.queryForList(sql, parameter, clazz);
-                if (CollectionUtil.isEmpty(batchResult)) {
-                    return false;
-                } else {
-                    partitionNum = batchResult.get(batchResult.size() - 1).getPartitionNum();
-                    container.addAll(batchResult);
-                    return true;
-                }
-            }
-        }
-
-        @Override
-        public T next() {
-            return container.poll();
-        }
-
-    }
-
-    protected Long getPartitionNum(String windowInstanceId, Class<T> clazz, boolean isMax) {
-        String partitionNumSQL = isMax ? "max(partition_num)" : "min(partition_num)";
-        String windowInstancePartitionId = StringUtil.createMD5Str(windowInstanceId);
-        String sql = "select " + partitionNumSQL + " as partition_num from " + ORMUtil.getTableName(clazz)
-            + " where window_instance_partition_id ='" + windowInstancePartitionId + "'";
-        WindowBaseValue windowBaseValue = ORMUtil.queryForObject(sql, new HashMap<>(4), clazz);
-        if (windowBaseValue == null) {
-            return null;
-        }
-        return windowBaseValue.getPartitionNum();
-    }
-
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/file/FileStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/file/FileStorage.java
deleted file mode 100644
index 200e93b..0000000
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/file/FileStorage.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window.storage.file;
-
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.rocketmq.streams.window.state.WindowBaseValue;
-
-public class FileStorage<T extends WindowBaseValue> {
-    private static final String SPLIT_SIGN = "############";
-    protected transient String filePath = "/tmp/storage/file.storage";
-    protected transient Map<String, String> cache = new HashMap<>();
-
-}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocketmq/DefaultStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocketmq/DefaultStorage.java
new file mode 100644
index 0000000..69d5b03
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocketmq/DefaultStorage.java
@@ -0,0 +1,390 @@
+package org.apache.rocketmq.streams.window.storage.rocketmq;
+/*
+ * 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.
+ */
+
+import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.streams.common.utils.CreateTopicUtil;
+import org.apache.rocketmq.streams.common.utils.SerializeUtil;
+import org.apache.rocketmq.streams.window.model.WindowInstance;
+import org.apache.rocketmq.streams.window.state.WindowBaseValue;
+import org.apache.rocketmq.streams.window.storage.AbstractStorage;
+import org.apache.rocketmq.streams.window.storage.DataType;
+import org.apache.rocketmq.streams.window.storage.IteratorWrap;
+import org.apache.rocketmq.streams.window.storage.RocksdbIterator;
+import org.apache.rocketmq.streams.window.storage.WindowJoinType;
+import org.apache.rocketmq.streams.window.storage.WindowType;
+import org.apache.rocketmq.streams.window.storage.rocksdb.RocksdbStorage;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+
+import static org.apache.rocketmq.streams.window.storage.DataType.WINDOW_INSTANCE;
+
+public class DefaultStorage extends AbstractStorage {
+    private final boolean isLocalStorageOnly;
+    private final RocksdbStorage rocksdbStorage;
+    private final String clusterName = "DefaultCluster";
+
+    //两个streams实例topic可能相同,但是tag不同
+    private final String topic;
+    private final String groupId;
+
+    private DefaultMQProducer producer;
+    private DefaultLitePullConsumer checkpointConsumer;
+    private DefaultLitePullConsumer loadConsumer;
+
+    private static final long pollTimeoutMillis = 50L;
+    private Map<Integer, MessageQueue> queueId2MQ = new HashMap<>();
+    private ExecutorService loadExecutor;
+    private ExecutorService checkpointExecutor;
+
+    public DefaultStorage(String topic, String groupId, String namesrv, int queueNum,
+                          boolean isLocalStorageOnly, RocksdbStorage rocksdbStorage) {
+        this.isLocalStorageOnly = isLocalStorageOnly;
+        this.rocksdbStorage = rocksdbStorage;
+
+        this.topic = topic;
+        this.groupId = groupId;
+
+
+        if (!isLocalStorageOnly) {
+            this.loadExecutor = Executors.newSingleThreadExecutor();
+            this.checkpointExecutor = Executors.newSingleThreadExecutor();
+            ;
+
+            try {
+                this.producer = new DefaultMQProducer(groupId);
+                this.producer.setNamesrvAddr(namesrv);
+                this.producer.start();
+                //create topic
+                CreateTopicUtil.create(clusterName, topic, queueNum, namesrv);
+
+                this.checkpointConsumer = new DefaultLitePullConsumer(this.groupId);
+                this.checkpointConsumer.setNamesrvAddr(namesrv);
+                this.checkpointConsumer.setAutoCommit(false);
+                this.checkpointConsumer.start();
+
+                this.loadConsumer = new DefaultLitePullConsumer(this.groupId);
+                this.loadConsumer.setNamesrvAddr(namesrv);
+                this.loadConsumer.setAutoCommit(false);
+                this.loadConsumer.start();
+            } catch (Throwable t) {
+                throw new RuntimeException("start rocketmq client error.", t);
+            }
+        }
+    }
+
+    @Override
+    public Future<?> load(String shuffleId) {
+        if (isLocalStorageOnly) {
+            return super.load(shuffleId);
+        }
+
+        MessageQueue messageQueue = getMessageQueue(shuffleId);
+        if (messageQueue == null) {
+            throw new RuntimeException("can not find MQ with shuffleId = [" + shuffleId + "]");
+        }
+
+        //从上一offset提交位置,poll到最新数据位置
+        return this.loadExecutor.submit(() -> this.pollToLast(messageQueue, true));
+    }
+
+    private void pollToLast(MessageQueue messageQueue, boolean replay) {
+        List<MessageQueue> temp = new ArrayList<>();
+        temp.add(messageQueue);
+
+        try {
+
+            if (replay) {
+                //assign 与poll必须原子;
+                synchronized (this.loadConsumer) {
+                    this.loadConsumer.assign(temp);
+
+                    List<MessageExt> msgs = this.loadConsumer.poll(pollTimeoutMillis);
+                    while (msgs.size() != 0) {
+                        this.replayState(msgs);
+                        msgs = this.loadConsumer.poll(pollTimeoutMillis);
+                    }
+                }
+            } else {
+                //assign 与poll必须原子;
+                synchronized (this.checkpointConsumer) {
+                    this.checkpointConsumer.assign(temp);
+
+                    List<MessageExt> msgs = this.checkpointConsumer.poll(pollTimeoutMillis);
+                    while (msgs.size() != 0) {
+                        msgs = this.checkpointConsumer.poll(pollTimeoutMillis);
+                    }
+                }
+            }
+
+        } catch (Throwable ignored) {
+        }
+    }
+
+    private void replayState(List<MessageExt> msgs) {
+        if (msgs == null || msgs.size() == 0) {
+            return;
+        }
+
+        //按照key进行分组;
+        Map<String, List<MessageExt>> collect = msgs.stream().parallel().collect(Collectors.groupingBy(MessageExt::getKeys));
+
+
+        //对每组key的所有msg的时间戳进行比较,过滤出最大时间戳的值,即为最后的状态
+        HashMap<String, MessageExt> lastStates = new HashMap<>();
+        collect.forEach((key, values) -> {
+
+            long maxBornTimestamp = 0;
+            MessageExt lastMsgExt = null;
+
+            for (MessageExt msgExt : values) {
+                long bornTimestamp = msgExt.getBornTimestamp();
+                if (bornTimestamp > maxBornTimestamp) {
+                    maxBornTimestamp = bornTimestamp;
+                    lastMsgExt = msgExt;
+                }
+            }
+            lastStates.put(key, lastMsgExt);
+        });
+
+        this.convert(lastStates);
+    }
+
+    private void convert(HashMap<String, MessageExt> lastStates) {
+        for (String key : lastStates.keySet()) {
+            MessageExt newState = lastStates.get(key);
+
+            if (key.startsWith(WINDOW_INSTANCE.getValue()) || key.startsWith(DataType.WINDOW_BASE_VALUE.getValue())
+                    || key.startsWith(DataType.MAX_OFFSET.getValue()) || key.startsWith(DataType.MAX_PARTITION_NUM.getValue())) {
+                updateState(key, newState);
+            }
+        }
+    }
+
+
+    private synchronized void updateState(String key, MessageExt newState) {
+        byte[] body = newState.getBody();
+        Object newValue = SerializeUtil.deserialize(body);
+        if (body == null || newValue == null) {
+            return;
+        }
+
+        byte[] oldBytes = rocksdbStorage.get(key);
+        Object oldValue = SerializeUtil.deserialize(oldBytes);
+
+
+        long newTimestamp = getTimestamp(newValue);
+        long oldTimestamp = getTimestamp(oldValue);
+
+        if (newTimestamp > oldTimestamp) {
+            rocksdbStorage.put(key, body);
+        }
+
+        //windowInstance为窗口元数据,不存在更新的情况
+    }
+
+
+    @Override
+    public void putWindowInstance(String shuffleId, String windowNamespace, String windowConfigureName, WindowInstance windowInstance) {
+        rocksdbStorage.putWindowInstance(shuffleId, windowNamespace, windowConfigureName, windowInstance);
+    }
+
+    @Override
+    public <T> RocksdbIterator<T> getWindowInstance(String shuffleId, String windowNamespace, String windowConfigureName) {
+        return rocksdbStorage.getWindowInstance(shuffleId, windowNamespace, windowConfigureName);
+    }
+
+    //put的key是什么,就按照什么key删除
+    @Override
+    public void deleteWindowInstance(String shuffleId, String windowNamespace, String windowConfigureName, String windowInstanceId) {
+        rocksdbStorage.deleteWindowInstance(shuffleId, windowNamespace, windowConfigureName, windowInstanceId);
+    }
+
+
+    @Override
+    public void putWindowBaseValue(String shuffleId, String windowInstanceId, WindowType windowType,
+                                   WindowJoinType joinType, List<WindowBaseValue> windowBaseValue) {
+        rocksdbStorage.putWindowBaseValue(shuffleId, windowInstanceId, windowType, joinType, windowBaseValue);
+    }
+
+    public void putWindowBaseValueIterator(String shuffleId, String windowInstanceId,
+                                           WindowType windowType, WindowJoinType joinType,
+                                           RocksdbIterator<? extends WindowBaseValue> windowBaseValueIterator) {
+        rocksdbStorage.putWindowBaseValueIterator(shuffleId, windowInstanceId, windowType, joinType, windowBaseValueIterator);
+    }
+
+    //读取消息重放,或者查询并存储到内存
+    @Override
+    public <T> RocksdbIterator<T> getWindowBaseValue(String shuffleId, String windowInstanceId, WindowType windowType, WindowJoinType joinType) {
+        return rocksdbStorage.getWindowBaseValue(shuffleId, windowInstanceId, windowType, joinType);
+    }
+
+    //按照put key的前缀删除,没有唯一键,删除一批
+    @Override
+    public void deleteWindowBaseValue(String shuffleId, String windowInstanceId, WindowType windowType, WindowJoinType joinType) {
+        rocksdbStorage.deleteWindowBaseValue(shuffleId, windowInstanceId, windowType, joinType);
+    }
+
+    public void deleteWindowBaseValue(String shuffleId, String windowInstanceId, WindowType windowType, WindowJoinType joinType, String msgKey) {
+        rocksdbStorage.deleteWindowBaseValue(shuffleId, windowInstanceId, windowType, joinType, msgKey);
+    }
+
+    @Override
+    public String getMaxOffset(String shuffleId, String windowConfigureName, String oriQueueId) {
+        return rocksdbStorage.getMaxOffset(shuffleId, windowConfigureName, oriQueueId);
+    }
+
+    @Override
+    public void putMaxOffset(String shuffleId, String windowConfigureName, String oriQueueId, String offset) {
+        rocksdbStorage.putMaxOffset(shuffleId, windowConfigureName, oriQueueId, offset);
+    }
+
+    @Override
+    public void deleteMaxOffset(String shuffleId, String windowConfigureName, String oriQueueId) {
+        rocksdbStorage.deleteMaxOffset(shuffleId, windowConfigureName, oriQueueId);
+    }
+
+    @Override
+    public void putMaxPartitionNum(String shuffleId, String windowInstanceId, long maxPartitionNum) {
+        rocksdbStorage.putMaxPartitionNum(shuffleId, windowInstanceId, maxPartitionNum);
+    }
+
+    @Override
+    public Long getMaxPartitionNum(String shuffleId, String windowInstanceId) {
+        return rocksdbStorage.getMaxPartitionNum(shuffleId, windowInstanceId);
+    }
+
+    @Override
+    public void deleteMaxPartitionNum(String shuffleId, String windowInstanceId) {
+        rocksdbStorage.deleteMaxPartitionNum(shuffleId, windowInstanceId);
+    }
+
+    //按照queueId提交offset,避免了不同streams实例,多次提交offset
+    @Override
+    public int flush(List<String> queueIdList) {
+        if (isLocalStorageOnly) {
+            return super.flush(queueIdList);
+        }
+
+        int successNum = 0;
+        try {
+            for (String queueId : queueIdList) {
+                successNum += sendSync(queueId);
+            }
+
+            //提交上次checkpoint/load时,poll消息的offset
+            for (String queueId : queueIdList) {
+                final MessageQueue queue = getMessageQueue(queueId);
+                //todo 指定messageQueue提交offset
+                HashSet<MessageQueue> set = new HashSet<>();
+                set.add(queue);
+
+                this.checkpointConsumer.commit(set, true);
+
+                //poll到最新的checkpoint,为下一次提交offset做准备;
+                this.checkpointExecutor.execute(() -> this.pollToLast(queue, false));
+            }
+
+        } catch (Throwable t) {
+            throw new RuntimeException("send data to rocketmq synchronously,error.", t);
+        }
+
+        return successNum;
+    }
+
+    private int sendSync(String shuffleId) {
+        int count = 0;
+
+        for (DataType dataType : DataType.values()) {
+            count += send(shuffleId, dataType);
+        }
+
+        return count;
+    }
+
+    private int send(String shuffleId, DataType dataType) {
+        int count = 0;
+
+        RocksdbIterator<Object> iterator = rocksdbStorage.getData(shuffleId, dataType);
+        while (iterator.hasNext()) {
+            IteratorWrap<Object> wrap = iterator.next();
+
+            byte[] raw = wrap.getRaw();
+            if (raw != null && raw.length != 0) {
+                count += send0(shuffleId, wrap.getKey(), raw);
+            }
+        }
+
+        return count;
+    }
+
+
+    private int send0(String shuffleId, String key, byte[] body) {
+        MessageQueue queue = getMessageQueue(shuffleId);
+        try {
+
+            Message message = new Message(topic, "", key, body);
+            //选择MQ写入,后面commitOffset时对这个MQ进行
+            producer.send(message, queue);
+
+            return 1;
+        } catch (Throwable t) {
+            throw new RuntimeException("send data to rocketmq asynchronously,error.", t);
+        }
+    }
+
+    //状态topic的MQ数量与shuffle topic的MQ数量需要相同
+    private MessageQueue getMessageQueue(String shuffleId) {
+        //最后四位为queueId
+        String substring = shuffleId.substring(shuffleId.length() - 3);
+        Integer queueIdNumber = Integer.parseInt(substring);
+
+        MessageQueue result = queueId2MQ.get(queueIdNumber);
+
+        if (result == null) {
+            try {
+                Collection<MessageQueue> mqs = this.checkpointConsumer.fetchMessageQueues(topic);
+                if (mqs != null) {
+                    Map<Integer, List<MessageQueue>> temp = mqs.stream().collect(Collectors.groupingBy(MessageQueue::getQueueId));
+                    for (Integer queueId : temp.keySet()) {
+                        List<MessageQueue> messageQueues = temp.get(queueId);
+                        assert messageQueues.size() == 1;
+                        this.queueId2MQ.put(queueId, messageQueues.get(0));
+                    }
+                }
+            } catch (Throwable t) {
+                System.out.println(t);
+            }
+        }
+
+        return queueId2MQ.get(queueIdNumber);
+    }
+
+}
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/RocksdbStorage.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/RocksdbStorage.java
index 3455667..c9aeff9 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/RocksdbStorage.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocksdb/RocksdbStorage.java
@@ -1,3 +1,4 @@
+package org.apache.rocketmq.streams.window.storage.rocksdb;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -14,321 +15,374 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.streams.window.storage.rocksdb;
 
-import com.alibaba.fastjson.JSONArray;
-import java.io.UnsupportedEncodingException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-import org.apache.rocketmq.streams.common.utils.Base64Utils;
-import org.apache.rocketmq.streams.common.utils.CollectionUtil;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
 import org.apache.rocketmq.streams.common.utils.SerializeUtil;
-import org.apache.rocketmq.streams.common.utils.StringUtil;
 import org.apache.rocketmq.streams.state.kv.rocksdb.RocksDBOperator;
 import org.apache.rocketmq.streams.window.model.WindowInstance;
 import org.apache.rocketmq.streams.window.state.WindowBaseValue;
-import org.apache.rocketmq.streams.window.storage.AbstractWindowStorage;
-import org.apache.rocketmq.streams.window.storage.WindowStorage.WindowBaseValueIterator;
-import org.rocksdb.ReadOptions;
+import org.apache.rocketmq.streams.window.state.impl.JoinState;
+import org.apache.rocketmq.streams.window.storage.AbstractStorage;
+import org.apache.rocketmq.streams.window.storage.DataType;
+import org.apache.rocketmq.streams.window.storage.IteratorWrap;
+import org.apache.rocketmq.streams.window.storage.RocksdbIterator;
+import org.apache.rocketmq.streams.window.storage.WindowJoinType;
+import org.apache.rocketmq.streams.window.storage.WindowType;
 import org.rocksdb.RocksDB;
-import org.rocksdb.RocksDBException;
-import org.rocksdb.RocksIterator;
-import org.rocksdb.WriteBatch;
 import org.rocksdb.WriteOptions;
 
-public class RocksdbStorage<T extends WindowBaseValue> extends AbstractWindowStorage<T> {
-    protected static String DB_PATH = "/tmp/rocksdb";
-    protected static String UTF8 = "UTF8";
-    protected static AtomicBoolean hasCreate = new AtomicBoolean(false);
-    protected static RocksDB rocksDB = new RocksDBOperator().getInstance();
-    protected WriteOptions writeOptions = new WriteOptions();
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
 
-    @Override
-    public void removeKeys(Collection<String> keys) {
+public class RocksdbStorage extends AbstractStorage {
+    private RocksDB rocksDB;
+    private WriteOptions writeOptions;
 
-        for (String key : keys) {
-            try {
-                rocksDB.delete(getKeyBytes(key));
-            } catch (RocksDBException e) {
-                throw new RuntimeException("delete error " + key);
-            }
-        }
-
+    public RocksdbStorage() {
+        rocksDB = new RocksDBOperator().getInstance();
+        writeOptions = new WriteOptions();
+        writeOptions.setSync(false);
+        writeOptions.setDisableWAL(true);
     }
 
-    @Override
-    public WindowBaseValueIterator<T> loadWindowInstanceSplitData(String localStorePrefix, String queueId,
-        String windowInstanceId, String key, Class<T> clazz) {
-        String keyPrefix = MapKeyUtil.createKey(queueId, windowInstanceId, key);
-        if (StringUtil.isNotEmpty(localStorePrefix)) {
-            keyPrefix = localStorePrefix + keyPrefix;
-        }
-        return getByKeyPrefix(keyPrefix, clazz, false);
-    }
-
-    @Override public Long getMaxSplitNum(WindowInstance windowInstance, Class<T> clazz) {
-        throw new RuntimeException("can not support this method");
-    }
 
     @Override
-    public void multiPut(Map<String, T> values) {
-        if (values == null) {
+    public void putWindowInstance(String shuffleId, String windowNamespace, String windowConfigureName, WindowInstance windowInstance) {
+        if (windowInstance == null) {
             return;
         }
-        try {
-            WriteBatch writeBatch = new WriteBatch();
-            Iterator<Entry<String, T>> it = values.entrySet().iterator();
-            while (it.hasNext()) {
-                Entry<String, T> entry = it.next();
-                String key = entry.getKey();
-                byte[] value = SerializeUtil.serialize(entry.getValue());
-                writeBatch.put(key.getBytes(UTF8), value);
-            }
 
-            WriteOptions writeOptions = new WriteOptions();
-            writeOptions.setSync(false);
-            writeOptions.setDisableWAL(true);
-            rocksDB.write(writeOptions, writeBatch);
-            writeBatch.close();
-            writeOptions.close();
+        //唯一键
+        String windowInstanceId = windowInstance.getWindowInstanceId();
+
+        String key = super.merge(DataType.WINDOW_INSTANCE.getValue(), shuffleId, windowNamespace, windowConfigureName, windowInstanceId);
+
+        byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
+        byte[] valueBytes = SerializeUtil.serialize(windowInstance);
+
+        try {
+            rocksDB.put(writeOptions, keyBytes, valueBytes);
         } catch (Exception e) {
-            e.printStackTrace();
-            throw new RuntimeException("put data to rocksdb error", e);
+            throw new RuntimeException("putWindowInstance to rocksdb error", e);
         }
     }
 
     @Override
-    public Map<String, T> multiGet(Class<T> clazz, List<String> keys) {
-        if (keys == null || keys.size() == 0) {
-            return new HashMap<>();
-        }
-        List<byte[]> keyByteList = new ArrayList<>();
-        List<String> keyStrList = new ArrayList<>();
-        for (String key : keys) {
-            keyByteList.add(getKeyBytes(key));
-            keyStrList.add(key);
-        }
-        try {
-            Map<String, T> jsonables = new HashMap<>();
-            //            List<byte[]>  list=  rocksDB.multiGetAsList(keyByteList);
-            Map<byte[], byte[]> map = rocksDB.multiGet(keyByteList);
-            int i = 0;
-            Iterator<Entry<byte[], byte[]>> it = map.entrySet().iterator();
-            while (it.hasNext()) {
-                Entry<byte[], byte[]> entry = it.next();
-                String key = getValueFromByte(entry.getKey());
-                T value = (T) SerializeUtil.deserialize(entry.getValue());
-                jsonables.put(key, value);
-            }
-            //            for(byte[] bytes:list){
-            return jsonables;
-        } catch (RocksDBException e) {
-            throw new RuntimeException("can not get value from rocksdb ", e);
-        }
+    public <T> RocksdbIterator<T> getWindowInstance(String shuffleId, String windowNamespace, String windowConfigureName) {
+        String keyPrefix = super.merge(DataType.WINDOW_INSTANCE.getValue(), shuffleId, windowNamespace, windowConfigureName);
 
+        return new RocksdbIterator<>(keyPrefix, rocksDB);
     }
 
-    @Override public void multiPutList(Map<String, List<T>> elements) {
-        if (CollectionUtil.isEmpty(elements)) {
+    @Override
+    public void deleteWindowInstance(String shuffleId, String windowNamespace, String windowConfigureName, String windowInstanceId) {
+        if (windowInstanceId == null) {
             return;
         }
+
+        String key = super.merge(DataType.WINDOW_INSTANCE.getValue(), shuffleId, windowNamespace, windowConfigureName, windowInstanceId);
+
         try {
-            WriteBatch writeBatch = new WriteBatch();
-            Iterator<Entry<String, List<T>>> it = elements.entrySet().iterator();
-            while (it.hasNext()) {
-                Entry<String, List<T>> entry = it.next();
-                String key = entry.getKey();
-                List<T> valueList = entry.getValue();
-                JSONArray array = new JSONArray();
-                for (T value : valueList) {
-                    array.add(Base64Utils.encode(SerializeUtil.serialize(value)));
-                }
-                writeBatch.put(key.getBytes(UTF8), array.toJSONString().getBytes(UTF8));
-            }
-            WriteOptions writeOptions = new WriteOptions();
-            writeOptions.setSync(false);
-            writeOptions.setDisableWAL(true);
-            rocksDB.write(writeOptions, writeBatch);
-            writeBatch.close();
-            writeOptions.close();
+            byte[] bytes = key.getBytes(StandardCharsets.UTF_8);
+            rocksDB.delete(writeOptions, bytes);
         } catch (Exception e) {
-            e.printStackTrace();
-            throw new RuntimeException("put data to rocksdb error", e);
-        }
-    }
-
-    @Override public Map<String, List<T>> multiGetList(Class<T> clazz, List<String> keys) {
-        if (CollectionUtil.isEmpty(keys)) {
-            return new HashMap<>(4);
-        }
-        List<byte[]> keyByteList = new ArrayList<>();
-        for (String key : keys) {
-            keyByteList.add(getKeyBytes(key));
-        }
-        try {
-            Map<String, List<T>> resultMap = new HashMap<>();
-            Map<byte[], byte[]> map = rocksDB.multiGet(keyByteList);
-            int i = 0;
-            Iterator<Entry<byte[], byte[]>> it = map.entrySet().iterator();
-            while (it.hasNext()) {
-                Entry<byte[], byte[]> entry = it.next();
-                String key = getValueFromByte(entry.getKey());
-                String value = getValueFromByte(entry.getValue());
-                JSONArray array = JSONArray.parseArray(value);
-                List<T> valueList = new ArrayList<>();
-                for (int index = 0; index < array.size(); index++) {
-                    String objectString = array.getString(index);
-                    byte[] bytes = Base64Utils.decode(objectString);
-                    T valueObject = SerializeUtil.deserialize(bytes);
-                    valueList.add(valueObject);
-                }
-                resultMap.put(key, valueList);
-            }
-            return resultMap;
-        } catch (RocksDBException e) {
-            e.printStackTrace();
-            throw new RuntimeException("can not get multi value from rocksdb! ", e);
+            throw new RuntimeException("deleteWindowInstance from rocksdb error", e);
         }
     }
 
     @Override
-    public void clearCache(ISplit split, Class<T> clazz) {
-        deleteRange(split.getQueueId(), clazz);
+    public void putWindowBaseValue(String shuffleId, String windowInstanceId, WindowType windowType, WindowJoinType joinType, List<WindowBaseValue> windowBaseValue) {
+        if (windowBaseValue == null || windowBaseValue.size() == 0) {
+            return;
+        }
+
+
+        for (WindowBaseValue baseValue : windowBaseValue) {
+            doPut(baseValue, shuffleId, windowInstanceId, windowType, joinType);
+        }
+    }
+
+    public void putWindowBaseValueIterator(String shuffleId, String windowInstanceId,
+                                           WindowType windowType, WindowJoinType joinType,
+                                           RocksdbIterator<? extends WindowBaseValue> windowBaseValueIterator) {
+        if (windowBaseValueIterator == null) {
+            return;
+        }
+
+        while (windowBaseValueIterator.hasNext()) {
+            IteratorWrap<? extends WindowBaseValue> next = windowBaseValueIterator.next();
+            WindowBaseValue data = next.getData();
+
+            doPut(data, shuffleId, windowInstanceId, windowType, joinType);
+        }
+    }
+
+
+    private void doPut(WindowBaseValue baseValue, String shuffleId, String windowInstanceId, WindowType windowType, WindowJoinType joinType) {
+        String key = createKey(shuffleId, windowInstanceId, windowType, joinType, baseValue);
+
+
+        try {
+            byte[] valueBytes;
+
+            byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
+            valueBytes = SerializeUtil.serialize(baseValue);
+            rocksDB.put(writeOptions, keyBytes, valueBytes);
+        } catch (Throwable t) {
+            throw new RuntimeException("put data to rocksdb error", t);
+        }
+
+    }
+
+
+    @Override
+    public <T> RocksdbIterator<T> getWindowBaseValue(String shuffleId, String windowInstanceId, WindowType windowType, WindowJoinType joinType) {
+
+        String keyPrefix = createKey(shuffleId, windowInstanceId, windowType, joinType, null);
+
+        return new RocksdbIterator<>(keyPrefix, rocksDB);
     }
 
     @Override
-    public void delete(String windowInstanceId, String queueId, Class<T> clazz) {
-        //范围删除影响性能,改成了通过removekey删除
-        //String plusWindowInstaceId=null;
-        //  String lastWord=windowInstanceId.substring(windowInstanceId.length()-2,windowInstanceId.length());
-        String firstKey = MapKeyUtil.createKey(queueId, windowInstanceId);
-        deleteRange(firstKey, clazz);
-
-    }
-
-    protected void deleteRange(String startKey, Class<T> clazz) {
+    public void deleteWindowBaseValue(String shuffleId, String windowInstanceId, WindowType windowType, WindowJoinType joinType) {
         try {
-            // rocksDB.deleteRange(getKeyBytes(startKey),getKeyBytes(endKey));
-            WindowBaseValueIterator<T> iterator = getByKeyPrefix(startKey, clazz, true);
-            Set<String> deleteKeys = new HashSet<>();
-            while (iterator.hasNext()) {
-                WindowBaseValue windowBaseValue = iterator.next();
-                if (windowBaseValue == null) {
+            String keyPrefix = createKey(shuffleId, windowInstanceId, windowType, joinType, null);
+
+            //查询msgKey
+            RocksdbIterator<WindowBaseValue> rocksdbIterator = new RocksdbIterator<>(keyPrefix, rocksDB);
+
+            ArrayList<String> msgKeys = new ArrayList<>();
+            while (rocksdbIterator.hasNext()) {
+                IteratorWrap<WindowBaseValue> baseValue = rocksdbIterator.next();
+                WindowBaseValue data = baseValue.getData();
+                if (data == null) {
                     continue;
                 }
-                deleteKeys.add(windowBaseValue.getMsgKey());
-                if (deleteKeys.size() >= 1000) {
-                    this.removeKeys(deleteKeys);
-                    deleteKeys = new HashSet<>();
-                }
+
+                msgKeys.add(data.getMsgKey());
             }
-            if (deleteKeys.size() > 0) {
-                this.removeKeys(deleteKeys);
+
+            //组合成真正的key后,在挨个删除
+            for (String msgKey : msgKeys) {
+                String key = super.merge(keyPrefix, msgKey);
+                byte[] bytes = key.getBytes(StandardCharsets.UTF_8);
+                rocksDB.delete(writeOptions, bytes);
             }
+
         } catch (Exception e) {
-            throw new RuntimeException(e);
+            throw new RuntimeException("deleteWindowBaseValue from rocksdb error", e);
         }
     }
 
-    protected WindowBaseValueIterator<T> getByKeyPrefix(String keyPrefix, Class<? extends T> clazz, boolean needKey) {
-        return new LocalIterator<T>(keyPrefix, clazz, needKey);
+    public void deleteWindowBaseValue(String shuffleId, String windowInstanceId, WindowType windowType, WindowJoinType joinType, String msgKey) {
+        String key;
+        if (joinType != null) {
+            key = super.merge(shuffleId, windowInstanceId, windowType.name(), joinType.name(), msgKey);
+        } else {
+            key = super.merge(shuffleId, windowInstanceId, windowType.name(), msgKey);
+        }
+
+        try {
+            byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
+            rocksDB.delete(writeOptions, keyBytes);
+        } catch (Throwable t) {
+            throw new RuntimeException("deleteWindowBaseValue from rocksdb error", t);
+        }
     }
 
-    public static class LocalIterator<T extends WindowBaseValue> extends WindowBaseValueIterator<T> {
-        protected volatile boolean hasNext = true;
-        protected AtomicBoolean hasInit = new AtomicBoolean(false);
-        ReadOptions readOptions = new ReadOptions();
-        private RocksIterator iter;
-        protected String keyPrefix;
-        protected Class<? extends T> clazz;
-        protected boolean needKey;
-
-        public LocalIterator(String keyPrefix, Class<? extends T> clazz, boolean needKey) {
-            readOptions.setPrefixSameAsStart(true).setTotalOrderSeek(true);
-            iter = rocksDB.newIterator(readOptions);
-            this.keyPrefix = keyPrefix;
-            this.clazz = clazz;
-            this.needKey = needKey;
-        }
-
-        @Override
-        public boolean hasNext() {
-            if (hasInit.compareAndSet(false, true)) {
-                iter.seek(keyPrefix.getBytes());
-            }
-            return iter.isValid() && hasNext;
-        }
-
-        @Override
-        public T next() {
-            String key = new String(iter.key());
-            if (!key.startsWith(keyPrefix)) {
-                hasNext = false;
-                return null;
-            }
-            T windowBaseValue = (T) SerializeUtil.deserialize(iter.value());
-//            T windowBaseValue = ReflectUtil.forInstance(clazz);
-//            windowBaseValue.toObject(value);
-            if (needKey) {
-                windowBaseValue.setMsgKey(key);
-            }
-            while (windowBaseValue.getPartitionNum() < this.partitionNum) {
-                iter.next();
-                windowBaseValue = next();
-                if (windowBaseValue == null) {
-                    hasNext = false;
-                    return null;
+    private String createKey(String shuffleId, String windowInstanceId, WindowType windowType, WindowJoinType joinType, WindowBaseValue baseValue) {
+        String result;
+        switch (windowType) {
+            case SESSION_WINDOW:
+            case NORMAL_WINDOW: {
+                result = super.merge(DataType.WINDOW_BASE_VALUE.getValue(), shuffleId, windowInstanceId, windowType.name());
+                if (baseValue != null) {
+                    result = super.merge(result, baseValue.getMsgKey());
                 }
+
+                break;
             }
-            iter.next();
-            return windowBaseValue;
+            case JOIN_WINDOW: {
+                result = super.merge(DataType.WINDOW_BASE_VALUE.getValue(), shuffleId, windowInstanceId, windowType.name(), joinType.name());
+
+                if (baseValue != null) {
+                    JoinState joinState = (JoinState) baseValue;
+                    result = super.merge(result, joinState.getMessageId());
+                }
+
+                break;
+            }
+            default:
+                throw new RuntimeException("windowType " + windowType + "illegal.");
         }
 
+        return result;
     }
 
-    /**
-     * 把key转化成byte
-     *
-     * @param key
-     * @return
-     */
-    protected byte[] getKeyBytes(String key) {
+    @Override
+    public String getMaxOffset(String shuffleId, String windowConfigureName, String oriQueueId) {
+        String key = super.merge(DataType.MAX_OFFSET.getValue(), shuffleId, windowConfigureName, oriQueueId);
+
         try {
-            if (StringUtil.isEmpty(key)) {
+            byte[] bytes = rocksDB.get(key.getBytes(StandardCharsets.UTF_8));
+            if (bytes == null) {
                 return null;
             }
-            return key.getBytes(UTF8);
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException("get bytes error ", e);
+
+            String temp = new String(bytes, StandardCharsets.UTF_8);
+
+            List<String> split = super.split(temp);
+
+            return split.get(1);
+        } catch (Exception e) {
+            throw new RuntimeException("getMaxOffset from rocksdb error", e);
         }
     }
 
-    /**
-     * 把byte转化成值
-     *
-     * @param bytes
-     * @return
-     */
-    protected static String getValueFromByte(byte[] bytes) {
+    @Override
+    public void putMaxOffset(String shuffleId, String windowConfigureName, String oriQueueId, String offset) {
+        String key = super.merge(DataType.MAX_OFFSET.getValue(), shuffleId, windowConfigureName, oriQueueId);
+
         try {
-            return new String(bytes, UTF8);
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException(e);
+            byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
+
+            String mergeOffset = super.merge(getCurrentTimestamp(), offset);
+            byte[] offsetBytes = mergeOffset.getBytes(StandardCharsets.UTF_8);
+            rocksDB.put(writeOptions, keyBytes, offsetBytes);
+        } catch (Exception e) {
+            throw new RuntimeException("put data to rocksdb error", e);
         }
     }
 
-    public static void main(String[] args) {
-        String x = "2012-01-03 00:03:09";
-        System.out.println(x.substring(x.length() - 2, x.length()));
+    @Override
+    public void deleteMaxOffset(String shuffleId, String windowConfigureName, String oriQueueId) {
+        String key = super.merge(DataType.MAX_OFFSET.getValue(), shuffleId, windowConfigureName, oriQueueId);
+
+        try {
+            byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
+            rocksDB.delete(writeOptions, keyBytes);
+        } catch (Exception e) {
+            throw new RuntimeException("deleteMaxOffset from rocksdb error", e);
+        }
     }
+
+    @Override
+    public void putMaxPartitionNum(String shuffleId, String windowInstanceId, long maxPartitionNum) {
+        String key = super.merge(DataType.MAX_PARTITION_NUM.getValue(), shuffleId, windowInstanceId);
+
+        try {
+            byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
+
+            String mergeMaxPartitionNum = super.merge(getCurrentTimestamp(), String.valueOf(maxPartitionNum));
+
+            byte[] bytes = mergeMaxPartitionNum.getBytes(StandardCharsets.UTF_8);
+            rocksDB.put(writeOptions, keyBytes, bytes);
+        } catch (Exception e) {
+            throw new RuntimeException("put data to rocksdb error", e);
+        }
+    }
+
+    @Override
+    public Long getMaxPartitionNum(String shuffleId, String windowInstanceId) {
+        String key = super.merge(DataType.MAX_PARTITION_NUM.getValue(), shuffleId, windowInstanceId);
+
+        try {
+            byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
+            byte[] bytes = rocksDB.get(keyBytes);
+            if (bytes == null || bytes.length == 0) {
+                return null;
+            }
+
+            String temp = new String(bytes, StandardCharsets.UTF_8);
+            List<String> list = super.split(temp);
+
+            return Long.parseLong(list.get(1));
+        } catch (Exception e) {
+            throw new RuntimeException("get data from rocksdb error", e);
+        }
+    }
+
+    @Override
+    public void deleteMaxPartitionNum(String shuffleId, String windowInstanceId) {
+        String key = super.merge(DataType.MAX_PARTITION_NUM.getValue(), shuffleId, windowInstanceId);
+
+        try {
+            byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
+            rocksDB.delete(writeOptions, keyBytes);
+        } catch (Exception e) {
+            throw new RuntimeException("deleteMaxPartitionNum from rocksdb error", e);
+        }
+    }
+
+    public void delete(String key) {
+        if (key == null) {
+            return;
+        }
+
+        try {
+            byte[] bytes = key.getBytes(StandardCharsets.UTF_8);
+            rocksDB.delete(writeOptions, bytes);
+        } catch (Exception e) {
+            throw new RuntimeException();
+        }
+    }
+
+    public byte[] get(String key) {
+        if (key == null) {
+            return null;
+        }
+
+        try {
+            byte[] bytes = key.getBytes(StandardCharsets.UTF_8);
+            return rocksDB.get(bytes);
+        } catch (Exception e) {
+            throw new RuntimeException();
+        }
+    }
+
+    public void put(String key, byte[] value) {
+        if (key == null) {
+            return;
+        }
+
+        byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
+        try {
+            rocksDB.put(writeOptions, keyBytes, value);
+        } catch (Exception e) {
+            throw new RuntimeException();
+        }
+    }
+
+
+    @Override
+    public void clearCache(String queueId) {
+        String keyPrefix;
+        //删除windowInstance缓存
+        for (DataType type : DataType.values()) {
+            keyPrefix = super.merge(type.getValue(), queueId);
+            deleteByKeyPrefix(keyPrefix);
+        }
+    }
+
+    private void deleteByKeyPrefix(String keyPrefix) {
+        RocksdbIterator<Object> data = new RocksdbIterator<>(keyPrefix, rocksDB);
+
+        while (data.hasNext()) {
+            IteratorWrap<Object> iteratorWrap = data.next();
+            String key = iteratorWrap.getKey();
+            try {
+                rocksDB.delete(writeOptions, key.getBytes(StandardCharsets.UTF_8));
+            } catch (Throwable t) {
+                throw new RuntimeException();
+            }
+        }
+    }
+
+    public <T> RocksdbIterator<T> getData(String queueId, DataType type) {
+        String keyPrefix = super.merge(type.getValue(), queueId);
+
+        return new RocksdbIterator<>(keyPrefix, rocksDB);
+    }
+
+
 }
diff --git a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/trigger/WindowTrigger.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/trigger/WindowTrigger.java
index a872f3a..b012b17 100644
--- a/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/trigger/WindowTrigger.java
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/trigger/WindowTrigger.java
@@ -150,7 +150,7 @@
             return;
         }
         synchronized (this) {
-            if (this.firingWindowInstances.containsKey(windowInstance.createWindowInstanceTriggerId())) {
+            if (this.firingWindowInstances.containsKey(triggerId)) {
                 return;
             }
             this.orderWindowInstancs.offer(windowInstance);
@@ -206,7 +206,7 @@
             if (windowInstance.getLastMaxUpdateTime() == null) {
                 windowInstance.setLastMaxUpdateTime(window.getMaxEventTime(windowInstance.getSplitId()));
             }
-            int fireCount = window.fireWindowInstance(windowInstance, null);
+            int fireCount = window.fireWindowInstance(windowInstance);
             LOG.debug("fire instance(" + windowInstanceTriggerId + " fire count is " + fireCount);
             firingWindowInstances.remove(windowInstanceTriggerId);
         } catch (Exception e) {
diff --git a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/storage/RocksdbTest.java b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/storage/RocksdbTest.java
index 54091fb..40af253 100644
--- a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/storage/RocksdbTest.java
+++ b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/storage/RocksdbTest.java
@@ -1,206 +1,204 @@
-/*
- * 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.streams.storage;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.window.model.WindowInstance;
-import org.apache.rocketmq.streams.window.operator.impl.SessionOperator;
-import org.apache.rocketmq.streams.window.state.WindowBaseValue;
-import org.apache.rocketmq.streams.window.state.impl.WindowValue;
-import org.apache.rocketmq.streams.window.storage.WindowStorage;
-import org.apache.rocketmq.streams.window.storage.rocksdb.RocksdbStorage;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class RocksdbTest {
-
-    private static RocksdbStorage storage = new RocksdbStorage<>();
-
-    @Test
-    public void testMultiProcess() {
-        //
-        RocksdbStorage storage1 = new RocksdbStorage();
-        RocksdbStorage storage2 = new RocksdbStorage();
-        //
-        //
-        WindowBaseValue value1 = new WindowBaseValue();
-        value1.setStartTime("2021-09-07 11:00:00");
-        value1.setEndTime("2021-09-07 11:10:00");
-        value1.setFireTime("2021-09-07 11:11:00");
-        WindowBaseValue value2 = new WindowBaseValue();
-        value2.setStartTime("2021-09-07 12:00:00");
-        value2.setEndTime("2021-09-07 12:10:00");
-        value2.setFireTime("2021-09-07 12:11:00");
-        //
-        storage1.put("storage_1", value1);
-        storage2.put("storage_2", value2);
-        //
-        RocksdbStorage storage3 = new RocksdbStorage();
-        Map<String, WindowBaseValue> valueMap = storage3.multiGet(WindowBaseValue.class, new ArrayList<String>() {{
-            add("storage_1");
-            add("storage_2");
-        }});
-        Assert.assertEquals(2, valueMap.size());
-    }
-
-    @Test
-    public void testMultiValues() {
-        //
-        List<WindowBaseValue> valueList = new ArrayList<>();
-        WindowBaseValue value1 = new WindowBaseValue();
-        value1.setStartTime("2021-09-07 11:00:00");
-        value1.setEndTime("2021-09-07 11:10:00");
-        value1.setFireTime("2021-09-07 11:11:00");
-        WindowBaseValue value2 = new WindowBaseValue();
-        value2.setStartTime("2021-09-07 12:00:00");
-        value2.setEndTime("2021-09-07 12:10:00");
-        value2.setFireTime("2021-09-07 12:11:00");
-        valueList.add(value1);
-        valueList.add(value2);
-        //
-        String key = "test";
-        Map<String, List<WindowBaseValue>> theMap = new HashMap<>();
-        theMap.put(key, valueList);
-        storage.multiPutList(theMap);
-        Map<String, List<WindowBaseValue>> resultMap = storage.multiGetList(WindowBaseValue.class, new ArrayList<String>() {{
-            add(key);
-        }});
-        Assert.assertEquals(1, resultMap.size());
-        Assert.assertEquals(2, resultMap.get(key).size());
-        Assert.assertEquals("2021-09-07 11:00:00", resultMap.get(key).get(0).getStartTime());
-        Assert.assertEquals("2021-09-07 12:00:00", resultMap.get(key).get(1).getStartTime());
-        //
-        storage.removeKeys(theMap.keySet());
-    }
-
-    @Test
-    public void testOverwrite() {
-        //
-        List<WindowBaseValue> valueList = new ArrayList<>();
-        WindowBaseValue value1 = new WindowBaseValue();
-        value1.setStartTime("2021-09-07 11:00:00");
-        value1.setEndTime("2021-09-07 11:10:00");
-        value1.setFireTime("2021-09-07 11:11:00");
-        WindowBaseValue value2 = new WindowBaseValue();
-        value2.setStartTime("2021-09-07 12:00:00");
-        value2.setEndTime("2021-09-07 12:10:00");
-        value2.setFireTime("2021-09-07 12:11:00");
-        valueList.add(value1);
-        valueList.add(value2);
-        //
-        String key = "test";
-        Map<String, List<WindowBaseValue>> theMap = new HashMap<>();
-        theMap.put(key, valueList);
-        storage.multiPutList(theMap);
-        //
-        valueList = new ArrayList<>();
-        value1 = new WindowBaseValue();
-        value1.setStartTime("2021-09-11 11:00:00");
-        value1.setEndTime("2021-09-11 11:10:00");
-        value1.setFireTime("2021-09-11 11:11:00");
-        valueList.add(value1);
-        theMap.put(key, valueList);
-        storage.multiPutList(theMap);
-        Map<String, List<WindowBaseValue>> resultMap = storage.multiGetList(WindowBaseValue.class, new ArrayList<String>() {{
-            add(key);
-        }});
-        Assert.assertEquals(1, resultMap.size());
-        Assert.assertEquals("2021-09-11 11:00:00", resultMap.get(key).get(0).getStartTime());
-        //
-        storage.removeKeys(resultMap.keySet());
-    }
-
-    @Test
-    public void testValueWithPrefix() {
-        //
-        WindowInstance windowInstance = new WindowInstance();
-        windowInstance.setStartTime(SessionOperator.SESSION_WINDOW_BEGIN_TIME);
-        windowInstance.setEndTime(SessionOperator.SESSION_WINDOW_END_TIME);
-        windowInstance.setFireTime("2021-09-07 12:00:00");
-        //
-        Map<String, WindowValue> valueMap = new HashMap<>();
-        WindowValue value1 = new WindowValue();
-        value1.setStartTime("2021-09-07 11:00:00");
-        value1.setEndTime("2021-09-07 11:10:00");
-        value1.setFireTime("2021-09-07 11:11:00");
-        value1.setPartitionNum(100001);
-        WindowValue value2 = new WindowValue();
-        value2.setStartTime("2021-09-07 12:00:00");
-        value2.setEndTime("2021-09-07 12:10:00");
-        value2.setFireTime("2021-09-07 12:11:00");
-        value2.setPartitionNum(100002);
-        WindowValue value3 = new WindowValue();
-        value3.setStartTime("2021-09-07 11:10:00");
-        value3.setEndTime("2021-09-07 11:20:00");
-        value3.setFireTime("2021-09-07 11:25:00");
-        value3.setPartitionNum(100003);
-        //
-        String prefix = "sorted_session_window_key";
-        String queueId = "001";
-        String groupByValue = "default";
-        String localPrefix = prefix + queueId;
-        String sortKey1 = MapKeyUtil.createKey(localPrefix, windowInstance.createWindowInstanceId(), value1.getFireTime(), String.valueOf(value1.getPartitionNum()), groupByValue);
-        String sortKey2 = MapKeyUtil.createKey(localPrefix, windowInstance.createWindowInstanceId(), value2.getFireTime(), String.valueOf(value2.getPartitionNum()), groupByValue);
-        String sortKey3 = MapKeyUtil.createKey(localPrefix, windowInstance.createWindowInstanceId(), value3.getFireTime(), String.valueOf(value3.getPartitionNum()), groupByValue);
-        valueMap.put(sortKey1, value1);
-        valueMap.put(sortKey2, value2);
-        valueMap.put(sortKey3, value3);
-        storage.multiPut(valueMap);
-        //
-        WindowStorage.WindowBaseValueIterator<WindowValue> iterator = storage.loadWindowInstanceSplitData(prefix, queueId, windowInstance.createWindowInstanceId(), null, WindowValue.class);
-        List<WindowValue> valueList = new ArrayList<>();
-        while (iterator.hasNext()) {
-            WindowValue value = iterator.next();
-            valueList.add(value);
-        }
-        Assert.assertEquals(3, valueList.size());
-        Assert.assertEquals("2021-09-07 11:25:00", valueList.get(1).getFireTime());
-        //
-        List<WindowValue> sortList = new ArrayList<>(valueMap.values());
-        Collections.sort(sortList, Comparator.comparing(WindowValue::getStartTime));
-        for (WindowValue value : sortList) {
-            System.out.println(value.getStartTime() + " " + value.getEndTime() + " " + value.getFireTime());
-        }
-        //
-        WindowValue value4 = new WindowValue();
-        value4.setStartTime("2021-09-07 11:10:00");
-        value4.setEndTime("2021-09-07 11:21:00");
-        value4.setFireTime("2021-09-07 11:25:00");
-        value4.setPartitionNum(100003);
-        String sortKey4 = MapKeyUtil.createKey(localPrefix, windowInstance.createWindowInstanceId(), value4.getFireTime(), String.valueOf(value4.getPartitionNum()), groupByValue);
-        valueMap.put(sortKey4, value4);
-        storage.multiPut(valueMap);
-        iterator = storage.loadWindowInstanceSplitData(prefix, queueId, windowInstance.createWindowInstanceId(), null, WindowValue.class);
-        valueList = new ArrayList<>();
-        while (iterator.hasNext()) {
-            WindowValue value = iterator.next();
-            valueList.add(value);
-        }
-        for (WindowValue value : valueList) {
-            System.out.println(value.getStartTime() + " " + value.getEndTime() + " " + value.getFireTime() + " " + value.getPartitionNum());
-        }
-        //
-        storage.removeKeys(valueMap.keySet());
-    }
-
-}
+///*
+// * 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.streams.storage;
+//
+//import java.util.ArrayList;
+//import java.util.Collections;
+//import java.util.Comparator;
+//import java.util.HashMap;
+//import java.util.List;
+//import java.util.Map;
+//import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
+//import org.apache.rocketmq.streams.window.model.WindowInstance;
+//import org.apache.rocketmq.streams.window.operator.impl.SessionOperator;
+//import org.apache.rocketmq.streams.window.state.WindowBaseValue;
+//import org.apache.rocketmq.streams.window.state.impl.WindowValue;
+//import org.junit.Assert;
+//import org.junit.Test;
+//
+//public class RocksdbTest {
+//
+//    private static RocksdbStorage storage = new RocksdbStorage<>();
+//
+//    @Test
+//    public void testMultiProcess() {
+//        //
+//        RocksdbStorage storage1 = new RocksdbStorage();
+//        RocksdbStorage storage2 = new RocksdbStorage();
+//        //
+//        //
+//        WindowBaseValue value1 = new WindowBaseValue();
+//        value1.setStartTime("2021-09-07 11:00:00");
+//        value1.setEndTime("2021-09-07 11:10:00");
+//        value1.setFireTime("2021-09-07 11:11:00");
+//        WindowBaseValue value2 = new WindowBaseValue();
+//        value2.setStartTime("2021-09-07 12:00:00");
+//        value2.setEndTime("2021-09-07 12:10:00");
+//        value2.setFireTime("2021-09-07 12:11:00");
+//        //
+//        storage1.put("storage_1", value1);
+//        storage2.put("storage_2", value2);
+//        //
+//        RocksdbStorage storage3 = new RocksdbStorage();
+//        Map<String, WindowBaseValue> valueMap = storage3.multiGet(WindowBaseValue.class, new ArrayList<String>() {{
+//            add("storage_1");
+//            add("storage_2");
+//        }});
+//        Assert.assertEquals(2, valueMap.size());
+//    }
+//
+//    @Test
+//    public void testMultiValues() {
+//        //
+//        List<WindowBaseValue> valueList = new ArrayList<>();
+//        WindowBaseValue value1 = new WindowBaseValue();
+//        value1.setStartTime("2021-09-07 11:00:00");
+//        value1.setEndTime("2021-09-07 11:10:00");
+//        value1.setFireTime("2021-09-07 11:11:00");
+//        WindowBaseValue value2 = new WindowBaseValue();
+//        value2.setStartTime("2021-09-07 12:00:00");
+//        value2.setEndTime("2021-09-07 12:10:00");
+//        value2.setFireTime("2021-09-07 12:11:00");
+//        valueList.add(value1);
+//        valueList.add(value2);
+//        //
+//        String key = "test";
+//        Map<String, List<WindowBaseValue>> theMap = new HashMap<>();
+//        theMap.put(key, valueList);
+//        storage.multiPutList(theMap);
+//        Map<String, List<WindowBaseValue>> resultMap = storage.multiGetList(WindowBaseValue.class, new ArrayList<String>() {{
+//            add(key);
+//        }});
+//        Assert.assertEquals(1, resultMap.size());
+//        Assert.assertEquals(2, resultMap.get(key).size());
+//        Assert.assertEquals("2021-09-07 11:00:00", resultMap.get(key).get(0).getStartTime());
+//        Assert.assertEquals("2021-09-07 12:00:00", resultMap.get(key).get(1).getStartTime());
+//        //
+//        storage.removeKeys(theMap.keySet());
+//    }
+//
+//    @Test
+//    public void testOverwrite() {
+//        //
+//        List<WindowBaseValue> valueList = new ArrayList<>();
+//        WindowBaseValue value1 = new WindowBaseValue();
+//        value1.setStartTime("2021-09-07 11:00:00");
+//        value1.setEndTime("2021-09-07 11:10:00");
+//        value1.setFireTime("2021-09-07 11:11:00");
+//        WindowBaseValue value2 = new WindowBaseValue();
+//        value2.setStartTime("2021-09-07 12:00:00");
+//        value2.setEndTime("2021-09-07 12:10:00");
+//        value2.setFireTime("2021-09-07 12:11:00");
+//        valueList.add(value1);
+//        valueList.add(value2);
+//        //
+//        String key = "test";
+//        Map<String, List<WindowBaseValue>> theMap = new HashMap<>();
+//        theMap.put(key, valueList);
+//        storage.multiPutList(theMap);
+//        //
+//        valueList = new ArrayList<>();
+//        value1 = new WindowBaseValue();
+//        value1.setStartTime("2021-09-11 11:00:00");
+//        value1.setEndTime("2021-09-11 11:10:00");
+//        value1.setFireTime("2021-09-11 11:11:00");
+//        valueList.add(value1);
+//        theMap.put(key, valueList);
+//        storage.multiPutList(theMap);
+//        Map<String, List<WindowBaseValue>> resultMap = storage.multiGetList(WindowBaseValue.class, new ArrayList<String>() {{
+//            add(key);
+//        }});
+//        Assert.assertEquals(1, resultMap.size());
+//        Assert.assertEquals("2021-09-11 11:00:00", resultMap.get(key).get(0).getStartTime());
+//        //
+//        storage.removeKeys(resultMap.keySet());
+//    }
+//
+//    @Test
+//    public void testValueWithPrefix() {
+//        //
+//        WindowInstance windowInstance = new WindowInstance();
+//        windowInstance.setStartTime(SessionOperator.SESSION_WINDOW_BEGIN_TIME);
+//        windowInstance.setEndTime(SessionOperator.SESSION_WINDOW_END_TIME);
+//        windowInstance.setFireTime("2021-09-07 12:00:00");
+//        //
+//        Map<String, WindowValue> valueMap = new HashMap<>();
+//        WindowValue value1 = new WindowValue();
+//        value1.setStartTime("2021-09-07 11:00:00");
+//        value1.setEndTime("2021-09-07 11:10:00");
+//        value1.setFireTime("2021-09-07 11:11:00");
+//        value1.setPartitionNum(100001);
+//        WindowValue value2 = new WindowValue();
+//        value2.setStartTime("2021-09-07 12:00:00");
+//        value2.setEndTime("2021-09-07 12:10:00");
+//        value2.setFireTime("2021-09-07 12:11:00");
+//        value2.setPartitionNum(100002);
+//        WindowValue value3 = new WindowValue();
+//        value3.setStartTime("2021-09-07 11:10:00");
+//        value3.setEndTime("2021-09-07 11:20:00");
+//        value3.setFireTime("2021-09-07 11:25:00");
+//        value3.setPartitionNum(100003);
+//        //
+//        String prefix = "sorted_session_window_key";
+//        String queueId = "001";
+//        String groupByValue = "default";
+//        String localPrefix = prefix + queueId;
+//        String sortKey1 = MapKeyUtil.createKey(localPrefix, windowInstance.createWindowInstanceId(), value1.getFireTime(), String.valueOf(value1.getPartitionNum()), groupByValue);
+//        String sortKey2 = MapKeyUtil.createKey(localPrefix, windowInstance.createWindowInstanceId(), value2.getFireTime(), String.valueOf(value2.getPartitionNum()), groupByValue);
+//        String sortKey3 = MapKeyUtil.createKey(localPrefix, windowInstance.createWindowInstanceId(), value3.getFireTime(), String.valueOf(value3.getPartitionNum()), groupByValue);
+//        valueMap.put(sortKey1, value1);
+//        valueMap.put(sortKey2, value2);
+//        valueMap.put(sortKey3, value3);
+//        storage.multiPut(valueMap);
+//        //
+//        WindowStorage.WindowBaseValueIterator<WindowValue> iterator = storage.loadWindowInstanceSplitData(prefix, queueId, windowInstance.createWindowInstanceId(), null, WindowValue.class);
+//        List<WindowValue> valueList = new ArrayList<>();
+//        while (iterator.hasNext()) {
+//            WindowValue value = iterator.next();
+//            valueList.add(value);
+//        }
+//        Assert.assertEquals(3, valueList.size());
+//        Assert.assertEquals("2021-09-07 11:25:00", valueList.get(1).getFireTime());
+//        //
+//        List<WindowValue> sortList = new ArrayList<>(valueMap.values());
+//        Collections.sort(sortList, Comparator.comparing(WindowValue::getStartTime));
+//        for (WindowValue value : sortList) {
+//            System.out.println(value.getStartTime() + " " + value.getEndTime() + " " + value.getFireTime());
+//        }
+//        //
+//        WindowValue value4 = new WindowValue();
+//        value4.setStartTime("2021-09-07 11:10:00");
+//        value4.setEndTime("2021-09-07 11:21:00");
+//        value4.setFireTime("2021-09-07 11:25:00");
+//        value4.setPartitionNum(100003);
+//        String sortKey4 = MapKeyUtil.createKey(localPrefix, windowInstance.createWindowInstanceId(), value4.getFireTime(), String.valueOf(value4.getPartitionNum()), groupByValue);
+//        valueMap.put(sortKey4, value4);
+//        storage.multiPut(valueMap);
+//        iterator = storage.loadWindowInstanceSplitData(prefix, queueId, windowInstance.createWindowInstanceId(), null, WindowValue.class);
+//        valueList = new ArrayList<>();
+//        while (iterator.hasNext()) {
+//            WindowValue value = iterator.next();
+//            valueList.add(value);
+//        }
+//        for (WindowValue value : valueList) {
+//            System.out.println(value.getStartTime() + " " + value.getEndTime() + " " + value.getFireTime() + " " + value.getPartitionNum());
+//        }
+//        //
+//        storage.removeKeys(valueMap.keySet());
+//    }
+//
+//}
diff --git a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowInstanceTest.java b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowInstanceTest.java
index e427ce3..4a6e9d2 100644
--- a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowInstanceTest.java
+++ b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowInstanceTest.java
@@ -36,19 +36,6 @@
 
 public class WindowInstanceTest {
 
-    /**
-     * save window instance
-     */
-    @Test
-    public void testWindowInstanceSave() {
-        WindowOperator window = new WindowOperator();
-        window.setNameSpace("namespace_chris");
-        window.setConfigureName("name");
-        WindowInstance windowInstance = window.createWindowInstance("2021-07-09 11:00:00", "2021-07-09 11:10:00", "2021-07-09 11:10:00", "1");
-        ORMUtil.batchReplaceInto(windowInstance);
-        WindowInstance queryWindowInstance = ORMUtil.queryForObject("select * from window_instance where window_instance_key='" + windowInstance.getWindowInstanceKey() + "'", null, WindowInstance.class);
-        assertTrue(queryWindowInstance != null);
-    }
 
     @Test
     public void testWindowInstanceNormalMode() {
diff --git a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowMaxValueTest.java b/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowMaxValueTest.java
deleted file mode 100644
index 2b39bde..0000000
--- a/rocketmq-streams-window/src/test/java/org/apache/rocketmq/streams/window/WindowMaxValueTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.streams.window;
-
-import java.util.Date;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
-import org.apache.rocketmq.streams.window.offset.WindowMaxValue;
-import org.junit.Test;
-
-public class WindowMaxValueTest {
-
-    @Test
-    public void testSave() {
-        WindowMaxValue windowMaxValue = new WindowMaxValue();
-        windowMaxValue.setMsgKey("chris");
-        windowMaxValue.setMaxEventTime(new Date().getTime());
-        windowMaxValue.setMaxValue(1232L);
-        ORMUtil.batchReplaceInto(windowMaxValue);
-    }
-}