merge(main) merge from main
diff --git a/README.md b/README.md
index 860361a..c95e238 100644
--- a/README.md
+++ b/README.md
@@ -119,7 +119,7 @@
         <groupId>org.apache.rocketmq</groupId>
         <artifactId>rocketmq-streams-clients</artifactId>
           <!--Newest version-->
-        <version>1.0.2-preview-SNAPSHOT</version>
+        <version>${version}</version>
     </dependency>
 </dependencies>
 
diff --git a/pom.xml b/pom.xml
index 31ffbe5..05a6583 100644
--- a/pom.xml
+++ b/pom.xml
@@ -83,16 +83,12 @@
 
     <modules>
         <module>rocketmq-streams-commons</module>
-        <module>rocketmq-streams-dim</module>
         <module>rocketmq-streams-transport-minio</module>
         <module>rocketmq-streams-script</module>
         <module>rocketmq-streams-configurable</module>
         <module>rocketmq-streams-serviceloader</module>
         <module>rocketmq-streams-filter</module>
-        <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>
@@ -100,11 +96,8 @@
         <module>rocketmq-streams-channel-http</module>
         <module>rocketmq-streams-state</module>
         <module>rocketmq-streams-examples</module>
-        <module>rocketmq-streams-checkpoint</module>
-        <module>rocketmq-streams-connectors</module>
         <module>rocketmq-streams-channel-syslog</module>
         <module>rocketmq-streams-channel-es</module>
-        <module>rocketmq-streams-runner</module>
         <module>rocketmq-streams-channel-mqtt</module>
     </modules>
 
@@ -133,7 +126,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.4</rocketmq.version>
         <hyperscan.version>5.4.0-2.0.0</hyperscan.version>
         <platform.version>3.5.2</platform.version>
         <gson.version>2.8.9</gson.version>
@@ -189,6 +182,7 @@
                         <exclude>**/*.sql</exclude>
                         <exclude>**/*.properties</exclude>
                         <exclude>docs/**/*</exclude>
+                        <exclude>**/*.sql</exclude>
                     </excludes>
                 </configuration>
             </plugin>
@@ -278,11 +272,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>
@@ -326,16 +315,7 @@
                 <artifactId>rocketmq-streams-filter</artifactId>
                 <version>${project.version}</version>
             </dependency>
-            <dependency>
-                <groupId>org.apache.rocketmq</groupId>
-                <artifactId>rocketmq-streams-lease</artifactId>
-                <version>${project.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.rocketmq</groupId>
-                <artifactId>rocketmq-streams-schedule</artifactId>
-                <version>${project.version}</version>
-            </dependency>
+
             <dependency>
                 <groupId>org.apache.rocketmq</groupId>
                 <artifactId>rocketmq-streams-script</artifactId>
@@ -386,11 +366,7 @@
                 <artifactId>rocketmq-streams-channel-rocketmq</artifactId>
                 <version>${project.version}</version>
             </dependency>
-            <dependency>
-                <groupId>org.apache.rocketmq</groupId>
-                <artifactId>rocketmq-streams-connectors</artifactId>
-                <version>${project.version}</version>
-            </dependency>
+
             <dependency>
                 <groupId>org.apache.rocketmq</groupId>
                 <artifactId>rocketmq-streams-examples</artifactId>
@@ -422,6 +398,12 @@
                 <version>${rocketmq.version}</version>
             </dependency>
 
+            <dependency>
+                <groupId>org.apache.rocketmq</groupId>
+                <artifactId>rocketmq-tools</artifactId>
+                <version>${rocketmq.version}</version>
+            </dependency>
+
             <!-- ================================================= -->
             <!-- tool library -->
             <!-- ================================================= -->
@@ -509,12 +491,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/RocketMQOffset.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQOffset.java
deleted file mode 100644
index 6650744..0000000
--- a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/RocketMQOffset.java
+++ /dev/null
@@ -1,103 +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;
-
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.rocketmq.client.consumer.store.OffsetStore;
-import org.apache.rocketmq.client.consumer.store.ReadOffsetType;
-import org.apache.rocketmq.client.exception.MQBrokerException;
-import org.apache.rocketmq.client.exception.MQClientException;
-import org.apache.rocketmq.common.message.MessageQueue;
-import org.apache.rocketmq.remoting.exception.RemotingException;
-import org.apache.rocketmq.streams.common.channel.source.AbstractSupportShuffleSource;
-import org.apache.rocketmq.streams.common.utils.ReflectUtil;
-import org.apache.rocketmq.streams.debug.DebugWriter;
-import org.apache.rocketmq.streams.queue.RocketMQMessageQueue;
-
-public class RocketMQOffset implements OffsetStore {
-    protected OffsetStore offsetStore;
-    protected AbstractSupportShuffleSource source;
-    private AtomicBoolean starting;
-
-    public RocketMQOffset(OffsetStore offsetStore, AbstractSupportShuffleSource source) {
-        this.offsetStore = offsetStore;
-        this.source = source;
-        this.starting = new AtomicBoolean(true);
-    }
-
-    @Override
-    public void load() throws MQClientException {
-        offsetStore.load();
-    }
-
-    @Override
-    public void updateOffset(MessageQueue mq, long offset, boolean increaseOnly) {
-        offsetStore.updateOffset(mq, offset, increaseOnly);
-    }
-
-    @Override
-    public long readOffset(MessageQueue mq, ReadOffsetType type) {
-        return offsetStore.readOffset(mq, type);
-    }
-
-    @Override
-    public void persistAll(Set<MessageQueue> mqs) {
-        offsetStore.persistAll(mqs);
-    }
-
-    @Override
-    public void persist(MessageQueue mq) {
-        offsetStore.persist(mq);
-    }
-
-    @Override
-    public void removeOffset(MessageQueue mq) {
-        //todo 启动时第一次做rebalance时source中也没有原有消费mq,不做移除,做了会有副作用
-        //后续整个checkpoint机制都会调整成异步,整块代码都不会保留,目前为了整体跑通,不做修改。
-        if (starting.get()) {
-            starting.set(false);
-        } else {
-            Set<String> splitIds = new HashSet<>();
-            splitIds.add(new RocketMQMessageQueue(mq).getQueueId());
-            source.removeSplit(splitIds);
-        }
-
-        offsetStore.removeOffset(mq);
-    }
-
-    @Override
-    public Map<MessageQueue, Long> cloneOffsetTable(String topic) {
-        return offsetStore.cloneOffsetTable(topic);
-    }
-
-    @Override
-    public void updateConsumeOffsetToBroker(MessageQueue mq, long offset, boolean isOneway)
-        throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
-        source.sendCheckpoint(new RocketMQMessageQueue(mq).getQueueId());
-        if (DebugWriter.isOpenDebug()) {
-            ConcurrentMap<MessageQueue, AtomicLong> offsetTable = ReflectUtil.getDeclaredField(this.offsetStore, "offsetTable");
-            DebugWriter.getInstance(source.getTopic()).writeSaveOffset(mq, offsetTable.get(mq));
-        }
-        offsetStore.updateConsumeOffsetToBroker(mq, offset, isOneway);
-    }
-}
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..e693bf5 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,10 +27,12 @@
 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;
 import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.remoting.RPCHook;
 import org.apache.rocketmq.streams.common.channel.sink.AbstractSupportShuffleSink;
 import org.apache.rocketmq.streams.common.channel.split.ISplit;
 import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence;
@@ -56,6 +58,7 @@
 
     private Long pullIntervalMs;
     private String namesrvAddr;
+    private RPCHook rpcHook;
 
     public RocketMQSink() {
     }
@@ -146,7 +149,7 @@
             synchronized (this) {
                 if (producer == null) {
                     destroy();
-                    producer = new DefaultMQProducer(groupName + "producer", true, null);
+                    producer = new DefaultMQProducer(null, groupName + "producer", rpcHook,false, null);
                     try {
                         //please not use the code,the name srv addr may be empty in jmenv
 //                        if (this.namesrvAddr == null || "".equals(this.namesrvAddr)) {
@@ -201,6 +204,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 +247,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;
@@ -346,4 +348,12 @@
     public void setOrder(boolean order) {
         this.order = order;
     }
+
+    public RPCHook getRpcHook() {
+        return rpcHook;
+    }
+
+    public void setRpcHook(RPCHook rpcHook) {
+        this.rpcHook = rpcHook;
+    }
 }
diff --git a/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/source/MessageListenerDelegator.java b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/source/MessageListenerDelegator.java
new file mode 100644
index 0000000..38023af
--- /dev/null
+++ b/rocketmq-streams-channel-rocketmq/src/main/java/org/apache/rocketmq/streams/source/MessageListenerDelegator.java
@@ -0,0 +1,80 @@
+package org.apache.rocketmq.streams.source;
+/*
+ * 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.MessageQueueListener;
+import org.apache.rocketmq.common.message.MessageQueue;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+
+public class MessageListenerDelegator implements MessageQueueListener {
+    private final MessageQueueListener delegator;
+    private final Set<MessageQueue> lastDivided = new HashSet<>();
+    private final Set<MessageQueue> removingQueue = new HashSet<>();
+    private final AtomicBoolean needSync = new AtomicBoolean(false);
+    private final Object mutex = new Object();
+
+    public MessageListenerDelegator(MessageQueueListener delegator) {
+        this.delegator = delegator;
+    }
+
+
+    @Override
+    public void messageQueueChanged(String topic, Set<MessageQueue> mqAll, Set<MessageQueue> mqDivided) {
+        //上一次分配有,但是这一次没有,需要对这些mq进行状态移除
+        for (MessageQueue last : lastDivided) {
+            if (!mqDivided.contains(last)) {
+                removingQueue.add(last);
+            }
+        }
+
+        this.lastDivided.clear();
+        this.lastDivided.addAll(mqDivided);
+
+        needSync.set(true);
+        delegator.messageQueueChanged(topic, mqAll, mqDivided);
+
+        synchronized (this.mutex) {
+            this.mutex.notifyAll();
+        }
+    }
+
+    public Set<MessageQueue> getLastDivided() {
+        return Collections.unmodifiableSet(this.lastDivided);
+    }
+
+    public Set<MessageQueue> getRemovingQueue() {
+        return Collections.unmodifiableSet(this.removingQueue);
+    }
+
+
+    public boolean needSync() {
+        return this.needSync.get();
+    }
+
+    public void hasSynchronized() {
+        this.needSync.set(false);
+    }
+
+    public Object getMutex() {
+        return mutex;
+    }
+}
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..3a02abf 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
@@ -18,7 +18,9 @@
 package org.apache.rocketmq.streams.source;
 
 import com.alibaba.fastjson.JSONObject;
+
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -26,36 +28,27 @@
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.client.consumer.AllocateMessageQueueStrategy;
-import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
-import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyStatus;
-import org.apache.rocketmq.client.consumer.listener.MessageListenerOrderly;
-import org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueAveragely;
-import org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueByConfig;
-import org.apache.rocketmq.client.consumer.store.RemoteBrokerOffsetStore;
-import org.apache.rocketmq.client.exception.MQBrokerException;
+import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;
+import org.apache.rocketmq.client.consumer.MessageQueueListener;
 import org.apache.rocketmq.client.exception.MQClientException;
-import org.apache.rocketmq.client.impl.MQClientManager;
-import org.apache.rocketmq.client.impl.consumer.DefaultMQPushConsumerImpl;
-import org.apache.rocketmq.client.impl.factory.MQClientInstance;
 import org.apache.rocketmq.common.consumer.ConsumeFromWhere;
 import org.apache.rocketmq.common.message.MessageExt;
 import org.apache.rocketmq.common.message.MessageQueue;
-import org.apache.rocketmq.common.protocol.NamespaceUtil;
 import org.apache.rocketmq.common.protocol.body.Connection;
 import org.apache.rocketmq.common.protocol.body.ConsumerConnection;
 import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo;
-import org.apache.rocketmq.common.protocol.heartbeat.MessageModel;
-import org.apache.rocketmq.remoting.exception.RemotingException;
+import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
+import org.apache.rocketmq.remoting.RPCHook;
 import org.apache.rocketmq.streams.common.channel.source.AbstractSupportShuffleSource;
 import org.apache.rocketmq.streams.common.channel.split.ISplit;
 import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence;
-import org.apache.rocketmq.streams.common.utils.ReflectUtil;
-import org.apache.rocketmq.streams.debug.DebugWriter;
 import org.apache.rocketmq.streams.queue.RocketMQMessageQueue;
 import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
 
@@ -64,24 +57,19 @@
     protected static final Log LOG = LogFactory.getLog(RocketMQSource.class);
 
     private static final String STRATEGY_AVERAGE = "average";
-    private static final String STRATEGY_MACHINE = "machine";
-    private static final String STRATEGY_CONFIG = "config";
 
     @ENVDependence
-    protected String tags = "*";
+    private String tags = SubscriptionData.SUB_ALL;
 
-    /**
-     * 消息队列命名空间接入点
-     */
-    protected String namesrvAddr;
-
-    protected Long pullIntervalMs;
-
-    protected String strategyName;
-
-    protected transient DefaultMQPushConsumer consumer;
-    protected transient ConsumeFromWhere consumeFromWhere;//默认从哪里消费,不会被持久化。不设置默认从尾部消费
-    protected transient String consumerOffset;//从哪里开始消费
+    private int userPullThreadNum = 1;
+    private long pullTimeout;
+    private long commitInternalMs = 1000;
+    private String strategyName;
+    private transient ConsumeFromWhere consumeFromWhere = ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET;//默认从哪里消费,不会被持久化。不设置默认从尾部消费
+    private RPCHook rpcHook;
+    private transient DefaultLitePullConsumer pullConsumer;
+    private transient ExecutorService executorService;
+    private transient PullTask[] pullTasks;
 
     public RocketMQSource() {
     }
@@ -107,109 +95,56 @@
     protected boolean startSource() {
         try {
             destroyConsumer();
-            consumer = startConsumer();
+
+            this.pullConsumer = buildPullConsumer(topic, groupName, namesrvAddr, tags, rpcHook, consumeFromWhere);
+
+            if (this.executorService == null) {
+                this.executorService = new ThreadPoolExecutor(userPullThreadNum, userPullThreadNum, 0, TimeUnit.MILLISECONDS,
+                        new ArrayBlockingQueue<>(1000), r -> new Thread(r, "RStream-poll-thread"));
+            }
+
+            pullTasks = new PullTask[userPullThreadNum];
+            for (int i = 0; i < userPullThreadNum; i++) {
+                pullTasks[i] = new PullTask(this.pullConsumer, pullTimeout, commitInternalMs);
+                this.executorService.execute(pullTasks[i]);
+            }
+
+            this.pullConsumer.start();
+
             return true;
-        } catch (Exception e) {
+        } catch (MQClientException e) {
             setInitSuccess(false);
-            e.printStackTrace();
             throw new RuntimeException("start rocketmq channel error " + topic, e);
         }
     }
 
-    protected DefaultMQPushConsumer startConsumer() {
-        try {
-            DefaultMQPushConsumer consumer = new DefaultMQPushConsumer(groupName);
-            if (pullIntervalMs != null) {
-                consumer.setPullInterval(pullIntervalMs);
-            }
-            AllocateMessageQueueStrategy defaultStrategy = new AllocateMessageQueueAveragely();
-            if (STRATEGY_AVERAGE.equalsIgnoreCase(this.strategyName)) {
-                consumer.setAllocateMessageQueueStrategy(defaultStrategy);
-            } else if (STRATEGY_MACHINE.equalsIgnoreCase(this.strategyName)) {
-                //consumer.setAllocateMessageQueueStrategy(new AllocateMessageQueueByMachine(defaultStrategy));
-            } else if (STRATEGY_CONFIG.equalsIgnoreCase(this.strategyName)) {
-                consumer.setAllocateMessageQueueStrategy(new AllocateMessageQueueByConfig());
-            }
+    private DefaultLitePullConsumer buildPullConsumer(String topic, String groupName, String namesrv, String tags,
+                                                      RPCHook rpcHook, ConsumeFromWhere consumeFromWhere) throws MQClientException {
+        DefaultLitePullConsumer pullConsumer = new DefaultLitePullConsumer(groupName, rpcHook);
+        pullConsumer.setNamesrvAddr(namesrv);
+        pullConsumer.setConsumeFromWhere(consumeFromWhere);
+        pullConsumer.subscribe(topic, tags);
+        pullConsumer.setAutoCommit(false);
+        pullConsumer.setPullBatchSize(1000);
 
-            consumer.setPersistConsumerOffsetInterval((int) this.checkpointTime);
-            consumer.setConsumeMessageBatchMaxSize(maxFetchLogGroupSize);
-            consumer.setNamesrvAddr(this.namesrvAddr);
-            if (consumeFromWhere != null) {
-                consumer.setConsumeFromWhere(consumeFromWhere);
-                if (consumerOffset != null) {
-                    consumer.setConsumeTimestamp(consumerOffset);
-                }
-            }
-            Map<String, Boolean> isFirstDataForQueue = new HashMap<>();
-            //consumer.setCommitOffsetWithPullRequestEnable(false);
-            consumer.subscribe(topic, tags);
-            consumer.registerMessageListener((MessageListenerOrderly) (msgs, context) -> {
-                try {
-                    int i = 0;
-                    for (MessageExt msg : msgs) {
+        MessageQueueListener origin = pullConsumer.getMessageQueueListener();
 
-                        JSONObject jsonObject = create(msg.getBody(), msg.getProperties());
+        MessageListenerDelegator delegator = new MessageListenerDelegator(origin);
 
-                        String queueId = RocketMQMessageQueue.getQueueId(context.getMessageQueue());
-                        String offset = msg.getQueueOffset() + "";
-                        org.apache.rocketmq.streams.common.context.Message message = createMessage(jsonObject, queueId, offset, false);
-                        message.getHeader().setOffsetIsLong(true);
+        pullConsumer.setMessageQueueListener(delegator);
 
-                        if (DebugWriter.isOpenDebug()) {
-                            Boolean isFirstData = isFirstDataForQueue.get(queueId);
-                            if (isFirstData == null) {
-                                synchronized (this) {
-                                    isFirstData = isFirstDataForQueue.get(queueId);
-                                    if (isFirstData == null) {
-                                        isFirstDataForQueue.put(queueId, true);
-                                    }
-                                    DebugWriter.getInstance(getTopic()).receiveFirstData(queueId, msg.getQueueOffset());
-                                }
-                            }
-                        }
-
-                        if (i == msgs.size() - 1) {
-                            message.getHeader().setNeedFlush(true);
-                        }
-                        executeMessage(message);
-                        i++;
-                    }
-                } catch (Exception e) {
-
-                    LOG.error("consume message from rocketmq error " + e, e);
-                    e.printStackTrace();
-
-                }
-
-                return ConsumeOrderlyStatus.SUCCESS;// 返回消费成功
-            });
-
-            setOffsetStore(consumer);
-            consumer.start();
-
-            return consumer;
-        } catch (Exception e) {
-            setInitSuccess(false);
-            e.printStackTrace();
-            throw new RuntimeException("start metaq channel error " + topic, e);
-        }
+        return pullConsumer;
     }
 
+
     @Override
     public List<ISplit> getAllSplits() {
         try {
             List<ISplit> messageQueues = new ArrayList<>();
-            if (messageQueues == null || messageQueues.size() == 0) {
-                Set<MessageQueue> metaqQueueSet = consumer.fetchSubscribeMessageQueues(this.topic);
-                for (MessageQueue queue : metaqQueueSet) {
-                    RocketMQMessageQueue metaqMessageQueue = new RocketMQMessageQueue(queue);
-                    if (isNotDataSplit(metaqMessageQueue.getQueueId())) {
-                        continue;
-                    }
-
-                    messageQueues.add(metaqMessageQueue);
-
-                }
+            Collection<MessageQueue> metaqQueueSet = this.pullConsumer.fetchMessageQueues(this.topic);
+            for (MessageQueue queue : metaqQueueSet) {
+                RocketMQMessageQueue metaqMessageQueue = new RocketMQMessageQueue(queue);
+                messageQueues.add(metaqMessageQueue);
             }
             return messageQueues;
         } catch (MQClientException e) {
@@ -218,12 +153,13 @@
         }
     }
 
+    //todo 计算正在工作的分片?
     @Override
     public Map<String, List<ISplit>> getWorkingSplitsGroupByInstances() {
         DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt();
         defaultMQAdminExt.setVipChannelEnabled(false);
         defaultMQAdminExt.setAdminExtGroup(UUID.randomUUID().toString());
-        defaultMQAdminExt.setInstanceName(this.consumer.getInstanceName());
+        defaultMQAdminExt.setInstanceName(this.pullConsumer.getInstanceName());
         try {
             defaultMQAdminExt.start();
             Map<MessageQueue, String> queue2Instances = getMessageQueueAllocationResult(defaultMQAdminExt, this.groupName);
@@ -234,11 +170,7 @@
                     continue;
                 }
                 String instanceName = queue2Instances.get(messageQueue);
-                List<ISplit> splits = instanceOwnerQueues.get(instanceName);
-                if (splits == null) {
-                    splits = new ArrayList<>();
-                    instanceOwnerQueues.put(instanceName, splits);
-                }
+                List<ISplit> splits = instanceOwnerQueues.computeIfAbsent(instanceName, k -> new ArrayList<>());
                 splits.add(metaqMessageQueue);
             }
             return instanceOwnerQueues;
@@ -250,9 +182,9 @@
         }
     }
 
-    protected Map<MessageQueue, String> getMessageQueueAllocationResult(DefaultMQAdminExt defaultMQAdminExt,
-        String groupName) {
-        HashMap results = new HashMap();
+    private Map<MessageQueue, String> getMessageQueueAllocationResult(DefaultMQAdminExt defaultMQAdminExt,
+                                                                      String groupName) {
+        HashMap<MessageQueue, String> results = new HashMap<>();
 
         try {
             ConsumerConnection consumerConnection = defaultMQAdminExt.examineConsumerConnectionInfo(groupName);
@@ -276,49 +208,9 @@
         return results;
     }
 
-    /**
-     * 设置offset存储,包装原有的RemoteBrokerOffsetStore,在保存offset前发送系统消息 this method suggest to be removed, use check barrier to achieve checkpoint asynchronous.
-     *
-     * @param consumer
-     */
-    protected void setOffsetStore(DefaultMQPushConsumer consumer) {
-        DefaultMQPushConsumerImpl defaultMQPushConsumer = consumer.getDefaultMQPushConsumerImpl();
-        if (consumer.getMessageModel() == MessageModel.CLUSTERING) {
-            consumer.changeInstanceNameToPID();
-        }
-        MQClientInstance mQClientFactory = MQClientManager.getInstance().getAndCreateMQClientInstance(defaultMQPushConsumer.getDefaultMQPushConsumer());
-        RemoteBrokerOffsetStore offsetStore = new RemoteBrokerOffsetStore(mQClientFactory, NamespaceUtil.wrapNamespace(consumer.getNamespace(), consumer.getConsumerGroup())) {
-            Set<MessageQueue> firstComing = new HashSet<>();
-            @Override
-            public void removeOffset(MessageQueue mq) {
-                if (!firstComing.contains(mq)){
-                    firstComing.add(mq);
-                } else {
-                    Set<String> splitIds = new HashSet<>();
-                    splitIds.add(new RocketMQMessageQueue(mq).getQueueId());
-                    removeSplit(splitIds);
-                }
-                super.removeOffset(mq);
-            }
-
-            @Override
-            public void updateConsumeOffsetToBroker(MessageQueue mq, long offset,
-                boolean isOneway) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
-                sendCheckpoint(new RocketMQMessageQueue(mq).getQueueId());
-                if (DebugWriter.isOpenDebug()) {
-                    ConcurrentMap<MessageQueue, AtomicLong> offsetTable = ReflectUtil.getDeclaredField(this, "offsetTable");
-                    DebugWriter.getInstance(getTopic()).writeSaveOffset(mq, offsetTable.get(mq));
-                }
-                //                LOG.info("the queue Id is " + new RocketMQMessageQueue(mq).getQueueId() + ",rocketmq start save offset,the save time is " + DateUtil.getCurrentTimeString());
-                super.updateConsumeOffsetToBroker(mq, offset, isOneway);
-            }
-        };
-        consumer.setOffsetStore(offsetStore);//每个一分钟运行一次
-    }
-
     @Override
     protected boolean isNotDataSplit(String queueId) {
-        return queueId.toUpperCase().startsWith("RETRY") || queueId.toUpperCase().startsWith("%RETRY%");
+        return false;
     }
 
     @Override
@@ -337,21 +229,24 @@
     }
 
     public void destroyConsumer() {
-        List<DefaultMQPushConsumer> oldConsumers = new ArrayList<>();
-        if (consumer != null) {
-            oldConsumers.add(consumer);
-        }
-        try {
-            for (DefaultMQPushConsumer consumer : oldConsumers) {
-                consumer.shutdown();
-            }
-
-        } catch (Throwable t) {
-            if (LOG.isWarnEnabled()) {
-                LOG.warn(t.getMessage(), t);
-            }
+        if (this.pullConsumer == null || this.pullTasks == null || this.pullTasks.length == 0) {
+            return;
         }
 
+        //不在拉取新的数据
+        for (PullTask pullTask : pullTasks) {
+            pullTask.shutdown();
+        }
+
+        //线程池关闭
+        this.executorService.shutdown();
+
+        //关闭消费实例
+        this.pullConsumer.shutdown();
+    }
+
+    public void commit(Set<MessageQueue> messageQueues) {
+        this.pullConsumer.commit(messageQueues, true);
     }
 
     @Override
@@ -360,12 +255,111 @@
         destroyConsumer();
     }
 
-    public String getStrategyName() {
-        return strategyName;
+    public class PullTask implements Runnable {
+        private final long pullTimeout;
+        private final long commitInternalMs;
+        private volatile long lastCommit = 0L;
+
+        private final DefaultLitePullConsumer pullConsumer;
+        private final MessageListenerDelegator delegator;
+
+        private volatile boolean isStopped = false;
+
+        public PullTask(DefaultLitePullConsumer pullConsumer, long pullTimeout, long commitInternalMs) {
+            this.pullConsumer = pullConsumer;
+            this.delegator = (MessageListenerDelegator) pullConsumer.getMessageQueueListener();
+            this.pullTimeout = pullTimeout == 0 ? pullConsumer.getPollTimeoutMillis() : pullTimeout;
+            this.commitInternalMs = commitInternalMs;
+        }
+
+        private void afterRebalance() {
+            //if rebalance happen, need block all other thread, wait remove split or load states from new split;
+            Set<MessageQueue> removingQueue = this.delegator.getRemovingQueue();
+
+            Set<String> splitIds = new HashSet<>();
+            for (MessageQueue mq : removingQueue) {
+                splitIds.add(new RocketMQMessageQueue(mq).getQueueId());
+            }
+
+            RocketMQSource.this.removeSplit(splitIds);
+
+            Set<MessageQueue> allQueueInLastRebalance = this.delegator.getLastDivided();
+            newRebalance(allQueueInLastRebalance);
+
+            this.delegator.hasSynchronized();
+        }
+
+        @Override
+        public void run() {
+
+            try {
+                //wait rebalance
+                synchronized (this.delegator.getMutex()) {
+                    this.delegator.getMutex().wait();
+                }
+                afterRebalance();
+            } catch (InterruptedException ignored) {
+            }
+
+            while (!this.isStopped) {
+
+                if (this.delegator.needSync()) {
+                    synchronized (this.pullConsumer) {
+                        if (this.delegator.needSync()) {
+                            afterRebalance();
+                        }
+                    }
+                }
+
+
+                List<MessageExt> msgs = pullConsumer.poll(pullTimeout);
+
+                int i = 0;
+                for (MessageExt msg : msgs) {
+                    JSONObject jsonObject = create(msg.getBody(), msg.getProperties());
+
+                    String topic = msg.getTopic();
+                    int queueId = msg.getQueueId();
+                    String brokerName = msg.getBrokerName();
+                    MessageQueue queue = new MessageQueue(topic, brokerName, queueId);
+                    String unionQueueId = RocketMQMessageQueue.getQueueId(queue);
+
+
+                    String offset = msg.getQueueOffset() + "";
+                    org.apache.rocketmq.streams.common.context.Message message = createMessage(jsonObject, unionQueueId, offset, false);
+                    message.getHeader().setOffsetIsLong(true);
+
+                    if (i == msgs.size() - 1) {
+                        message.getHeader().setNeedFlush(true);
+                    }
+                    executeMessage(message);
+                    i++;
+                }
+
+                //拉取的批量消息处理完成以后判断是否提交位点;
+                synchronized (this.pullConsumer) {
+                    if (System.currentTimeMillis() - lastCommit >= commitInternalMs || isStopped) {
+                        lastCommit = System.currentTimeMillis();
+                        //向broker提交消费位点,todo 从consumer那里拿不到正在消费哪些messageQueue
+                        commit(this.delegator.getLastDivided());
+                    }
+                }
+            }
+        }
+
+        public void shutdown() {
+            this.isStopped = true;
+        }
     }
 
-    public void setStrategyName(String strategyName) {
-        this.strategyName = strategyName;
+    private void newRebalance(Set<MessageQueue> allQueueInLastRebalance){
+        Set<String> temp = new HashSet<>();
+        for (MessageQueue queue : allQueueInLastRebalance) {
+            String unionQueueId = RocketMQMessageQueue.getQueueId(queue);
+            temp.add(unionQueueId);
+        }
+
+        super.addNewSplit(temp);
     }
 
     public String getTags() {
@@ -376,39 +370,45 @@
         this.tags = tags;
     }
 
-    public DefaultMQPushConsumer getConsumer() {
-        return consumer;
+    public Long getPullTimeout() {
+        return pullTimeout;
     }
 
-    public Long getPullIntervalMs() {
-        return pullIntervalMs;
+    public void setPullTimeout(Long pullTimeout) {
+        this.pullTimeout = pullTimeout;
     }
 
-    public void setPullIntervalMs(Long pullIntervalMs) {
-        this.pullIntervalMs = pullIntervalMs;
+    public String getStrategyName() {
+        return strategyName;
     }
 
-    public String getNamesrvAddr() {
-        return namesrvAddr;
+    public void setStrategyName(String strategyName) {
+        this.strategyName = strategyName;
     }
 
-    public void setNamesrvAddr(String namesrvAddr) {
-        this.namesrvAddr = namesrvAddr;
+    public RPCHook getRpcHook() {
+        return rpcHook;
     }
 
-    public ConsumeFromWhere getConsumeFromWhere() {
-        return consumeFromWhere;
+    public void setRpcHook(RPCHook rpcHook) {
+        this.rpcHook = rpcHook;
     }
 
-    public void setConsumeFromWhere(ConsumeFromWhere consumeFromWhere) {
-        this.consumeFromWhere = consumeFromWhere;
+    public int getUserPullThreadNum() {
+        return userPullThreadNum;
     }
 
-    public String getConsumerOffset() {
-        return consumerOffset;
+    public void setUserPullThreadNum(int userPullThreadNum) {
+        this.userPullThreadNum = userPullThreadNum;
     }
 
-    public void setConsumerOffset(String consumerOffset) {
-        this.consumerOffset = consumerOffset;
+    public long getCommitInternalMs() {
+        return commitInternalMs;
     }
+
+    public void setCommitInternalMs(long commitInternalMs) {
+        this.commitInternalMs = commitInternalMs;
+    }
+
+
 }
\ No newline at end of file
diff --git a/rocketmq-streams-checkpoint/pom.xml b/rocketmq-streams-checkpoint/pom.xml
deleted file mode 100644
index 6da59e3..0000000
--- a/rocketmq-streams-checkpoint/pom.xml
+++ /dev/null
@@ -1,64 +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-checkpoint</artifactId>
-    <name>ROCKETMQ STREAMS :: checkpoint</name>
-    <packaging>jar</packaging>
-
-    <properties>
-        <maven.compiler.source>8</maven.compiler.source>
-        <maven.compiler.target>8</maven.compiler.target>
-    </properties>
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-commons</artifactId>
-            <exclusions>
-                <exclusion>
-                    <groupId>com.google.auto.service</groupId>
-                    <artifactId>auto-service</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-db-operator</artifactId>
-            <exclusions>
-                <exclusion>
-                    <groupId>com.google.auto.service</groupId>
-                    <artifactId>auto-service</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
-            <groupId>com.google.auto.service</groupId>
-            <artifactId>auto-service</artifactId>
-            <optional>true</optional>
-        </dependency>
-    </dependencies>
-
-</project>
\ No newline at end of file
diff --git a/rocketmq-streams-checkpoint/src/main/java/org/apache/rocketmq/streams/checkpoint/db/DBCheckPointStorage.java b/rocketmq-streams-checkpoint/src/main/java/org/apache/rocketmq/streams/checkpoint/db/DBCheckPointStorage.java
deleted file mode 100644
index 69150ac..0000000
--- a/rocketmq-streams-checkpoint/src/main/java/org/apache/rocketmq/streams/checkpoint/db/DBCheckPointStorage.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.checkpoint.db;
-
-import java.util.List;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.channel.source.ISource;
-import org.apache.rocketmq.streams.common.checkpoint.AbstractCheckPointStorage;
-import org.apache.rocketmq.streams.common.checkpoint.CheckPoint;
-import org.apache.rocketmq.streams.common.checkpoint.CheckPointManager;
-import org.apache.rocketmq.streams.common.checkpoint.SourceSnapShot;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
-
-/**
- * @description
- */
-public class DBCheckPointStorage extends AbstractCheckPointStorage {
-
-    static final Log logger = LogFactory.getLog(DBCheckPointStorage.class);
-    static final String STORAGE_NAME = "DB";
-
-    public DBCheckPointStorage() {
-
-    }
-
-    @Override
-    public String getStorageName() {
-        return STORAGE_NAME;
-    }
-
-    @Override
-    public <T> void save(List<T> checkPointState) {
-        logger.info(String.format("save checkpoint size %d", checkPointState.size()));
-        ORMUtil.batchReplaceInto(checkPointState);
-    }
-
-    @Override
-    public void finish() {
-
-    }
-
-    @Override
-    //todo
-    public CheckPoint recover(ISource iSource, String queueId) {
-        String sourceName = CheckPointManager.createSourceName(iSource, null);
-        String key = CheckPointManager.createCheckPointKey(sourceName, queueId);
-        String sql = "select * from source_snap_shot where `key` = " + "'" + key + "';";
-        SourceSnapShot snapShot = ORMUtil.queryForObject(sql, null, SourceSnapShot.class);
-
-        logger.info(String.format("checkpoint recover key is %s, sql is %s, recover sourceSnapShot : %s", key, sql, snapShot == null ? "null snapShot" : snapShot.toString()));
-        return new CheckPoint().fromSnapShot(snapShot);
-    }
-}
diff --git a/rocketmq-streams-clients/pom.xml b/rocketmq-streams-clients/pom.xml
index 3cee17d..ff0e4cb 100644
--- a/rocketmq-streams-clients/pom.xml
+++ b/rocketmq-streams-clients/pom.xml
@@ -52,17 +52,9 @@
         </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>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-connectors</artifactId>
-        </dependency>
-        <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>
             <version>1.7.26</version>
diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/ScheduledStreamBuilder.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/ScheduledStreamBuilder.java
deleted file mode 100644
index af7d5cd..0000000
--- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/ScheduledStreamBuilder.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.client;
-
-import java.util.Arrays;
-import java.util.Set;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import org.apache.commons.lang3.concurrent.BasicThreadFactory;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.utils.ThreadUtil;
-
-/**
- * @description
- */
-public class ScheduledStreamBuilder {
-
-    static final Log logger = LogFactory.getLog(ScheduledStreamBuilder.class);
-
-    protected ScheduledExecutorService balanceExecutor;
-
-    TimeUnit timeUnit;
-
-    int interval;
-
-    ScheduledTask task;
-
-    public ScheduledStreamBuilder(int interval, TimeUnit timeUnit) {
-        this.interval = interval;
-        this.timeUnit = timeUnit;
-        balanceExecutor = new ScheduledThreadPoolExecutor(1, new BasicThreadFactory.Builder().namingPattern("cycle-builder-task-%d").daemon(true).build());
-    }
-
-    public void setTask(ScheduledTask task) {
-        this.task = task;
-    }
-
-    public void start() {
-        balanceExecutor.scheduleAtFixedRate(task, 0, interval, timeUnit);
-
-        while (true) {
-            Set<Thread> threadSet = Thread.getAllStackTraces().keySet();
-            for (Thread th : threadSet) {
-                logger.error(String.format("CycleStreamBuilder size %d, name is %s, stack is %s. ", threadSet.size(), th.getName(), Arrays.toString(th.getStackTrace())));
-            }
-
-            ThreadUtil.sleep(10000);
-        }
-    }
-}
diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/ScheduledTask.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/ScheduledTask.java
deleted file mode 100644
index 1af571c..0000000
--- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/ScheduledTask.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.client;
-
-import java.util.Date;
-import org.apache.rocketmq.streams.client.source.DataStreamSource;
-import org.apache.rocketmq.streams.client.transform.DataStream;
-import org.apache.rocketmq.streams.common.component.ComponentCreator;
-import org.apache.rocketmq.streams.common.configure.ConfigureFileKey;
-import org.apache.rocketmq.streams.connectors.source.filter.CycleSchedule;
-import org.apache.rocketmq.streams.db.sink.EnhanceDBSink;
-
-/**
- * @description
- */
-public class ScheduledTask implements Runnable {
-
-    CycleSchedule schedule;
-    String sinkTableName;
-    String sourceTableName;
-    String url;
-    String userName;
-    String password;
-
-    public ScheduledTask(String expression, String url, String userName, String password, String sourceTableName,
-        String sinkTableName) {
-        schedule = CycleSchedule.getInstance(expression, new Date());
-        this.sourceTableName = sourceTableName;
-        this.sinkTableName = sinkTableName;
-        this.url = url;
-        this.userName = userName;
-        this.password = password;
-        ComponentCreator.getProperties().put(ConfigureFileKey.CHECKPOINT_STORAGE_NAME, "db");
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, url);//数据库连接url
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME, userName);//用户名
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD, password);//password
-        ComponentCreator.getProperties().put(ConfigureFileKey.IS_ATOMIC_DB_SINK, "true");
-
-    }
-
-    @Override
-    public void run() {
-        CycleSchedule.Cycle cycle = schedule.nextCycle(new Date());
-        DataStreamSource dataStreamsource = StreamBuilder.dataStream("test_baseline" + "_" + cycle.getCycleDateStr(), "baseline_pipeline");
-        DataStream datastream = dataStreamsource.fromCycleSource(url, userName, password, sourceTableName, cycle, 3);
-        EnhanceDBSink sink = new EnhanceDBSink();
-        sink.setAtomic(true);
-        sink.setTableName(sinkTableName);
-        sink.setUrl(url);
-        sink.setUserName(userName);
-        sink.setPassword(password);
-        sink.init();
-        datastream.to(sink).start(true);
-    }
-}
diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/source/DataStreamSource.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/source/DataStreamSource.java
index 9422519..8ee4568 100644
--- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/source/DataStreamSource.java
+++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/source/DataStreamSource.java
@@ -36,6 +36,8 @@
 import com.alibaba.fastjson.JSONObject;
 import com.google.common.collect.Sets;
 import java.util.Set;
+
+import org.apache.rocketmq.remoting.RPCHook;
 import org.apache.rocketmq.streams.client.transform.DataStream;
 import org.apache.rocketmq.streams.common.channel.impl.CollectionSource;
 import org.apache.rocketmq.streams.common.channel.impl.file.FileSource;
@@ -43,9 +45,6 @@
 import org.apache.rocketmq.streams.common.channel.impl.memory.MemorySource;
 import org.apache.rocketmq.streams.common.channel.source.ISource;
 import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder;
-import org.apache.rocketmq.streams.connectors.source.CycleDynamicMultipleDBScanSource;
-import org.apache.rocketmq.streams.connectors.source.DynamicMultipleDBScanSource;
-import org.apache.rocketmq.streams.connectors.source.filter.CycleSchedule;
 import org.apache.rocketmq.streams.mqtt.source.PahoSource;
 import org.apache.rocketmq.streams.source.RocketMQSource;
 
@@ -96,44 +95,26 @@
     }
 
     public DataStream fromRocketmq(String topic, String groupName, boolean isJson, String namesrvAddress) {
-        return fromRocketmq(topic, groupName, "*", isJson, namesrvAddress);
+        return fromRocketmq(topic, groupName, "*", isJson, namesrvAddress, null);
     }
 
-    public DataStream fromRocketmq(String topic, String groupName, String tags, boolean isJson, String namesrvAddress) {
+    public DataStream fromRocketmq(String topic, String groupName, boolean isJson, String namesrvAddress, RPCHook rpcHook) {
+        return fromRocketmq(topic, groupName, "*", isJson, namesrvAddress, rpcHook);
+    }
+
+    public DataStream fromRocketmq(String topic, String groupName, String tags, boolean isJson, String namesrvAddress, RPCHook rpcHook) {
         RocketMQSource rocketMQSource = new RocketMQSource();
         rocketMQSource.setTopic(topic);
         rocketMQSource.setTags(tags);
         rocketMQSource.setGroupName(groupName);
         rocketMQSource.setJsonData(isJson);
         rocketMQSource.setNamesrvAddr(namesrvAddress);
+        rocketMQSource.setRpcHook(rpcHook);
         this.mainPipelineBuilder.setSource(rocketMQSource);
         return new DataStream(this.mainPipelineBuilder, null);
     }
 
-    public DataStream fromMultipleDB(String url, String userName, String password, String tablePattern) {
-        DynamicMultipleDBScanSource source = new DynamicMultipleDBScanSource();
-        source.setUrl(url);
-        source.setUserName(userName);
-        source.setPassword(password);
-        source.setBatchSize(10);
-        source.setLogicTableName(tablePattern);
-        this.mainPipelineBuilder.setSource(source);
-        return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null);
-    }
 
-    public DataStream fromCycleSource(String url, String userName, String password, String tablePattern,
-        CycleSchedule.Cycle cycle, int balanceSec) {
-        CycleDynamicMultipleDBScanSource source = new CycleDynamicMultipleDBScanSource(cycle);
-        source.setUrl(url);
-        source.setUserName(userName);
-        source.setPassword(password);
-        source.setBatchSize(10);
-        source.setLogicTableName(tablePattern);
-        source.setBalanceTimeSecond(balanceSec);
-
-        this.mainPipelineBuilder.setSource(source);
-        return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null);
-    }
 
     public DataStream fromCollection(JSONObject... elements) {
         CollectionSource source = new CollectionSource();
diff --git a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/DataStream.java b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/DataStream.java
index 80ae510..c3c9f66 100644
--- a/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/DataStream.java
+++ b/rocketmq-streams-clients/src/main/java/org/apache/rocketmq/streams/client/transform/DataStream.java
@@ -26,6 +26,7 @@
 import java.util.Properties;
 import java.util.Set;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.remoting.RPCHook;
 import org.apache.rocketmq.streams.client.strategy.LogFingerprintStrategy;
 import org.apache.rocketmq.streams.client.strategy.Strategy;
 import org.apache.rocketmq.streams.client.transform.window.WindowInfo;
@@ -61,8 +62,6 @@
 import org.apache.rocketmq.streams.db.sink.DBSink;
 import org.apache.rocketmq.streams.db.sink.DynamicMultipleDBSink;
 import org.apache.rocketmq.streams.db.sink.EnhanceDBSink;
-import org.apache.rocketmq.streams.dim.model.DBDim;
-import org.apache.rocketmq.streams.dim.model.FileDim;
 import org.apache.rocketmq.streams.filter.operator.FilterOperator;
 import org.apache.rocketmq.streams.filter.operator.Rule;
 import org.apache.rocketmq.streams.mqtt.sink.PahoSink;
@@ -371,116 +370,7 @@
         return new SplitStream(this.mainPipelineBuilder, this.otherPipelineBuilders, stage);
     }
 
-    /**
-     * 维表join,mysql场景,不需要指定jdbcdriver
-     *
-     * @param url
-     * @param userName
-     * @param password
-     * @param sqlOrTableName
-     * @return
-     */
-    @Deprecated
-    public JoinStream join(String url, String userName, String password, String sqlOrTableName,
-        long pollingTimeMintue) {
-        return join(url, userName, password, sqlOrTableName, null, pollingTimeMintue);
-    }
 
-    /**
-     * 维表join
-     *
-     * @param url
-     * @param userName
-     * @param password
-     * @param sqlOrTableName
-     * @return
-     */
-    @Deprecated
-    public JoinStream join(String url, String userName, String password, String sqlOrTableName, String jdbcDriver,
-        long pollingTimeMinute) {
-        DBDim dbDim = new DBDim();
-        dbDim.setUrl(url);
-        dbDim.setUserName(userName);
-        dbDim.setPassword(password);
-        dbDim.setSql(sqlOrTableName);
-        dbDim.setPollingTimeMinute(pollingTimeMinute);
-        dbDim.setJdbcdriver(jdbcDriver);
-        this.mainPipelineBuilder.addConfigurables(dbDim);
-        return new JoinStream(dbDim, mainPipelineBuilder, otherPipelineBuilders, currentChainStage, true);
-    }
-
-    public JoinStream dimJoin(String url, String userName, String password, String sqlOrTableName,
-        Long pollingTimeMinute) {
-        return dimJoin(url, userName, password, sqlOrTableName, "com.mysql.jdbc.Driver", pollingTimeMinute);
-    }
-
-    public JoinStream dimJoin(String url, String userName, String password, String sqlOrTableName, String jdbcDriver,
-        Long pollingTimeMinute) {
-        return dimJoin(url, userName, password, sqlOrTableName, jdbcDriver, pollingTimeMinute, JoinStream.JoinType.INNER_JOIN);
-    }
-
-    public JoinStream dimJoin(String filePath, Long pollingTimeMinute) {
-        return dimJoin(filePath, pollingTimeMinute, JoinStream.JoinType.INNER_JOIN);
-    }
-
-    public JoinStream dimLeftJoin(String url, String userName, String password, String sqlOrTableName,
-        Long pollingTimeMinute) {
-        return dimLeftJoin(url, userName, password, sqlOrTableName, "com.mysql.jdbc.Driver", pollingTimeMinute);
-    }
-
-    public JoinStream dimLeftJoin(String url, String userName, String password, String sqlOrTableName,
-        String jdbcDriver, Long pollingTimeMinute) {
-        return dimJoin(url, userName, password, sqlOrTableName, jdbcDriver, pollingTimeMinute, JoinStream.JoinType.LEFT_JOIN);
-    }
-
-    public JoinStream dimLeftJoin(String filePath, Long pollingTimeMinute) {
-        return dimJoin(filePath, pollingTimeMinute, JoinStream.JoinType.LEFT_JOIN);
-    }
-
-    protected JoinStream dimJoin(String filePath, Long pollingTimeMinute, JoinStream.JoinType joinType) {
-        FileDim fileDim = new FileDim();
-        fileDim.setFilePath(filePath);
-        fileDim.setPollingTimeMinute(pollingTimeMinute);
-        this.mainPipelineBuilder.addConfigurables(fileDim);
-        return new JoinStream(fileDim, mainPipelineBuilder, otherPipelineBuilders, currentChainStage, true, joinType);
-    }
-
-    protected JoinStream dimJoin(String url, String userName, String password, String sqlOrTableName, String jdbcDriver,
-        Long pollingTimeMinute, JoinStream.JoinType joinType) {
-        DBDim dbDim = new DBDim();
-        dbDim.setUrl(url);
-        dbDim.setUserName(userName);
-        dbDim.setPassword(password);
-        dbDim.setSql(sqlOrTableName);
-        dbDim.setPollingTimeMinute(pollingTimeMinute);
-        dbDim.setJdbcdriver(jdbcDriver);
-        this.mainPipelineBuilder.addConfigurables(dbDim);
-        return new JoinStream(dbDim, mainPipelineBuilder, otherPipelineBuilders, currentChainStage, true, joinType);
-    }
-
-    /**
-     * 维表join
-     *
-     * @param filePath
-     * @return
-     */
-    @Deprecated
-    public JoinStream join(String filePath, long pollingTimeMinute) {
-        FileDim fileDim = new FileDim();
-        fileDim.setFilePath(filePath);
-        fileDim.setPollingTimeMinute(pollingTimeMinute);
-        this.mainPipelineBuilder.addConfigurables(fileDim);
-        return new JoinStream(fileDim, mainPipelineBuilder, otherPipelineBuilders, currentChainStage, true);
-    }
-
-    @Deprecated
-    public JoinStream innerJoin(String filePath, long pollingTimeMinute) {
-        FileDim fileDim = new FileDim();
-        fileDim.setFilePath(filePath);
-        fileDim.setPollingTimeMinute(pollingTimeMinute);
-        this.mainPipelineBuilder.addConfigurables(fileDim);
-        return new JoinStream(fileDim, mainPipelineBuilder, otherPipelineBuilders, currentChainStage, true);
-    }
 
     /**
      * 遍历所有数据
@@ -608,19 +498,23 @@
     }
 
     public DataStream toRocketmq(String topic, String nameServerAddress) {
-        return toRocketmq(topic, "*", null, -1, nameServerAddress, null, false);
+        return toRocketmq(topic, null, nameServerAddress);
     }
 
     public DataStream toRocketmq(String topic, String groupName, String nameServerAddress) {
-        return toRocketmq(topic, "*", groupName, -1, nameServerAddress, null, false);
+        return toRocketmq(topic, "*", groupName,  nameServerAddress);
     }
 
     public DataStream toRocketmq(String topic, String tags, String groupName, String nameServerAddress) {
-        return toRocketmq(topic, tags, groupName, -1, nameServerAddress, null, false);
+        return toRocketmq(topic, tags, groupName, nameServerAddress, null);
+    }
+
+    public DataStream toRocketmq(String topic, String tags, String groupName, String nameServerAddress, RPCHook rpcHook) {
+        return toRocketmq(topic, tags, groupName, -1, nameServerAddress, null, false, rpcHook);
     }
 
     public DataStream toRocketmq(String topic, String tags, String groupName, int batchSize, String nameServerAddress,
-        String clusterName, boolean order) {
+                                 String clusterName, boolean order, RPCHook rpcHook) {
         RocketMQSink rocketMQSink = new RocketMQSink();
         if (StringUtils.isNotBlank(topic)) {
             rocketMQSink.setTopic(topic);
@@ -641,6 +535,8 @@
             rocketMQSink.setBatchSize(batchSize);
         }
         rocketMQSink.setOrder(order);
+        rocketMQSink.setRpcHook(rpcHook);
+
         ChainStage<?> output = this.mainPipelineBuilder.createStage(rocketMQSink);
         this.mainPipelineBuilder.setTopologyStages(currentChainStage, output);
         return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, output);
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 69f3960..4ef73b9 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
@@ -28,7 +28,6 @@
 import org.apache.rocketmq.streams.common.topology.ChainStage;
 import org.apache.rocketmq.streams.common.topology.builder.PipelineBuilder;
 import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.dim.model.AbstractDim;
 import org.apache.rocketmq.streams.filter.builder.ExpressionBuilder;
 import org.apache.rocketmq.streams.filter.function.expression.Equals;
 import org.apache.rocketmq.streams.filter.operator.expression.Expression;
@@ -40,9 +39,6 @@
 
     private static final String INNER_VAR_NAME_PREFIX = "___";
     protected JoinWindow joinWindow;//完成join 条件的添加
-    protected boolean isDimJoin = false;//是否是维表join
-
-    protected AbstractDim dim;//维度表对象
     protected String onCondition;//条件
     protected JoinType joinType;//连接类型
 
@@ -77,41 +73,6 @@
     }
 
     /**
-     * 维表join 场景
-     *
-     * @param pipelineBuilder
-     * @param pipelineBuilders
-     * @param currentChainStage
-     */
-    public JoinStream(AbstractDim dim, PipelineBuilder pipelineBuilder, Set<PipelineBuilder> pipelineBuilders,
-        ChainStage<?> currentChainStage) {
-        this.dim = dim;
-        this.pipelineBuilder = pipelineBuilder;
-        this.otherPipelineBuilders = pipelineBuilders;
-        this.currentChainStage = currentChainStage;
-        this.isDimJoin = true;
-    }
-
-    public JoinStream(AbstractDim dim, PipelineBuilder pipelineBuilder, Set<PipelineBuilder> pipelineBuilders,
-        ChainStage<?> currentChainStage, Boolean isDimJoin) {
-        this.dim = dim;
-        this.pipelineBuilder = pipelineBuilder;
-        this.otherPipelineBuilders = pipelineBuilders;
-        this.currentChainStage = currentChainStage;
-        this.isDimJoin = isDimJoin;
-    }
-
-    public JoinStream(AbstractDim dim, PipelineBuilder pipelineBuilder, Set<PipelineBuilder> pipelineBuilders,
-        ChainStage<?> currentChainStage, Boolean isDimJoin, JoinType joinType) {
-        this.pipelineBuilder = pipelineBuilder;
-        this.otherPipelineBuilders = pipelineBuilders;
-        this.currentChainStage = currentChainStage;
-        this.dim = dim;
-        this.isDimJoin = isDimJoin;
-        this.joinType = joinType;
-    }
-
-    /**
      * 设置join的类型, 推荐直接使用DataStream中的特定join来实现
      *
      * @param joinType
@@ -130,11 +91,6 @@
      * @return
      */
     public JoinStream window(Time time) {
-
-        //维表join 不需要设置
-        if (isDimJoin) {
-            throw new RuntimeException("can not support this method");
-        }
         joinWindow.setTimeUnitAdjust(1);
         joinWindow.setSizeInterval(time.getValue());
         joinWindow.setSlideInterval(time.getValue());
@@ -142,6 +98,11 @@
         return this;
     }
 
+    public JoinStream setLocalStorageOnly(boolean localStorageOnly) {
+        this.joinWindow.setLocalStorageOnly(localStorageOnly);
+        return this;
+    }
+
     /**
      * 增加条件,用表达式形式表达(leftFieldName,function,rightFieldName)&&({name,==,otherName}||(age,==,age)) 后续再增加结构化的方法 。 后续该方法将下线,推荐使用on
      *
@@ -161,38 +122,10 @@
     }
 
     public DataStream toDataStream() {
-        if (isDimJoin) {
-            return doDimJoin();
-        } else {
-            return doJoin();
-        }
-
+        return doJoin();
     }
 
-    /**
-     * 维度表join的场景
-     */
-    protected DataStream doDimJoin() {
-        if (StringUtil.isNotEmpty(this.onCondition)) {
-            this.dim.createIndexByJoinCondition(this.onCondition, new AbstractDim.IDimField() {
-                @Override
-                public boolean isDimField(Object fieldName) {
-                    return true;
-                }
-            });
-        }
-        String script = null;
-        if (JoinType.INNER_JOIN == joinType) {
-            String data = createName("inner_join");
-            script = data + "=inner_join('" + dim.getNameSpace() + "','" + dim.getConfigureName() + "','" + onCondition + "', '' ,''," + ");splitArray('" + data + "');rm(" + data + ");";
-        } else if ((JoinType.LEFT_JOIN == joinType)) {
-            String data = createName("left_join");
-            script = data + "=left_join('" + dim.getNameSpace() + "','" + dim.getConfigureName() + "','" + onCondition + "', '' ,''," + ");if(!null(" + data + ")){splitArray('" + data + "');};rm(" + data + ");";
-        }
-        ChainStage<?> stage = this.pipelineBuilder.createStage(new ScriptOperator(script));
-        this.pipelineBuilder.setTopologyStages(currentChainStage, stage);
-        return new DataStream(pipelineBuilder, otherPipelineBuilders, stage);
-    }
+
 
     /**
      * 双流join的场景
diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/JoinTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/JoinTest.java
index 1b15d71..ba3f8b7 100644
--- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/JoinTest.java
+++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/JoinTest.java
@@ -62,22 +62,4 @@
 
     }
 
-    @Test
-    public void testDim() {
-        DataStream stream = (StreamBuilder.dataStream("namespace", "name")
-            .fromFile("/Users/junjie.cheng/workspace/rocketmq-streams-apache/rocketmq-streams-clients/src/test/resources/window_msg_10.txt")
-            .filter((FilterFunction<JSONObject>) value -> {
-                if (value.getString("ProjectName") == null || value.getString("LogStore") == null) {
-                    return true;
-                }
-                return false;
-            }))
-            .join("dburl", "dbUserName", "dbPassowrd", "tableNameOrSQL", 5)
-            .setCondition("(name,==,name)")
-            .toDataStream()
-            .selectFields("name", "age", "address")
-            .toPrint();
-        stream.start();
-
-    }
 }
diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/LeaseTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/LeaseTest.java
deleted file mode 100644
index 1d3ec8a..0000000
--- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/LeaseTest.java
+++ /dev/null
@@ -1,98 +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.client;
-
-import java.util.Date;
-import org.apache.rocketmq.streams.common.component.ComponentCreator;
-import org.apache.rocketmq.streams.common.configure.ConfigureFileKey;
-import org.apache.rocketmq.streams.db.driver.DriverBuilder;
-import org.apache.rocketmq.streams.db.driver.JDBCDriver;
-import org.apache.rocketmq.streams.lease.LeaseComponent;
-import org.apache.rocketmq.streams.lease.model.LeaseInfo;
-import org.apache.rocketmq.streams.lease.service.ILeaseGetCallback;
-import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-
-public class LeaseTest {
-
-    private String URL="";
-    protected String USER_NAME="";
-    protected String PASSWORD="";
-
-    public LeaseTest(){
-
-        //正式使用时,在配置文件配置
-        ComponentCreator.getProperties().put(ConfigureFileKey.CONNECT_TYPE,"DB");
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL,URL);//数据库连接url
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME,USER_NAME);//用户名
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD,PASSWORD);//password
-
-        /**
-         * 创建lease info表
-         */
-        JDBCDriver driver= DriverBuilder.createDriver();
-        driver.execute(LeaseInfo.createTableSQL());
-    }
-
-    @Test
-    public void testLease() throws InterruptedException {
-        String leaseName="lease.test";
-        int leaseTime=5;
-        LeaseComponent.getInstance().getService().startLeaseTask(leaseName, leaseTime, new ILeaseGetCallback() {
-            @Override
-            public void callback(Date nextLeaseDate) {
-                System.out.println("I get lease");
-            }
-        });
-        assertTrue(LeaseComponent.getInstance().getService().hasLease(leaseName));
-        Thread.sleep(5000);
-        assertTrue(LeaseComponent.getInstance().getService().hasLease(leaseName));//会一直续约
-        Thread.sleep(5000);
-        assertTrue(LeaseComponent.getInstance().getService().hasLease(leaseName));//会一直续约
-    }
-
-    @Test
-    public void testLock() throws InterruptedException {
-        String name="dipper";
-        String lockName="lease.test";
-        int leaseTime=5;
-        boolean success=LeaseComponent.getInstance().getService().lock(name,lockName,leaseTime);//锁定5秒钟
-        assertTrue(success);//获取锁
-        Thread.sleep(6000);
-        assertTrue(!LeaseComponent.getInstance().getService().hasHoldLock(name,lockName));//超期释放
-    }
-
-    /**
-     * holdlock是一直持有锁,和租约的区别是,当释放锁后,无其他实例抢占
-     *
-     * @throws InterruptedException
-     */
-    @Test
-    public void testHoldLock() throws InterruptedException {
-        String name="dipper";
-        String lockName="lease.test";
-        int leaseTime=6;
-        boolean success=LeaseComponent.getInstance().getService().holdLock(name,lockName,leaseTime);//锁定5秒钟
-        assertTrue(success);//获取锁
-        Thread.sleep(8000);
-        assertTrue(LeaseComponent.getInstance().getService().hasHoldLock(name,lockName));//会自动续约,不会释放,可以手动释放
-        LeaseComponent.getInstance().getService().unlock(name,lockName);
-        assertTrue(!LeaseComponent.getInstance().getService().hasHoldLock(name,lockName));
-    }
-}
diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQGroupByTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQGroupByTest.java
index 014316b..2a91565 100644
--- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQGroupByTest.java
+++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQGroupByTest.java
@@ -31,7 +31,7 @@
     @Test
     public void testRocketMq() throws Exception {
         DataStreamSource.create("tmp", "tmp")
-            .fromRocketmq("dipper_test_write_merge5", "dipper_group", true, "localhost:9876")
+            .fromRocketmq("dipper_test_write_merge5", "dipper_group", true, "localhost:9876", null)
             .window(TumblingWindow.of(Time.seconds(5)))
             .groupBy("host_uuid", "cmdline")
             .count("c")
diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQTest.java
index e0648d1..3c1e88b 100644
--- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQTest.java
+++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/RocketMQTest.java
@@ -127,7 +127,7 @@
     @Test
     public void testConsumer() {
         DataStreamSource.create("tmp", "tmp")
-            .fromRocketmq("dipper_test_write_merge", "dipper_group", true, "localhost:9876")
+            .fromRocketmq("dipper_test_write_merge", "dipper_group", true, "localhost:9876",null)
             .flatMap(new FlatMapFunction<JSONObject, JSONObject>() {
                 @Override public List<JSONObject> flatMap(JSONObject message) throws Exception {
                     JSONArray jsonArray = message.getJSONArray("data");
diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceTest.java
index 10d144e..71c0e7e 100644
--- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceTest.java
+++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SourceTest.java
@@ -72,10 +72,6 @@
         AtomicLong startTime = new AtomicLong(System.currentTimeMillis());
         DataStreamSource.create("tmp", "tmp")
             .fromRocketmq("dipper_test_1", "dipper_group1", true, "localhost:9876")
-//           .window(TumblingWindow.of(Time.seconds(10)))
-//            .count("c")
-//            .groupBy("")
-//            .toDataSteam()
             .forEachMessage(new ForEachMessageFunction() {
                 @Override public void foreach(IMessage message, AbstractContext context) {
                     System.out.println(message.getMessageBody());
@@ -94,19 +90,5 @@
             .start();
     }
 
-    @Test
-    public void testImportMsgFromSource() {
-        DataStreamSource.create("tmp", "tmp")
-            .fromRocketmq("TOPIC_AEGIS_DETECT_MSG", "chris_test", "T_MSG_PROC", true, null)
-            .toFile("/tmp/aegis_proc.txt", true)
-            .start();
-    }
 
-    @Test
-    public void testImportMsgFromNet() {
-        DataStreamSource.create("tmp", "tmp")
-            .fromRocketmq("TOPIC_AEGIS_DETECT_MSG", "chris_test", "T_MSG_NETSTAT", true, null)
-            .toFile("/tmp/aegis_net.txt", true)
-            .start();
-    }
 }
diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SplitTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SplitTest.java
deleted file mode 100644
index 949fb6a..0000000
--- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/SplitTest.java
+++ /dev/null
@@ -1,86 +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.client;
-
-import com.alibaba.fastjson.JSONObject;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.rocketmq.streams.client.transform.DataStream;
-import org.apache.rocketmq.streams.client.transform.SplitStream;
-import org.apache.rocketmq.streams.client.transform.window.Time;
-import org.apache.rocketmq.streams.client.transform.window.TumblingWindow;
-import org.apache.rocketmq.streams.common.functions.FilterFunction;
-import org.apache.rocketmq.streams.common.functions.FlatMapFunction;
-import org.apache.rocketmq.streams.common.functions.SplitFunction;
-import org.junit.Test;
-
-public class SplitTest implements Serializable {
-
-    @Test
-    public void testOperator() throws InterruptedException {
-        DataStream stream = (StreamBuilder.dataStream("namespace", "name").fromFile("/Users/yuanxiaodong/chris/sls_1000.txt").flatMap(new FlatMapFunction<JSONObject, String>() {
-            @Override
-            public List<JSONObject> flatMap(String message) throws Exception {
-                List<JSONObject> msgs = new ArrayList<>();
-                for (int i = 0; i < 10; i++) {
-                    JSONObject msg = JSONObject.parseObject(message);
-                    msg.put("index", i);
-                    msgs.add(msg);
-                }
-                return msgs;
-            }
-        }).filter(message -> ((JSONObject) message).getString("Project") == null));
-
-        SplitStream splitStream = stream.split(new SplitFunction<JSONObject>() {
-            @Override
-            public String split(JSONObject o) {
-                if (o.getInteger("age") < 18) {
-                    return "children";
-                } else if (o.getInteger("age") >= 18) {
-                    return "adult";
-                }
-                return null;
-            }
-        });
-
-        DataStream children = splitStream.select("children");
-        DataStream adult = splitStream.select("adult");
-        children.union(adult).join("dburl", "dbUserName", "dbPassowrd", "tableNameOrSQL", 5).setCondition("(name,==,name)").toDataStream().window(TumblingWindow.of(Time.seconds(5))).groupBy("ProjectName", "LogStore").setLocalStorageOnly(true).count("total").sum("OutFlow", "OutFlow").sum("InFlow", "InFlow").toDataStream().toPrint().asyncStart();
-        while (true) {
-            Thread.sleep(1000);
-        }
-
-    }
-
-    @Test
-    public void testDim() {
-        DataStream stream = (StreamBuilder.dataStream("namespace", "name").fromFile("/Users/yuanxiaodong/chris/sls_1000.txt").filter(new FilterFunction<JSONObject>() {
-
-            @Override
-            public boolean filter(JSONObject value) throws Exception {
-                if (value.getString("ProjectName") == null || value.getString("LogStore") == null) {
-                    return true;
-                }
-                return false;
-            }
-        })).join("dburl", "dbUserName", "dbPassowrd", "tableNameOrSQL", 5).setCondition("(name,==,name)").toDataStream().selectFields("name", "age", "address").toPrint();
-        stream.start();
-
-    }
-}
diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/UnionTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/UnionTest.java
index 1106625..da4ad13 100644
--- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/UnionTest.java
+++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/UnionTest.java
@@ -56,27 +56,4 @@
         leftStream.union(rightStream).toPrint().start();
     }
 
-    @Test
-    public void testDim() {
-        DataStream stream = (StreamBuilder.dataStream("namespace", "name")
-            .fromFile("/Users/yuanxiaodong/chris/sls_1000.txt")
-            .filter(new FilterFunction<JSONObject>() {
-
-                @Override
-                public boolean filter(JSONObject value) throws Exception {
-                    if (value.getString("ProjectName") == null || value.getString("LogStore") == null) {
-                        return true;
-                    }
-                    return false;
-                }
-            }))
-            .join("dburl", "dbUserName", "dbPassowrd", "tableNameOrSQL", 5)
-            .setCondition("(name,==,name)")
-            .toDataStream()
-            .selectFields("name", "age", "address")
-            .toPrint();
-
-        stream.start();
-
-    }
 }
diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/DimTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/DimTest.java
deleted file mode 100644
index abc5003..0000000
--- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/DimTest.java
+++ /dev/null
@@ -1,54 +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.client.example;
-
-import org.apache.rocketmq.streams.client.StreamBuilder;
-import org.apache.rocketmq.streams.client.transform.JoinStream;
-import org.junit.Test;
-
-public class DimTest {
-
-    @Test
-    public void testInnerJoinDim() {
-        StreamBuilder.dataStream("tmp", "tmp")
-            .fromFile("window_msg_10.txt", true)
-            .join("classpath://dim.txt", 10000)
-            .setJoinType(JoinStream.JoinType.INNER_JOIN)
-            .setCondition("(ProjectName,=,project)")
-            .toDataStream()
-            .toPrint()
-            .start();
-    }
-
-    @Test
-    public void testLeftDim() {
-        StreamBuilder.dataStream("tmp", "tmp")
-            .fromFile("window_msg_10.txt", true)
-            .join("classpath://dim.txt", 10000)
-            .setJoinType(JoinStream.JoinType.LEFT_JOIN)
-            .setCondition("(ProjectName,=,project)")
-            .toDataStream()
-            .toPrint()
-            .start();
-    }
-
-    @Test
-    public void testDim() {
-
-    }
-
-}
diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/JoinTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/JoinTest.java
index 53bbee4..dd1f17d 100644
--- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/JoinTest.java
+++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/example/JoinTest.java
@@ -80,9 +80,9 @@
     @Test
     public void testRocketmqJoin() {
 
-        DataStream left = StreamBuilder.dataStream("tmp", "tmp").fromRocketmq("TopicTest", "groupA", true, "localhost:9876");
+        DataStream left = StreamBuilder.dataStream("tmp", "tmp").fromRocketmq("TopicTest", "groupA", true, "localhost:9876", null);
         //DataStream right = StreamBuilder.dataStream("tmp", "tmp2").fromFile("dim.txt", true);
-        DataStream right = StreamBuilder.dataStream("tmp", "tmp").fromRocketmq("TopicTest", "groupB", true, "localhost:9876");
+        DataStream right = StreamBuilder.dataStream("tmp", "tmp").fromRocketmq("TopicTest", "groupB", true, "localhost:9876", null);
 
         left.join(right).on("(ProjectName,=,project)").toDataStream().toPrint().start();
     }
diff --git a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/MultiSplitTest.java b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/MultiSplitTest.java
index a628f28..862ede6 100644
--- a/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/MultiSplitTest.java
+++ b/rocketmq-streams-clients/src/test/java/org/apache/rocketmq/streams/client/windows/MultiSplitTest.java
@@ -32,7 +32,7 @@
     protected DataStream createSourceDataStream() {
 
         return StreamBuilder.dataStream("namespace", "name1")
-            .fromRocketmq(topic, "window_test", true, null);
+            .fromRocketmq(topic, "window_test", true, null, null);
     }
 
     protected int getSourceCount() {
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..939b1d5 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);
         }
 
@@ -451,6 +455,10 @@
             return;
         }
         this.checkPointManager.deleteRemovingSplit(splitIds);
+        for (String splitId : splitIds) {
+            this.checkPointManager.addSplit(splitId);
+        }
+
 
         JSONObject msg = new JSONObject();
         Message message = createMessage(msg, null, null, false);
@@ -622,6 +630,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..832981f
--- /dev/null
+++ b/rocketmq-streams-commons/src/main/java/org/apache/rocketmq/streams/common/utils/CreateTopicUtil.java
@@ -0,0 +1,57 @@
+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.HashMap;
+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());
+
+        HashMap<String, String> temp = new HashMap<>();
+        temp.put("+delete.policy", "COMPACTION");
+//        topicConfig.setAttributes(temp);
+
+        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-connectors/pom.xml b/rocketmq-streams-connectors/pom.xml
deleted file mode 100755
index 8c8277e..0000000
--- a/rocketmq-streams-connectors/pom.xml
+++ /dev/null
@@ -1,40 +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">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.rocketmq</groupId>
-        <artifactId>rocketmq-streams</artifactId>
-        <version>1.0.2-preview-SNAPSHOT</version>
-    </parent>
-    <artifactId>rocketmq-streams-connectors</artifactId>
-    <packaging>jar</packaging>
-    <name>ROCKETMQ STREAMS :: connectors</name>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-lease</artifactId>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-commons</artifactId>
-        </dependency>
-    </dependencies>
-</project>
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBounded.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBounded.java
deleted file mode 100644
index 41e5378..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBounded.java
+++ /dev/null
@@ -1,27 +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.connectors;
-
-/**
- * @description bounded stream interface
- */
-@Deprecated
-public interface IBounded {
-
-    boolean isFinished();
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBoundedSourceReader.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBoundedSourceReader.java
deleted file mode 100644
index 03995c3..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBoundedSourceReader.java
+++ /dev/null
@@ -1,26 +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.connectors;
-
-import org.apache.rocketmq.streams.common.interfaces.ILifeCycle;
-
-/**
- * @description
- */
-public interface IBoundedSourceReader extends ILifeCycle {
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/AbstractBalance.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/AbstractBalance.java
deleted file mode 100644
index 5c2b266..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/AbstractBalance.java
+++ /dev/null
@@ -1,207 +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.connectors.balance;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-import org.apache.rocketmq.streams.connectors.source.SourceInstance;
-
-public abstract class AbstractBalance implements ISourceBalance {
-
-    protected int balanceCount = 0;
-
-    @Override
-    public SplitChanged doBalance(List<ISplit> allSplits, List<ISplit> ownerSplits) {
-        balanceCount++;
-        heartBeat();
-        List<SourceInstance> sourceInstances = fetchSourceInstances();
-        List<ISplit> workingSplits = fetchWorkingSplits(allSplits);
-        SplitChanged splitChanged = getAdditionSplits(allSplits, sourceInstances, workingSplits, ownerSplits);
-        if (splitChanged != null) {
-            return splitChanged;
-        }
-        splitChanged = getRemoveSplits(allSplits, sourceInstances, workingSplits, ownerSplits);
-        return splitChanged;
-    }
-
-    protected void heartBeat() {
-        holdLockSourceInstance();
-    }
-
-    /**
-     * get all dispatch splits
-     *
-     * @return
-     */
-    protected abstract List<ISplit> fetchWorkingSplits(List<ISplit> allSplitS);
-
-    /**
-     * get all instacne for the source
-     *
-     * @return
-     */
-    protected abstract List<SourceInstance> fetchSourceInstances();
-
-    /**
-     * lock the source ,the lock is globel,only one source instance can get it in same time
-     *
-     * @return
-     */
-    protected abstract boolean holdLockSourceInstance();
-
-    /**
-     * unlock
-     */
-    protected abstract void unlockSourceInstance();
-
-    /**
-     * juge need add split,根据调度策略选择
-     * 每次最大增加的分片数,根据调度次数决定
-     *
-     * @param allSplits
-     * @param sourceInstances
-     * @param workingSplits
-     * @return
-     */
-    protected SplitChanged getAdditionSplits(List<ISplit> allSplits, List<SourceInstance> sourceInstances,
-        List<ISplit> workingSplits, List<ISplit> ownerSplits) {
-        SplitChanged splitChanged = getChangedSplitCount(allSplits, sourceInstances, workingSplits.size(), ownerSplits.size());
-        if (splitChanged == null) {
-            return null;
-        }
-        if (splitChanged.isNewSplit == false) {
-            return null;
-        }
-        if (splitChanged.splitCount <= 0) {
-            return null;
-        }
-        List<ISplit> noWorkingSplits = getNoWorkingSplits(allSplits, workingSplits);
-        List<ISplit> newSplits = new ArrayList<>();
-        for (int i = 0; i < noWorkingSplits.size(); i++) {
-            boolean success = holdLockSplit(noWorkingSplits.get(i));
-            if (success) {
-                newSplits.add(noWorkingSplits.get(i));
-                if (newSplits.size() >= splitChanged.splitCount) {
-                    break;
-                }
-            }
-        }
-        splitChanged.setChangedSplits(newSplits);
-        return splitChanged;
-
-    }
-
-    protected List<ISplit> getNoWorkingSplits(List<ISplit> allSplits, List<ISplit> workingSplits) {
-        Set<String> workingSplitIds = new HashSet<>();
-        for (ISplit split : workingSplits) {
-            workingSplitIds.add(split.getQueueId());
-        }
-        List<ISplit> splits = new ArrayList<>();
-        for (ISplit split : allSplits) {
-            if (!workingSplitIds.contains(split.getQueueId())) {
-                splits.add(split);
-            }
-        }
-        return splits;
-    }
-
-    /**
-     * 获取需要删除的分片
-     *
-     * @param allSplits
-     * @param sourceInstances
-     * @param workingSplits
-     * @return
-     */
-    protected SplitChanged getRemoveSplits(List<ISplit> allSplits, List<SourceInstance> sourceInstances,
-        List<ISplit> workingSplits, List<ISplit> ownerSplits) {
-        SplitChanged splitChanged = getChangedSplitCount(allSplits, sourceInstances, workingSplits.size(), ownerSplits.size());
-        if (splitChanged == null) {
-            return null;
-        }
-        if (splitChanged.isNewSplit == true) {
-            return null;
-        }
-
-        if (splitChanged.splitCount <= 0) {
-            return null;
-        }
-        //List<ISplit> ownerSplits=source.ownerSplits();
-        List<ISplit> removeSplits = new ArrayList();
-        for (int i = 0; i < splitChanged.splitCount; i++) {
-            removeSplits.add(ownerSplits.get(i));
-        }
-        splitChanged.setChangedSplits(removeSplits);
-        return splitChanged;
-    }
-
-    /**
-     * 获取需要变动的分片个数,新增或删除
-     * 分配策略,只有有未分配的分片时才会分配新分片,为了减少分片切换,前面几次尽可能少分,后面越来越多
-     *
-     * @return 需要本实例有变化的分配,新增或删除
-     */
-    protected SplitChanged getChangedSplitCount(List<ISplit> allSplits, List<SourceInstance> sourceInstances,
-        int splitCountInWorking, int ownerSplitCount) {
-        //int ownerSplitCount=source.ownerSplits().size();
-        int instanceCount = sourceInstances.size();
-        if (instanceCount == 0) {
-            instanceCount = 1;
-        }
-        int allSplitCount = allSplits.size();
-        int minSplitCount = allSplitCount / instanceCount;
-        int maxSplitCount = minSplitCount + (allSplitCount % instanceCount == 0 ? 0 : 1);
-        //已经是最大分片数了
-        if (ownerSplitCount == maxSplitCount) {
-            return null;
-        }
-        if (ownerSplitCount > maxSplitCount) {
-            int changeSplitCount = ownerSplitCount - maxSplitCount;
-            return new SplitChanged(changeSplitCount, false);
-        }
-        //分片已经全部在处理,当前分片也符合最小分片分配策略,不需要重新分配
-        if (splitCountInWorking == allSplitCount && ownerSplitCount >= minSplitCount) {
-            return null;
-        }
-        //如果还有未分配的分片,且当前实例还有分片的可行性,则分配分片
-        if (splitCountInWorking < allSplitCount && ownerSplitCount < maxSplitCount) {
-            int changeSplitCount = Math.min(maxSplitCount - ownerSplitCount, getMaxSplitCountInOneBalance());
-
-            return new SplitChanged(changeSplitCount, true);
-        }
-        return null;
-    }
-
-    @Override
-    public int getBalanceCount() {
-        return balanceCount;
-    }
-
-    /**
-     * 每次负载均衡最大的分片个数,目的是前几次,少分配分配,可能有实例在启动中,以免频繁切换分片,到后面实例都启动了,斤可能多分配分片
-     *
-     * @return
-     */
-    private int getMaxSplitCountInOneBalance() {
-        int balanceCount = getBalanceCount();
-        return (int) Math.pow(2, balanceCount - 1);
-    }
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/ISourceBalance.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/ISourceBalance.java
deleted file mode 100644
index b012b32..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/ISourceBalance.java
+++ /dev/null
@@ -1,60 +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.connectors.balance;
-
-import java.util.List;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-
-public interface ISourceBalance {
-
-    /**
-     * 做负载均衡
-
-     * @return
-     */
-    SplitChanged doBalance(List<ISplit> allSplits, List<ISplit> ownerSplits);
-
-    /**
-     * 从启动开始,做了多少次均衡
-     * @return
-     */
-    int getBalanceCount();
-
-
-
-    boolean getRemoveSplitLock();
-
-    void unLockRemoveSplitLock();
-
-    /**
-     * lock the split and hold it util the instance is shutdown or remove split
-     * @param split
-     * @return
-     */
-    boolean holdLockSplit(ISplit split);
-
-    /**
-     * unlock split lock
-     * @param split
-     */
-    void unlockSplit(ISplit split);
-
-
-    void setSourceIdentification(String sourceIdentification);
-
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/SplitChanged.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/SplitChanged.java
deleted file mode 100644
index c01c151..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/SplitChanged.java
+++ /dev/null
@@ -1,55 +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.connectors.balance;
-
-import java.util.List;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-
-public class SplitChanged {
-
-    protected int splitCount;//变动多分片个数
-    protected boolean isNewSplit;//是否新增,false是删除
-    protected List<ISplit> changedSplits;
-    public SplitChanged(int splitCount,boolean isNewSplit){
-        this.splitCount=splitCount;
-        this.isNewSplit=isNewSplit;
-    }
-
-    public int getSplitCount() {
-        return splitCount;
-    }
-
-    public void setSplitCount(int splitCount) {
-        this.splitCount = splitCount;
-    }
-
-    public boolean isNewSplit() {
-        return isNewSplit;
-    }
-
-    public void setNewSplit(boolean newSplit) {
-        isNewSplit = newSplit;
-    }
-
-    public List<ISplit> getChangedSplits() {
-        return changedSplits;
-    }
-
-    public void setChangedSplits(List<ISplit> changedSplits) {
-        this.changedSplits = changedSplits;
-    }
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/impl/LeaseBalanceImpl.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/impl/LeaseBalanceImpl.java
deleted file mode 100644
index dc504e5..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/impl/LeaseBalanceImpl.java
+++ /dev/null
@@ -1,144 +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.connectors.balance.impl;
-
-import com.google.auto.service.AutoService;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-import org.apache.rocketmq.streams.common.model.ServiceName;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.common.utils.RuntimeUtil;
-import org.apache.rocketmq.streams.connectors.balance.AbstractBalance;
-import org.apache.rocketmq.streams.connectors.balance.ISourceBalance;
-import org.apache.rocketmq.streams.connectors.source.SourceInstance;
-import org.apache.rocketmq.streams.lease.LeaseComponent;
-import org.apache.rocketmq.streams.lease.model.LeaseInfo;
-import org.apache.rocketmq.streams.lease.service.ILeaseService;
-
-@AutoService(ISourceBalance.class)
-@ServiceName(LeaseBalanceImpl.DB_BALANCE_NAME)
-public class LeaseBalanceImpl extends AbstractBalance {
-
-    private static final Log logger = LogFactory.getLog(LeaseBalanceImpl.class);
-
-    public static final String DB_BALANCE_NAME = "db_balance";
-    private static final String REMOVE_SPLIT_LOCK_NAME = "lock_remove_split";
-    private static final String SOURCE_LOCK_PREFIX = "SOURCE_";
-    private static final String SPLIT_LOCK_PREFIX = "SPLIT_";
-    protected transient LeaseComponent leaseComponent = LeaseComponent.getInstance();
-    protected transient String sourceIdentification;
-
-    protected int lockTimeSecond = 5;
-
-    public LeaseBalanceImpl(String sourceIdentification) {
-
-        this.sourceIdentification = sourceIdentification;
-    }
-
-    public LeaseBalanceImpl() {
-
-    }
-
-    @Override
-    protected List<ISplit> fetchWorkingSplits(List<ISplit> allSplits) {
-        List<LeaseInfo> leaseInfos = leaseComponent.getService().queryLockedInstanceByNamePrefix(SPLIT_LOCK_PREFIX + this.sourceIdentification, null);
-        logger.info(String.format("lease SPLIT_LOCK_PREFIX is %s, sourceIdentification is %s. ", SPLIT_LOCK_PREFIX, sourceIdentification));
-        if (leaseInfos == null) {
-            return new ArrayList<>();
-        }
-
-        Map<String, ISplit> allSplitMap = new HashMap<>();
-        for (ISplit split : allSplits) {
-            allSplitMap.put(split.getQueueId(), split);
-        }
-        List<ISplit> splits = new ArrayList<>();
-        for (LeaseInfo leaseInfo : leaseInfos) {
-            String leaseName = leaseInfo.getLeaseName();
-            String splitId = MapKeyUtil.getLast(leaseName);
-            splits.add(allSplitMap.get(splitId));
-        }
-        logger.info(String.format("working split is %s", Arrays.toString(splits.toArray())));
-        return splits;
-    }
-
-    @Override
-    protected List<SourceInstance> fetchSourceInstances() {
-        List<LeaseInfo> leaseInfos = leaseComponent.getService().queryLockedInstanceByNamePrefix(SOURCE_LOCK_PREFIX + sourceIdentification, null);
-        if (leaseInfos == null) {
-            return new ArrayList<>();
-        }
-        List<SourceInstance> sourceInstances = new ArrayList<>();
-        for (LeaseInfo leaseInfo : leaseInfos) {
-            String leaseName = leaseInfo.getLeaseName();
-            sourceInstances.add(new SourceInstance(leaseName));
-        }
-        return sourceInstances;
-    }
-
-    @Override
-    protected boolean holdLockSourceInstance() {
-        return holdLock(SOURCE_LOCK_PREFIX + sourceIdentification, RuntimeUtil.getDipperInstanceId());
-    }
-
-    @Override
-    protected void unlockSourceInstance() {
-        leaseComponent.getService().unlock(SOURCE_LOCK_PREFIX + sourceIdentification, RuntimeUtil.getDipperInstanceId());
-    }
-
-    @Override
-    public boolean holdLockSplit(ISplit split) {
-        return holdLock(SPLIT_LOCK_PREFIX + this.sourceIdentification, split.getQueueId());
-    }
-
-    @Override
-    public void unlockSplit(ISplit split) {
-        leaseComponent.getService().unlock(SPLIT_LOCK_PREFIX + this.sourceIdentification, split.getQueueId());
-
-    }
-
-    @Override
-    public boolean getRemoveSplitLock() {
-        return holdLock(this.sourceIdentification, REMOVE_SPLIT_LOCK_NAME);
-    }
-
-    @Override
-    public void unLockRemoveSplitLock() {
-        leaseComponent.getService().unlock(this.sourceIdentification, REMOVE_SPLIT_LOCK_NAME);
-    }
-
-    public String getSourceIdentification() {
-        return sourceIdentification;
-    }
-
-    @Override
-    public void setSourceIdentification(String sourceIdentification) {
-        this.sourceIdentification = sourceIdentification;
-    }
-
-    protected boolean holdLock(String name, String lockName) {
-        ILeaseService leaseService = leaseComponent.getService();
-        boolean success = leaseService.holdLock(name, lockName, lockTimeSecond);
-        return success;
-    }
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/PullMessage.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/PullMessage.java
deleted file mode 100644
index 9bf3480..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/PullMessage.java
+++ /dev/null
@@ -1,50 +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.connectors.model;
-
-import org.apache.rocketmq.streams.common.context.MessageOffset;
-
-public class PullMessage<T> {
-    protected T message;
-    protected MessageOffset messageOffset;
-
-    public T getMessage() {
-        return message;
-    }
-
-    public void setMessage(T message) {
-        this.message = message;
-    }
-
-    public MessageOffset getMessageOffset() {
-        return messageOffset;
-    }
-
-    public void setMessageOffset(MessageOffset messageOffset) {
-        this.messageOffset = messageOffset;
-    }
-    /**
-     * 获取offset字符串,通过.把主offset和子offset串接在一起
-     * @return
-     */
-    public String getOffsetStr(){
-       return this.messageOffset.getOffsetStr();
-    }
-    public String getMainOffset() {
-        return messageOffset.getMainOffset();
-    }
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/ReaderStatus.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/ReaderStatus.java
deleted file mode 100644
index a4889b5..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/model/ReaderStatus.java
+++ /dev/null
@@ -1,120 +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.connectors.model;
-
-import java.util.Date;
-import java.util.List;
-import org.apache.rocketmq.streams.common.model.Entity;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
-
-/**
- * @description
- */
-public class ReaderStatus extends Entity {
-
-    /**
-     * 查询单个readerStatus
-     */
-    static final String queryReaderStatusByUK = "select * from reader_status where source_name = '%s' and reader_name = '%s' and is_finished = 1";
-
-    static final String queryReaderStatusList = "select * from reader_status where source_name = '%s' and is_finished = 1";
-
-    static final String clearReaderStatus = "update reader_status set gmt_modified = now(), is_finished = -1 where source_name = '%s' and reader_name = '%s'";
-
-    String sourceName;
-
-    String readerName;
-
-    int isFinished;
-
-    int totalReader;
-
-    public String getReaderName() {
-        return readerName;
-    }
-
-    public void setReaderName(String readerName) {
-        this.readerName = readerName;
-    }
-
-    public int getIsFinished() {
-        return isFinished;
-    }
-
-    public void setIsFinished(int isFinished) {
-        this.isFinished = isFinished;
-    }
-
-    public int getTotalReader() {
-        return totalReader;
-    }
-
-    public void setTotalReader(int totalReader) {
-        this.totalReader = totalReader;
-    }
-
-    public String getSourceName() {
-        return sourceName;
-    }
-
-    public void setSourceName(String sourceName) {
-        this.sourceName = sourceName;
-    }
-
-    @Override
-    public String toString() {
-        return "ReaderStatus{" +
-            "id=" + id +
-            ", gmtCreate=" + gmtCreate +
-            ", gmtModified=" + gmtModified +
-            ", sourceName='" + sourceName + '\'' +
-            ", readerName='" + readerName + '\'' +
-            ", isFinished=" + isFinished +
-            ", totalReader=" + totalReader +
-            '}';
-    }
-
-    public static ReaderStatus queryReaderStatusByUK(String sourceName, String readerName) {
-        String sql = String.format(queryReaderStatusByUK, sourceName, readerName);
-        ReaderStatus readerStatus = ORMUtil.queryForObject(sql, null, ReaderStatus.class);
-        return readerStatus;
-    }
-
-    public static List<ReaderStatus> queryReaderStatusListBySourceName(String sourceName) {
-        String sql = String.format(queryReaderStatusList, sourceName);
-        List<ReaderStatus> readerStatusList = ORMUtil.queryForList(sql, null, ReaderStatus.class);
-        return readerStatusList;
-    }
-
-    public static void clearReaderStatus(String sourceName, String readerName) {
-        String sql = String.format(clearReaderStatus, sourceName, readerName);
-        ORMUtil.executeSQL(sql, null);
-    }
-
-    public static ReaderStatus create(String sourceName, String readerName, int isFinished, int totalReader) {
-
-        ReaderStatus readerStatus = new ReaderStatus();
-        readerStatus.setSourceName(sourceName);
-        readerStatus.setReaderName(readerName);
-        readerStatus.setIsFinished(isFinished);
-        readerStatus.setTotalReader(totalReader);
-        readerStatus.setGmtCreate(new Date());
-        readerStatus.setGmtModified(new Date());
-        return readerStatus;
-
-    }
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/DBScanReader.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/DBScanReader.java
deleted file mode 100644
index 268e891..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/DBScanReader.java
+++ /dev/null
@@ -1,269 +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.connectors.reader;
-
-import com.alibaba.fastjson.JSON;
-import com.alibaba.fastjson.JSONObject;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.channel.source.ISource;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-import org.apache.rocketmq.streams.common.component.AbstractComponent;
-import org.apache.rocketmq.streams.common.context.MessageOffset;
-import org.apache.rocketmq.streams.common.utils.ThreadUtil;
-import org.apache.rocketmq.streams.connectors.IBoundedSource;
-import org.apache.rocketmq.streams.connectors.IBoundedSourceReader;
-import org.apache.rocketmq.streams.connectors.model.PullMessage;
-import org.apache.rocketmq.streams.connectors.model.ReaderStatus;
-import org.apache.rocketmq.streams.connectors.source.CycleDynamicMultipleDBScanSource;
-import org.apache.rocketmq.streams.db.driver.DriverBuilder;
-import org.apache.rocketmq.streams.db.driver.JDBCDriver;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
-
-/**
- * @description
- */
-public class DBScanReader implements ISplitReader, IBoundedSourceReader, Serializable {
-
-    private static final long serialVersionUID = 8172403250050893288L;
-    private static final Log logger = LogFactory.getLog(DBScanReader.class);
-    static final String sqlTemplate = "select * from %s where id >= %d and id < %d";
-
-    //是否完成了source的call back调用
-    transient volatile boolean isFinishedCall = false;
-    ISource iSource;
-    String url;
-    String userName;
-    String password;
-    String tableName;
-    int batchSize;
-    long offset;
-    long offsetStart;
-    long offsetEnd;
-    long maxOffset;
-    long minOffset;
-    ISplit iSplit;
-    transient List<PullMessage> pullMessages;
-    volatile boolean interrupt = false;
-    volatile boolean isClosed = false;
-
-    public String getUrl() {
-        return url;
-    }
-
-    public void setUrl(String url) {
-        this.url = url;
-    }
-
-    public String getUserName() {
-        return userName;
-    }
-
-    public void setUserName(String userName) {
-        this.userName = userName;
-    }
-
-    public String getPassword() {
-        return password;
-    }
-
-    public void setPassword(String password) {
-        this.password = password;
-    }
-
-    public String getTableName() {
-        return tableName;
-    }
-
-    public void setTableName(String tableName) {
-        this.tableName = tableName;
-    }
-
-    public int getBatchSize() {
-        return batchSize;
-    }
-
-    public void setBatchSize(int batchSize) {
-        this.batchSize = batchSize;
-    }
-
-    public ISplit getISplit() {
-        return iSplit;
-    }
-
-    public void setISplit(ISplit iSplit) {
-        this.iSplit = iSplit;
-    }
-
-    public DBScanReader() {
-
-    }
-
-    transient ThreadLocal<JDBCDriver> threadLocal = new ThreadLocal<JDBCDriver>() {
-
-        @Override
-        public JDBCDriver initialValue() {
-            logger.info(String.format("%s initial jdbcDriver. ", Thread.currentThread().getName()));
-            return DriverBuilder.createDriver(AbstractComponent.DEFAULT_JDBC_DRIVER, url, userName, password);
-        }
-
-    };
-
-    @Override
-    public void open(ISplit split) {
-        this.iSplit = split;
-        JDBCDriver jdbcDriver = threadLocal.get();
-        Map<String, Object> range = jdbcDriver.queryOneRow("select min(id) as min_id, max(id) as max_id from " + tableName);
-        minOffset = Long.parseLong(String.valueOf(range.get("min_id")));
-        maxOffset = Long.parseLong(String.valueOf(range.get("max_id")));
-        offsetStart = minOffset;
-        offset = minOffset;
-        logger.info(String.format("table %s min id [ %d ],  max id [ %d ]", tableName, minOffset, maxOffset));
-        pullMessages = new ArrayList<>();
-    }
-
-    @Override
-    public boolean next() {
-        if (interrupt) {
-            return false;
-        }
-        if (isFinished()) {
-            finish();
-            ThreadUtil.sleep(10 * 1000);
-            return false;
-        }
-        JDBCDriver jdbcDriver = threadLocal.get();
-        offsetEnd = offsetStart + batchSize;
-        String batchQuery = String.format(sqlTemplate, tableName, offsetStart, offsetEnd);
-        logger.debug(String.format("execute sql : %s", batchQuery));
-        List<Map<String, Object>> resultData = jdbcDriver.queryForList(batchQuery);
-        offsetStart = offsetEnd;
-        pullMessages.clear();
-        for (Map<String, Object> r : resultData) {
-            PullMessage msg = new PullMessage();
-            JSONObject data = JSONObject.parseObject(JSON.toJSONString(r));
-            msg.setMessage(data);
-            offset = offset > Long.parseLong(data.getString("id")) ? offset : Long.parseLong(data.getString("id"));
-            msg.setMessageOffset(new MessageOffset(String.valueOf(offset), true));
-            pullMessages.add(msg);
-        }
-        return offsetStart - batchSize <= maxOffset;
-    }
-
-    @Override
-    public List<PullMessage> getMessage() {
-//        logger.info(String.format("output messages %d", pullMessages.size()));
-        return pullMessages;
-    }
-
-    @Override
-    public SplitCloseFuture close() {
-//        interrupt = true;
-        isClosed = true;
-        threadLocal.remove();
-        pullMessages = null;
-        return new SplitCloseFuture(this, iSplit);
-    }
-
-    @Override
-    public void seek(String cursor) {
-        if (cursor == null || cursor.trim().equals("")) {
-            cursor = "0";
-        }
-        offset = Long.parseLong(cursor);
-        if (offset < minOffset) {
-            offset = minOffset;
-        }
-        offsetStart = offset;
-        logger.info(String.format("split %s seek %d.", iSplit.getQueueId(), offset));
-    }
-
-    @Override
-    public String getProgress() {
-        return String.valueOf(offset);
-    }
-
-    @Override
-    public long getDelay() {
-        return maxOffset - offset;
-    }
-
-    @Override
-    public long getFetchedDelay() {
-        return 0;
-    }
-
-    @Override
-    public boolean isClose() {
-        return isClosed;
-    }
-
-    @Override
-    public ISplit getSplit() {
-        return iSplit;
-    }
-
-    @Override
-    public boolean isInterrupt() {
-        return interrupt;
-    }
-
-    @Override
-    public boolean interrupt() {
-        interrupt = true;
-        return true;
-    }
-
-    @Override
-    public boolean isFinished() {
-        return offsetStart > maxOffset;
-    }
-
-    @Override
-    public void finish() {
-        if (isFinishedCall) {
-            return;
-        }
-        pullMessages = null;
-        updateReaderStatus();
-        IBoundedSource tmp = (IBoundedSource) iSource;
-        tmp.boundedFinishedCallBack(this.iSplit);
-        isFinishedCall = true;
-    }
-
-    public ISource getISource() {
-        return iSource;
-    }
-
-    public void setISource(ISource iSource) {
-        this.iSource = iSource;
-    }
-
-    private final void updateReaderStatus() {
-        String sourceName = CycleDynamicMultipleDBScanSource.createKey(this.getISource());
-        int finish = Integer.valueOf(1);
-        int total = ((CycleDynamicMultipleDBScanSource) iSource).getTotalReader();
-        ReaderStatus readerStatus = ReaderStatus.create(sourceName, iSplit.getQueueId(), finish, total);
-        logger.info(String.format("create reader status %s.", readerStatus));
-        ORMUtil.batchReplaceInto(readerStatus);
-    }
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/ISplitReader.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/ISplitReader.java
deleted file mode 100644
index 6b377cf..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/ISplitReader.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.connectors.reader;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.List;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-import org.apache.rocketmq.streams.connectors.model.PullMessage;
-
-public interface ISplitReader extends Serializable {
-
-    /**
-     * Open.
-     *
-     * @param split the split
-     * @throws IOException the io exception
-     */
-    void open(ISplit split);
-
-    /**
-     * Next boolean.
-     *
-     * @return the boolean
-     * @throws IOException          the io exception
-     * @throws InterruptedException the interrupted exception
-     */
-    boolean next();
-
-    /**
-     * Gets message.
-     *
-     * @return the message
-     */
-    List<PullMessage> getMessage();
-
-    /**
-     * Close.
-     *
-     * @throws IOException the io exception
-     */
-    SplitCloseFuture close();
-
-    /**
-     * Seek.
-     *
-     * @param cursor the cursor
-     * @throws IOException the io exception
-     */
-    void seek(String cursor);
-
-    /**
-     * Gets progress.
-     *
-     * @return the progress
-     * @throws IOException the io exception
-     */
-    String getProgress();
-
-    /**
-     * Get message delay (millseconds)
-     *
-     * @return delay
-     */
-    long getDelay();
-
-    /**
-     * Get message delay (millseconds) from being fetched
-     *
-     * @return delay
-     */
-    long getFetchedDelay();
-
-    boolean isClose();
-
-    ISplit getSplit();
-
-    boolean isInterrupt();
-
-    boolean interrupt();
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/SplitCloseFuture.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/SplitCloseFuture.java
deleted file mode 100644
index b28748b..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/reader/SplitCloseFuture.java
+++ /dev/null
@@ -1,83 +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.connectors.reader;
-
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-
-public class SplitCloseFuture implements Future<Boolean> {
-
-    protected ISplitReader reader;
-    protected ISplit split;
-
-    public SplitCloseFuture(ISplitReader reader, ISplit split) {
-        this.reader = reader;
-        this.split = split;
-    }
-
-    @Override
-    public boolean cancel(boolean mayInterruptIfRunning) {
-        return false;
-    }
-
-    @Override
-    public boolean isCancelled() {
-        return false;
-    }
-
-    @Override
-    public boolean isDone() {
-        return reader.isClose();
-    }
-
-    @Override
-    public Boolean get() throws InterruptedException, ExecutionException {
-        synchronized (reader) {
-            reader.wait();
-        }
-        return reader.isClose();
-    }
-
-    @Override
-    public Boolean get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
-        synchronized (reader) {
-            long time = timeout;
-            if (unit == TimeUnit.SECONDS) {
-                time = time * 1000;
-            } else if (unit == TimeUnit.MINUTES) {
-                time = time * 1000 * 60;
-            } else if (unit == TimeUnit.HOURS) {
-                time = time * 1000 * 60 * 60;
-            } else {
-                throw new RuntimeException("can not support this timeout, expect less hour " + timeout + " the unit is " + unit);
-            }
-            reader.wait(time);
-        }
-        return reader.isClose();
-    }
-
-    public ISplitReader getReader() {
-        return reader;
-    }
-
-    public ISplit getSplit() {
-        return split;
-    }
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/AbstractPullSource.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/AbstractPullSource.java
deleted file mode 100644
index fb09a6b..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/AbstractPullSource.java
+++ /dev/null
@@ -1,272 +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.connectors.source;
-
-import com.alibaba.fastjson.JSON;
-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.Set;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import org.apache.commons.lang3.concurrent.BasicThreadFactory;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.channel.source.AbstractSource;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-import org.apache.rocketmq.streams.common.checkpoint.CheckPoint;
-import org.apache.rocketmq.streams.common.checkpoint.CheckPointManager;
-import org.apache.rocketmq.streams.common.context.Message;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.connectors.balance.ISourceBalance;
-import org.apache.rocketmq.streams.connectors.balance.SplitChanged;
-import org.apache.rocketmq.streams.connectors.balance.impl.LeaseBalanceImpl;
-import org.apache.rocketmq.streams.connectors.model.PullMessage;
-import org.apache.rocketmq.streams.connectors.reader.ISplitReader;
-import org.apache.rocketmq.streams.connectors.reader.SplitCloseFuture;
-import org.apache.rocketmq.streams.serviceloader.ServiceLoaderComponent;
-
-public abstract class AbstractPullSource extends AbstractSource implements IPullSource<AbstractSource> {
-
-    private static final Log logger = LogFactory.getLog(AbstractPullSource.class);
-
-    protected transient ISourceBalance balance;// balance interface
-    protected transient ScheduledExecutorService balanceExecutor;//schdeule balance
-    protected transient Map<String, ISplitReader> splitReaders = new HashMap<>();//owner split readers
-    protected transient Map<String, ISplit> ownerSplits = new HashMap<>();//working splits by the source instance
-
-    //可以有多种实现,通过名字选择不同的实现
-    protected String balanceName = LeaseBalanceImpl.DB_BALANCE_NAME;
-    //balance schedule time
-    protected int balanceTimeSecond = 10;
-    protected long pullIntervalMs;
-    transient CheckPointManager checkPointManager = new CheckPointManager();
-
-    @Override
-    protected boolean startSource() {
-        ServiceLoaderComponent serviceLoaderComponent = ServiceLoaderComponent.getInstance(ISourceBalance.class);
-        balance = (ISourceBalance) serviceLoaderComponent.getService().loadService(balanceName);
-        balance.setSourceIdentification(MapKeyUtil.createKey(getNameSpace(), getConfigureName()));
-        balanceExecutor = new ScheduledThreadPoolExecutor(1, new BasicThreadFactory.Builder().namingPattern("balance-task-%d").daemon(true).build());
-        List<ISplit> allSplits = fetchAllSplits();
-        SplitChanged splitChanged = balance.doBalance(allSplits, new ArrayList(ownerSplits.values()));
-        doSplitChanged(splitChanged);
-        balanceExecutor.scheduleWithFixedDelay(new Runnable() {
-            @Override
-            public void run() {
-                logger.info("balance running..... current splits is " + ownerSplits);
-                List<ISplit> allSplits = fetchAllSplits();
-                SplitChanged splitChanged = balance.doBalance(allSplits, new ArrayList(ownerSplits.values()));
-                doSplitChanged(splitChanged);
-            }
-        }, balanceTimeSecond, balanceTimeSecond, TimeUnit.SECONDS);
-        return true;
-    }
-
-    @Override
-    public Map<String, ISplit> getAllSplitMap() {
-        List<ISplit> splits = fetchAllSplits();
-        if (splits == null) {
-            return new HashMap<>();
-        }
-        Map<String, ISplit> splitMap = new HashMap<>();
-        for (ISplit split : splits) {
-            splitMap.put(split.getQueueId(), split);
-        }
-        return splitMap;
-    }
-
-    protected void doSplitChanged(SplitChanged splitChanged) {
-        if (splitChanged == null) {
-            return;
-        }
-        if (splitChanged.getSplitCount() == 0) {
-            return;
-        }
-        if (splitChanged.isNewSplit()) {
-            doSplitAddition(splitChanged.getChangedSplits());
-        } else {
-            doSplitRelease(splitChanged.getChangedSplits());
-        }
-    }
-
-    protected void doSplitAddition(List<ISplit> changedSplits) {
-        if (changedSplits == null) {
-            return;
-        }
-        Set<String> splitIds = new HashSet<>();
-        for (ISplit split : changedSplits) {
-            splitIds.add(split.getQueueId());
-        }
-        addNewSplit(splitIds);
-        for (ISplit split : changedSplits) {
-            ISplitReader reader = createSplitReader(split);
-            reader.open(split);
-            reader.seek(loadSplitOffset(split));
-            splitReaders.put(split.getQueueId(), reader);
-            this.ownerSplits.put(split.getQueueId(), split);
-            logger.info("start next");
-            Thread thread = new Thread(new Runnable() {
-                long mLastCheckTime = System.currentTimeMillis();
-
-                @Override
-                public void run() {
-                    logger.info("start running");
-                    while (reader.isInterrupt() == false) {
-                        if (reader.next()) {
-                            List<PullMessage> messages = reader.getMessage();
-                            if (messages != null) {
-                                for (PullMessage pullMessage : messages) {
-                                    String queueId = split.getQueueId();
-                                    String offset = pullMessage.getOffsetStr();
-                                    JSONObject msg = createJson(pullMessage.getMessage());
-                                    Message message = createMessage(msg, queueId, offset, false);
-                                    message.getHeader().setOffsetIsLong(pullMessage.getMessageOffset().isLongOfMainOffset());
-                                    executeMessage(message);
-                                }
-                            }
-                        }
-                        long curTime = System.currentTimeMillis();
-                        if (curTime - mLastCheckTime > getCheckpointTime()) {
-                            sendCheckpoint(reader.getSplit().getQueueId());
-                            mLastCheckTime = curTime;
-                        }
-                        try {
-                            Thread.sleep(pullIntervalMs);
-                        } catch (InterruptedException e) {
-                            e.printStackTrace();
-                        }
-
-                    }
-                    try {
-                        Thread.sleep(10);
-                    } catch (InterruptedException e) {
-                        e.printStackTrace();
-                    }
-                    Set<String> removeSplits = new HashSet<>();
-                    removeSplits.add(reader.getSplit().getQueueId());
-                    removeSplit(removeSplits);
-                    balance.unlockSplit(split);
-                    reader.close();
-                    synchronized (reader) {
-                        reader.notifyAll();
-                    }
-
-                }
-            });
-            thread.setName("reader-task-" + reader.getSplit().getQueueId());
-            thread.start();
-        }
-
-    }
-
-    @Override
-    public String loadSplitOffset(ISplit split) {
-        String offset = null;
-        CheckPoint<String> checkPoint = checkPointManager.recover(this, split);
-        if (checkPoint != null) {
-            offset = JSON.parseObject(checkPoint.getData()).getString("offset");
-        }
-        return offset;
-    }
-
-    protected abstract ISplitReader createSplitReader(ISplit split);
-
-    protected void doSplitRelease(List<ISplit> changedSplits) {
-        boolean success = balance.getRemoveSplitLock();
-        if (!success) {
-            return;
-        }
-        try {
-            List<SplitCloseFuture> closeFutures = new ArrayList<>();
-            for (ISplit split : changedSplits) {
-                ISplitReader reader = this.splitReaders.get(split.getQueueId());
-                if (reader == null) {
-                    continue;
-                }
-                SplitCloseFuture future = reader.close();
-                closeFutures.add(future);
-            }
-            for (SplitCloseFuture future : closeFutures) {
-                try {
-                    future.get();
-                    this.splitReaders.remove(future.getSplit().getQueueId());
-                    this.ownerSplits.remove(future.getSplit().getQueueId());
-                } catch (InterruptedException e) {
-                    e.printStackTrace();
-                } catch (ExecutionException e) {
-                    e.printStackTrace();
-                }
-            }
-
-        } finally {
-            balance.unLockRemoveSplitLock();
-        }
-
-    }
-
-    @Override
-    public boolean supportNewSplitFind() {
-        return true;
-    }
-
-    @Override
-    public boolean supportRemoveSplitFind() {
-        return true;
-    }
-
-    @Override
-    public boolean supportOffsetRest() {
-        return true;
-    }
-
-    @Override
-    public Long getPullIntervalMs() {
-        return pullIntervalMs;
-    }
-
-    public String getBalanceName() {
-        return balanceName;
-    }
-
-    public void setBalanceName(String balanceName) {
-        this.balanceName = balanceName;
-    }
-
-    public int getBalanceTimeSecond() {
-        return balanceTimeSecond;
-    }
-
-    public void setBalanceTimeSecond(int balanceTimeSecond) {
-        this.balanceTimeSecond = balanceTimeSecond;
-    }
-
-    public void setPullIntervalMs(long pullIntervalMs) {
-        this.pullIntervalMs = pullIntervalMs;
-    }
-
-    @Override
-    public List<ISplit> ownerSplits() {
-        return new ArrayList(ownerSplits.values());
-    }
-
-}
\ No newline at end of file
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/CycleDynamicMultipleDBScanSource.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/CycleDynamicMultipleDBScanSource.java
deleted file mode 100644
index 561b48f..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/CycleDynamicMultipleDBScanSource.java
+++ /dev/null
@@ -1,213 +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.connectors.source;
-
-import com.alibaba.fastjson.JSONObject;
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.channel.source.AbstractSource;
-import org.apache.rocketmq.streams.common.channel.source.ISource;
-import org.apache.rocketmq.streams.common.channel.source.systemmsg.ChangeTableNameMessage;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-import org.apache.rocketmq.streams.common.context.Message;
-import org.apache.rocketmq.streams.common.metadata.MetaDataUtils;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.common.utils.ThreadUtil;
-import org.apache.rocketmq.streams.connectors.IBoundedSource;
-import org.apache.rocketmq.streams.connectors.model.ReaderStatus;
-import org.apache.rocketmq.streams.connectors.reader.ISplitReader;
-import org.apache.rocketmq.streams.connectors.source.filter.CycleSchedule;
-import org.apache.rocketmq.streams.connectors.source.filter.CycleScheduleFilter;
-import org.apache.rocketmq.streams.db.CycleSplit;
-
-/**
- * @description
- */
-public class CycleDynamicMultipleDBScanSource extends DynamicMultipleDBScanSource implements IBoundedSource, Serializable {
-
-    private static final long serialVersionUID = 6840988298037061128L;
-    private static final Log logger = LogFactory.getLog(CycleDynamicMultipleDBScanSource.class);
-
-    Map<String, Boolean> initReaderMap = new ConcurrentHashMap<>();
-    CycleSchedule.Cycle cycle;
-    transient AtomicInteger size = new AtomicInteger(0);
-
-    public CycleDynamicMultipleDBScanSource() {
-        super();
-    }
-
-    public CycleDynamicMultipleDBScanSource(CycleSchedule.Cycle cycle) {
-        super();
-        this.cycle = cycle;
-    }
-
-    public AtomicInteger getSize() {
-        return size;
-    }
-
-    public void setSize(AtomicInteger size) {
-        this.size = size;
-    }
-
-    /**
-     * @return
-     */
-    //todo
-    @Override
-    public synchronized List<ISplit> fetchAllSplits() {
-
-        if (this.filter == null) {
-            filter = new CycleScheduleFilter(cycle.getAllPattern());
-        }
-
-        //如果还是当前周期, 已经完成全部分区的加载, 则不在加载
-        if (size.get() == cycle.getCycleCount()) {
-            return splits;
-        }
-        String sourceName = createKey(this);
-        List<String> tableNames = MetaDataUtils.listTableNameByPattern(url, userName, password, logicTableName + "%");
-
-        logger.info(String.format("load all logic table : %s", Arrays.toString(tableNames.toArray())));
-        Iterator<String> it = tableNames.iterator();
-        while (it.hasNext()) {
-            String s = it.next();
-            String suffix = s.replace(logicTableName + "_", "");
-            if (filter.filter(sourceName, logicTableName, suffix)) {
-                logger.info(String.format("filter add %s", s));
-                CycleSplit split = new CycleSplit();
-                split.setLogicTableName(logicTableName);
-                split.setSuffix(suffix);
-                split.setCyclePeriod(cycle.getCycleDateStr());
-                String splitId = split.getQueueId();
-                if (initReaderMap.get(splitId) == null) {
-                    initReaderMap.put(splitId, false);
-                    splits.add(split);
-                    size.incrementAndGet();
-                }
-            } else {
-                logger.info(String.format("filter remove %s", s));
-                it.remove();
-            }
-        }
-
-        this.tableNames = tableNames;
-        return splits;
-    }
-
-    public Map<String, Boolean> getInitReaderMap() {
-        return initReaderMap;
-    }
-
-    public void setInitReaderMap(Map<String, Boolean> initReaderMap) {
-        this.initReaderMap = initReaderMap;
-    }
-
-    @Override
-    public void finish() {
-        super.finish();
-        for (Map.Entry<String, Boolean> entry : initReaderMap.entrySet()) {
-            String key = entry.getKey();
-            Boolean value = entry.getValue();
-            if (value == false) {
-                logger.error(String.format("split[%s] reader is not finish, exit with error. ", key));
-            }
-        }
-        this.initReaderMap.clear();
-        this.initReaderMap = null;
-        splits.clear();
-        splits = null;
-    }
-
-    @Override
-    public boolean isFinished() {
-        List<ReaderStatus> readerStatuses = ReaderStatus.queryReaderStatusListBySourceName(createKey(this));
-        if (readerStatuses == null) {
-            return false;
-        }
-        return readerStatuses.size() == size.get();
-    }
-
-    @Override
-    protected ISplitReader createSplitReader(ISplit iSplit) {
-        return super.createSplitReader(iSplit);
-    }
-
-    private void sendChangeTableNameMessage() {
-        logger.info(String.format("start send change table name message."));
-        ChangeTableNameMessage changeTableNameMessage = new ChangeTableNameMessage();
-        changeTableNameMessage.setScheduleCycle(cycle.getCycleDateStr());
-        Message message = createMessage(new JSONObject(), null, null, false);
-        message.setSystemMessage(changeTableNameMessage);
-        message.getHeader().setSystemMessage(true);
-        executeMessage(message);
-        logger.info(String.format("finish send change table name message."));
-    }
-
-    @Override
-    public synchronized void boundedFinishedCallBack(ISplit iSplit) {
-        this.initReaderMap.put(iSplit.getQueueId(), true);
-        logger.info(String.format("current map is %s, key is %s. ", initReaderMap, iSplit.getQueueId()));
-        if (statusCheckerStart.compareAndSet(false, true)) {
-            Thread thread = new Thread(new Runnable() {
-                @Override
-                public void run() {
-                    while (!isFinished()) {
-                        ThreadUtil.sleep(3 * 1000);
-                    }
-                    logger.info(String.format("source will be closed."));
-                    sendChangeTableNameMessage(); //下发修改name的消息
-                    ThreadUtil.sleep(1 * 1000);
-                    finish();
-                }
-
-            });
-            thread.setName(createKey(this) + "_callback");
-            thread.start();
-        }
-    }
-
-    public CycleSchedule.Cycle getCycle() {
-        return cycle;
-    }
-
-    public void setCycle(CycleSchedule.Cycle cycle) {
-        this.cycle = cycle;
-    }
-
-    @Override
-    public String createCheckPointName() {
-        return super.createCheckPointName();
-    }
-
-    public synchronized int getTotalReader() {
-        return size.get();
-    }
-
-    public static String createKey(ISource iSource) {
-        AbstractSource source = (AbstractSource) iSource;
-        CycleSchedule.Cycle cycle = ((CycleDynamicMultipleDBScanSource) iSource).getCycle();
-        return MapKeyUtil.createKey(source.getNameSpace(), source.getGroupName(), source.getConfigureName(), source.getTopic(), cycle.getCycleDateStr());
-    }
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/DynamicMultipleDBScanSource.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/DynamicMultipleDBScanSource.java
deleted file mode 100644
index ea2a118..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/DynamicMultipleDBScanSource.java
+++ /dev/null
@@ -1,190 +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.connectors.source;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-import org.apache.rocketmq.streams.common.metadata.MetaDataUtils;
-import org.apache.rocketmq.streams.connectors.reader.DBScanReader;
-import org.apache.rocketmq.streams.connectors.reader.ISplitReader;
-import org.apache.rocketmq.streams.connectors.source.filter.DataFormatPatternFilter;
-import org.apache.rocketmq.streams.connectors.source.filter.PatternFilter;
-import org.apache.rocketmq.streams.db.DynamicMultipleDBSplit;
-
-/**
- * @description DynamicMultipleDBScanSource
- */
-public class DynamicMultipleDBScanSource extends AbstractPullSource implements Serializable {
-
-    private static final long serialVersionUID = 3987103552547019739L;
-    private static final Log logger = LogFactory.getLog(DynamicMultipleDBScanSource.class);
-    public static final int DEFAULT_BATCH_SIZE = 50;
-    public static final int MAX_BATCH_SIZE = 100;
-
-    String url;
-    String userName;
-    String password;
-    String logicTableName;
-    String suffix;
-    int batchSize;
-    List<String> tableNames;
-    List<ISplit> splits;
-    transient volatile AtomicBoolean statusCheckerStart = new AtomicBoolean(false);
-
-    //todo
-    transient PatternFilter filter;
-
-    public DynamicMultipleDBScanSource() {
-        splits = new ArrayList<>();
-    }
-
-    @Override
-    protected boolean initConfigurable() {
-        setTopic(logicTableName);
-        return super.initConfigurable();
-    }
-
-    @Override
-    protected boolean isNotDataSplit(String queueId) {
-        return tableNames.contains(queueId);
-    }
-
-    @Override
-    protected ISplitReader createSplitReader(ISplit split) {
-
-        DBScanReader reader = new DBScanReader();
-        reader.setISplit(split);
-        reader.setUrl(url);
-        reader.setUserName(userName);
-        reader.setPassword(password);
-        reader.setTableName(String.valueOf(split.getQueue()));
-        int local = batchSize <= 0 ? DEFAULT_BATCH_SIZE : batchSize;
-        local = local > MAX_BATCH_SIZE ? MAX_BATCH_SIZE : local;
-        reader.setBatchSize(local);
-        reader.setISource(this);
-        logger.info(String.format("create reader for split %s", split.getQueueId()));
-        return reader;
-    }
-
-    @Override
-    public List<ISplit> fetchAllSplits() {
-
-        if (filter == null) {
-            filter = new DataFormatPatternFilter();
-        }
-
-//        String sourceName = createKey(this);
-
-        tableNames = MetaDataUtils.listTableNameByPattern(url, userName, password, logicTableName + "%");
-
-        logger.info(String.format("load all logic table : %s", Arrays.toString(tableNames.toArray())));
-
-        for (String s : tableNames) {
-            String suffix = s.replace(logicTableName + "_", "");
-            if (filter.filter(null, logicTableName, suffix)) {
-                logger.info(String.format("filter add %s", s));
-                DynamicMultipleDBSplit split = new DynamicMultipleDBSplit();
-                split.setLogicTableName(logicTableName);
-                split.setSuffix(suffix);
-                splits.add(split);
-            } else {
-                logger.info(String.format("filter remove %s", s));
-            }
-
-        }
-        return splits;
-    }
-
-    public String getUrl() {
-        return url;
-    }
-
-    public void setUrl(String url) {
-        this.url = url;
-    }
-
-    public String getUserName() {
-        return userName;
-    }
-
-    public void setUserName(String userName) {
-        this.userName = userName;
-    }
-
-    public String getPassword() {
-        return password;
-    }
-
-    public void setPassword(String password) {
-        this.password = password;
-    }
-
-    public String getLogicTableName() {
-        return logicTableName;
-    }
-
-    public void setLogicTableName(String logicTableName) {
-        this.logicTableName = logicTableName;
-    }
-
-    public String getSuffix() {
-        return suffix;
-    }
-
-    public void setSuffix(String suffix) {
-        this.suffix = suffix;
-    }
-
-    public int getBatchSize() {
-        return batchSize;
-    }
-
-    public void setBatchSize(int batchSize) {
-        this.batchSize = batchSize;
-    }
-
-    public List<String> getTableNames() {
-        return tableNames;
-    }
-
-    public void setTableNames(List<String> tableNames) {
-        this.tableNames = tableNames;
-    }
-
-    public List<ISplit> getSplits() {
-        return splits;
-    }
-
-    public void setSplits(List<ISplit> splits) {
-        this.splits = splits;
-    }
-
-    public PatternFilter getFilter() {
-        return filter;
-    }
-
-    public void setFilter(PatternFilter filter) {
-        this.filter = filter;
-    }
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/IPullSource.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/IPullSource.java
deleted file mode 100644
index 6733911..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/IPullSource.java
+++ /dev/null
@@ -1,60 +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.connectors.source;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import org.apache.rocketmq.streams.common.channel.source.ISource;
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
-
-/**
- * poll message,need balance
- */
-public interface IPullSource<T extends ISource> extends ISource<T> {
-
-    /**
-     * 拥有的分片格式
-     *
-     * @return
-     */
-    Collection<ISplit> ownerSplits();
-
-    /**
-     * get all split for the source
-     *
-     * @return
-     */
-    List<ISplit> fetchAllSplits();
-
-    /**
-     * get all split for the source
-     *
-     * @return
-     */
-    Map<String, ISplit> getAllSplitMap();
-
-    Long getPullIntervalMs();
-
-    /**
-     * get cusor from store
-     *
-     * @return
-     */
-    String loadSplitOffset(ISplit split);
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/SourceInstance.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/SourceInstance.java
deleted file mode 100644
index c0da5b6..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/SourceInstance.java
+++ /dev/null
@@ -1,37 +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.connectors.source;
-
-/**
- * i个消息队列的实例,一个实例i个
- */
-public class SourceInstance {
-    protected String sourceInstanceId;
-
-
-    public SourceInstance(String sourceInstanceId){
-        this.sourceInstanceId=sourceInstanceId;
-    }
-
-    public String getSourceInstanceId() {
-        return sourceInstanceId;
-    }
-
-    public void setSourceInstanceId(String sourceInstanceId) {
-        this.sourceInstanceId = sourceInstanceId;
-    }
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/AbstractPatternFilter.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/AbstractPatternFilter.java
deleted file mode 100644
index 0d5368a..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/AbstractPatternFilter.java
+++ /dev/null
@@ -1,38 +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.connectors.source.filter;
-
-import java.io.Serializable;
-
-/**
- * @description
- */
-public abstract class AbstractPatternFilter implements PatternFilter, Serializable {
-
-    private static final long serialVersionUID = 6500945777421871431L;
-
-    PatternFilter next;
-
-    public abstract boolean filter(String sourceName, String logicTableName, String tableName);
-
-
-    @Override
-    public PatternFilter setNext(PatternFilter filter) {
-        this.next = filter;
-        return this;
-    }
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/BoundedPatternFilter.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/BoundedPatternFilter.java
deleted file mode 100644
index c06de98..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/BoundedPatternFilter.java
+++ /dev/null
@@ -1,53 +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.connectors.source.filter;
-
-import java.io.Serializable;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.connectors.model.ReaderStatus;
-
-/**
- * @description 过滤掉已经完成的reader
- */
-@Deprecated
-public class BoundedPatternFilter extends AbstractPatternFilter implements Serializable {
-
-    static final Log logger = LogFactory.getLog(BoundedPatternFilter.class);
-
-    @Override
-    public boolean filter(String sourceName, String logicTableName, String tableName) {
-
-        ReaderStatus readerStatus = ReaderStatus.queryReaderStatusByUK(sourceName, logicTableName + "_" + tableName);
-        if (readerStatus != null) {
-            logger.info(String.format("filter sourceName %s, logicTableName %s, suffix %s. ", sourceName, logicTableName, tableName));
-            logger.info(String.format("query result %s", readerStatus.toString()));
-            return true;
-        }
-        if (next == null) {
-            return false;
-        }
-        return next.filter(sourceName, logicTableName, tableName);
-    }
-
-    @Override
-    public PatternFilter setNext(PatternFilter filter) {
-        super.setNext(filter);
-        return this;
-    }
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CyclePatternFilter.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CyclePatternFilter.java
deleted file mode 100644
index 3a0193f..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CyclePatternFilter.java
+++ /dev/null
@@ -1,173 +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.connectors.source.filter;
-
-import java.io.Serializable;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.List;
-
-/**
- * @description 用来做分区选取
- */
-public class CyclePatternFilter extends AbstractPatternFilter implements Serializable {
-
-    private static final long serialVersionUID = -5151597286296228754L;
-
-    public static final int INIT_CYCLE_VERSION = 0;
-
-    CyclePeriod cyclePeriod;
-
-    Date curCycleDateTime; //当前调度周期时间
-
-    long cycleId;
-
-    String firstStartTime; //当前最小时间
-
-    List<String> allPatterns;
-
-    String expression;
-
-    boolean isInit;
-
-    //历史数据读取时使用,表示比起当前相差多少个调度周期
-    final long cycleDiff;
-
-    //todo expr解析
-    public CyclePatternFilter(String expr, Date date) throws ParseException {
-        expression = expr;
-        cycleId = INIT_CYCLE_VERSION;
-        cyclePeriod = CyclePeriod.getInstance(expression);
-        curCycleDateTime = calCycleDateTime(date);
-        allPatterns = new ArrayList<>();
-        isInit = true;
-        if(cyclePeriod.isHistory){
-            Date tmp = cyclePeriod.getHisDate();
-            cycleDiff = curCycleDateTime.getTime()/1000 * 1000 - tmp.getTime()/1000*1000;
-        }else{
-            cycleDiff = 0;
-        }
-    }
-
-
-    /**
-     *
-     * @return 返回date格式的调度周期时间
-     */
-    private Date calCycleDateTime(Date date){
-        return cyclePeriod.format(date);
-    }
-
-    private long calCycle(Date date){
-        Date tmp = calCycleDateTime(date);
-        if(tmp.getTime()/1000 == curCycleDateTime.getTime()/1000){
-            return cycleId;
-        }
-        return nextCycle(tmp);
-    }
-
-    private long nextCycle(Date date){
-        curCycleDateTime = date;
-        cycleId++;
-        calAllPattern();
-        return cycleId;
-    }
-
-    private void calAllPattern(){
-        allPatterns.clear();
-        for(int i = 1; i <= cyclePeriod.getCycle(); i++){
-            long d = (curCycleDateTime.getTime()/1000)*1000 - i * cyclePeriod.getInterval() - cycleDiff;
-            String s = cyclePeriod.getDateFormat().format(new Date(d));
-            allPatterns.add(s);
-        }
-        firstStartTime = allPatterns.get(allPatterns.size() - 1);
-    }
-
-    public boolean isNextCycle(Date date){
-        if(isInit){
-            isInit = false;
-            calAllPattern();
-            return true;
-        }
-        long tmp = cycleId;
-        return calCycle(date) > tmp;
-    }
-
-    public List<String> getAllPatterns() {
-        return allPatterns;
-    }
-
-    public long getCycleId() {
-        return cycleId;
-    }
-
-    public Date getCurCycleDateTime(){
-        return curCycleDateTime;
-    }
-
-    public String getCurCycleDateTimeStr(){
-        return cyclePeriod.getDateFormat().format(curCycleDateTime);
-    }
-
-    public long getCycleDiff() {
-        return cycleDiff;
-    }
-
-    public long getCyclePeriodDiff(){
-        return cycleDiff/cyclePeriod.getInterval();
-    }
-
-    public int getCycle(){
-        return cyclePeriod.getCycle();
-    }
-
-    public String getFirstStartTime() {
-        return firstStartTime;
-    }
-
-    @Override
-    public boolean filter(String sourceName, String logicTableName, String tableName) {
-        return allPatterns.contains(tableName);
-    }
-
-
-
-    public static void main(String[] args) throws ParseException {
-
-        CyclePatternFilter cycle = new CyclePatternFilter("yyyyMMddHHmm - 15m", new Date());
-        System.out.println(cycle);
-
-        System.out.println(cycle.filter(null, null, "202109131650"));
-        System.out.println(cycle.filter(null, null, "20210902000000"));
-        System.out.println(cycle.filter(null, null, "20210908000000"));
-        System.out.println(cycle.filter(null, null, "20210910000000"));
-        System.out.println(cycle.filter(null, null, "20210909230000"));
-
-        System.out.println(new SimpleDateFormat("yyyyMMddHH").parse("2021090923"));
-        System.out.println(new SimpleDateFormat("yyyyMMddhhmmss").parse("20210909230000"));
-        System.out.println(new SimpleDateFormat("yyyyMMddHHmmss").parse("20210909100000"));
-        System.out.println(new SimpleDateFormat("yyyyMMddhhmmss").parse("20210909100000"));
-
-
-
-
-    }
-
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CyclePeriod.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CyclePeriod.java
deleted file mode 100644
index 4e6cdd6..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CyclePeriod.java
+++ /dev/null
@@ -1,222 +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.connectors.source.filter;
-
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * @Description
- */
-public enum CyclePeriod {
-
-    CYCLE_PERIOD_DATE() {
-        @Override
-        void argsParser(String expr) throws ParseException {
-            super.argsParser(expr);
-            interval = 24 * 3600 * 1000;
-            int length = expr.length();
-            if (length == 8 && checkFormat(expr, PatternFilter.yyyyMMdd)) {
-                format = PatternFilter.yyyyMMdd;
-            } else if (length == 14 && checkFormat(expr, PatternFilter.yyyyMMddHHmmss)) {
-                format = PatternFilter.yyyyMMddHHmmss;
-            } else {
-                throw new RuntimeException(String.format("unsupported format : %s, only support yyyymmdd 、 yyyymmddhhmmss.", expr));
-            }
-        }
-
-        @Override
-        Date format(Date strDate) {
-            Date date = new Date(strDate.getTime());
-            date.setHours(0);
-            date.setMinutes(0);
-            date.setSeconds(0);
-            return date;
-        }
-
-    },
-    CYCLE_PERIOD_HOUR() {
-        @Override
-        void argsParser(String expr) throws ParseException {
-            super.argsParser(expr);
-            interval = 3600 * 1000;
-
-            int length = expr.length();
-            if (length == 10 && checkFormat(expr, PatternFilter.yyyyMMddHH)) {
-                format = PatternFilter.yyyyMMddHH;
-            } else if (length == 14 && checkFormat(expr, PatternFilter.yyyyMMddHHmmss)) {
-                format = PatternFilter.yyyyMMddHHmmss;
-            } else {
-                throw new RuntimeException(String.format("unsupported format : %s, only support yyyymmdd 、 yyyymmddhhmmss.", expr));
-            }
-        }
-
-        @Override
-        Date format(Date strDate) {
-            Date date = new Date(strDate.getTime());
-            date.setMinutes(0);
-            date.setSeconds(0);
-            return date;
-        }
-
-    },
-    CYCLE_PERIOD_MINUTE() {
-        @Override
-        void argsParser(String expr) throws ParseException {
-            super.argsParser(expr);
-            interval = 60 * 1000;
-            int length = expr.length();
-            if (length == 12 && checkFormat(expr, PatternFilter.yyyyMMddHHmm)) {
-                format = PatternFilter.yyyyMMddHHmm;
-            } else if (length == 14 && checkFormat(expr, PatternFilter.yyyyMMddHHmmss)) {
-                format = PatternFilter.yyyyMMddHHmmss;
-            } else {
-                throw new RuntimeException(String.format("unsupported format : %s, only support yyyymmdd 、 yyyymmddhhmmss.", expr));
-            }
-        }
-
-        @Override
-        Date format(Date strDate) {
-            Date date = new Date(strDate.getTime());
-            date.setSeconds(0);
-            return date;
-        }
-
-    };
-
-    boolean isHistory = false;
-
-    long interval;
-
-    int cycle;
-
-    String format;
-
-    String hisDateString;
-
-    static final Log logger = LogFactory.getLog(CyclePeriod.class);
-
-    void argsParser(String expr) throws ParseException {
-        if (expr.matches("^\\d+$")) {
-            isHistory = true;
-            hisDateString = expr;
-        }
-    }
-
-    Date format(Date strDate) {
-        throw new RuntimeException(String.format("unsupported type.", strDate));
-    }
-
-    /**
-     * expr可能是yyyymmdd 或者 20210917
-     *
-     * @param expr
-     * @param format
-     * @return
-     */
-    final boolean checkFormat(String expr, String format) {
-
-        if (!isHistory) {
-            return expr.equalsIgnoreCase(format);
-        }
-
-        try {
-            new SimpleDateFormat(format).parse(expr);
-            return true;
-        } catch (ParseException e) {
-            logger.error(String.format("error format, expr is %s, format is %s.", expr, format));
-            e.printStackTrace();
-            return false;
-        }
-    }
-
-    public Date getHisDate() throws ParseException {
-        return getDateFormat().parse(hisDateString);
-    }
-
-    public SimpleDateFormat getDateFormat() {
-        return new SimpleDateFormat(format);
-    }
-
-    public long getInterval() {
-        return interval;
-    }
-
-    public boolean isHistory() {
-        return isHistory;
-    }
-
-    public void setHistory(boolean history) {
-        isHistory = history;
-    }
-
-    public void setInterval(long interval) {
-        this.interval = interval;
-    }
-
-    public int getCycle() {
-        return cycle;
-    }
-
-    public void setCycle(int cycle) {
-        this.cycle = cycle;
-    }
-
-    public void setFormat(String format) {
-        this.format = format;
-    }
-
-    public String getFormat() {
-        return format;
-    }
-
-    public String getHisDateString() {
-        return hisDateString;
-    }
-
-    public void setHisDateString(String hisDateString) {
-        this.hisDateString = hisDateString;
-    }
-
-    public static CyclePeriod getInstance(String expression) throws ParseException {
-
-        String[] str = expression.split("\\-");
-        assert str.length == 2 : String.format("expression error : %s. ", expression);
-        String expr = str[0].trim();
-        String tmp = str[1].trim().toLowerCase();
-        String cycleStr = tmp.substring(0, tmp.length() - 1);
-        int cycle = Integer.parseInt(cycleStr);
-        CyclePeriod cyclePeriod = null;
-        if (tmp.endsWith("d")) {
-            cyclePeriod = CYCLE_PERIOD_DATE;
-        } else if (tmp.endsWith("h")) {
-            cyclePeriod = CYCLE_PERIOD_HOUR;
-        } else if (tmp.endsWith("m")) {
-            cyclePeriod = CYCLE_PERIOD_MINUTE;
-        } else {
-            new RuntimeException(String.format("unsupported format : %s", expression));
-        }
-        cyclePeriod.argsParser(expr);
-        cyclePeriod.cycle = cycle;
-
-        return cyclePeriod;
-    }
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CycleSchedule.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CycleSchedule.java
deleted file mode 100644
index ba9a279..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CycleSchedule.java
+++ /dev/null
@@ -1,236 +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.connectors.source.filter;
-
-import java.io.Serializable;
-import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.rocketmq.streams.common.configurable.BasedConfigurable;
-
-/**
- * @description 用来做分区选取
- */
-public class CycleSchedule implements Serializable {
-
-    private static final long serialVersionUID = -5151597286296228754L;
-    public static final int INIT_CYCLE_VERSION = 0;
-    private static CycleSchedule INSTANCE;
-    CyclePeriod cyclePeriod;
-    AtomicLong cycleId = new AtomicLong(0);
-    String expression;
-    boolean isInit;
-    //历史数据读取时使用,表示比起当前相差多少个调度周期
-    final long cycleDiff;
-
-    public CycleSchedule(String expr, Date date) throws ParseException {
-        Date local = subMs(date);
-        expression = expr;
-        cycleId.set(INIT_CYCLE_VERSION);
-        cyclePeriod = CyclePeriod.getInstance(expression);
-        isInit = true;
-        if (cyclePeriod.isHistory) {
-            Date curCycleDateTime = calCycleDateTime(local);
-            Date tmp = subMs(cyclePeriod.getHisDate());
-            cycleDiff = curCycleDateTime.getTime() - tmp.getTime();
-        } else {
-            cycleDiff = 0;
-        }
-    }
-
-    /**
-     * 去掉毫秒时间戳
-     *
-     * @param date
-     * @return
-     */
-    private Date subMs(Date date) {
-        long time = date.getTime() / 1000 * 1000;
-        return new Date(time);
-    }
-
-    /**
-     * @return 返回date格式的调度周期时间
-     */
-    private Date calCycleDateTime(Date date) {
-        return cyclePeriod.format(date);
-    }
-
-    public Cycle nextCycle(Date date) {
-        Date local = subMs(date);
-        local = cyclePeriod.format(local);
-        if (isInit) {
-            isInit = false;
-        } else {
-            cycleId.incrementAndGet();
-        }
-        List<String> ret = calAllPattern(local);
-        Cycle cycle = new Cycle();
-        cycle.setCycleId(cycleId.get());
-        cycle.setAllPattern(ret);
-        cycle.setCycleDateStr(calCycleDateStr(local));
-        cycle.setCycleCount(cyclePeriod.getCycle());
-        cycle.setCurDateStr(cyclePeriod.getDateFormat().format(local));
-        cycle.setCycleDiff(cycleDiff);
-        return cycle;
-    }
-
-    private String calCycleDateStr(Date date) {
-        long d = date.getTime() - cycleDiff;
-        Date d1 = new Date(d);
-        return cyclePeriod.getDateFormat().format(d1);
-    }
-
-    private List<String> calAllPattern(Date date) {
-        List<String> allPatterns = new ArrayList<>();
-        for (int i = 1; i <= cyclePeriod.getCycle(); i++) {
-            long d = date.getTime() - i * cyclePeriod.getInterval() - cycleDiff;
-            String s = cyclePeriod.getDateFormat().format(new Date(d));
-            allPatterns.add(s);
-        }
-        return allPatterns;
-    }
-
-    public CyclePeriod getCyclePeriod() {
-        return cyclePeriod;
-    }
-
-    public void setCyclePeriod(CyclePeriod cyclePeriod) {
-        this.cyclePeriod = cyclePeriod;
-    }
-
-    public AtomicLong getCycleId() {
-        return cycleId;
-    }
-
-    public void setCycleId(AtomicLong cycleId) {
-        this.cycleId = cycleId;
-    }
-
-    public String getExpression() {
-        return expression;
-    }
-
-    public void setExpression(String expression) {
-        this.expression = expression;
-    }
-
-    public boolean isInit() {
-        return isInit;
-    }
-
-    public void setInit(boolean init) {
-        isInit = init;
-    }
-
-    public long getCycleDiff() {
-        return cycleDiff;
-    }
-
-    public static CycleSchedule getInstance(String expr, Date date) {
-        if (INSTANCE == null) {
-            synchronized (CycleSchedule.class) {
-                if (INSTANCE == null) {
-                    try {
-                        INSTANCE = new CycleSchedule(expr, date);
-                    } catch (ParseException e) {
-                        e.printStackTrace();
-                    }
-                }
-            }
-        }
-        return INSTANCE;
-    }
-
-    public static class Cycle extends BasedConfigurable implements Serializable {
-
-        private static final long serialVersionUID = 4842560538716388622L;
-        Long cycleId;
-        List<String> allPattern;
-        String cycleDateStr;
-        Integer cycleCount;
-        String curDateStr;
-        long cycleDiff;
-
-        public Integer getCycleCount() {
-            return cycleCount;
-        }
-
-        public void setCycleCount(Integer cycleCount) {
-            this.cycleCount = cycleCount;
-        }
-
-        public Cycle() {
-        }
-
-        public Long getCycleId() {
-            return cycleId;
-        }
-
-        public void setCycleId(Long cycleId) {
-            this.cycleId = cycleId;
-        }
-
-        public List<String> getAllPattern() {
-            return allPattern;
-        }
-
-        public void setAllPattern(List<String> allPattern) {
-            this.allPattern = allPattern;
-        }
-
-        public String getCycleDateStr() {
-            return cycleDateStr;
-        }
-
-        public void setCycleDateStr(String cycleDateStr) {
-            this.cycleDateStr = cycleDateStr;
-        }
-
-        public String getCurDateStr() {
-            return curDateStr;
-        }
-
-        public void setCurDateStr(String curDateStr) {
-            this.curDateStr = curDateStr;
-        }
-
-        public long getCycleDiff() {
-            return cycleDiff;
-        }
-
-        public void setCycleDiff(long cycleDiff) {
-            this.cycleDiff = cycleDiff;
-        }
-
-        @Override
-        public String toString() {
-            return "Cycle{" +
-                "cycleId=" + cycleId +
-                ", cycleDateStr='" + cycleDateStr + '\'' +
-                ", cycleCount=" + cycleCount +
-                ", curDateStr='" + curDateStr + '\'' +
-                ", cycleDiff=" + cycleDiff +
-                ", allPattern=" + Arrays.toString(allPattern.toArray()) +
-                '}';
-        }
-    }
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CycleScheduleFilter.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CycleScheduleFilter.java
deleted file mode 100644
index 507739d..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/CycleScheduleFilter.java
+++ /dev/null
@@ -1,37 +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.connectors.source.filter;
-
-import java.io.Serializable;
-import java.util.List;
-
-/**
- * @description
- */
-public class CycleScheduleFilter extends AbstractPatternFilter implements Serializable {
-
-    List<String> allPattern;
-
-    public CycleScheduleFilter(List<String> allPattern){
-        this.allPattern = allPattern;
-    }
-
-    @Override
-    public boolean filter(String sourceName, String logicTableName, String tableName) {
-        return allPattern.contains(tableName);
-    }
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/DataFormatPatternFilter.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/DataFormatPatternFilter.java
deleted file mode 100644
index 0cdc076..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/DataFormatPatternFilter.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.connectors.source.filter;
-
-import java.io.Serializable;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * @description
- */
-public class DataFormatPatternFilter extends AbstractPatternFilter implements Serializable {
-
-    private static final long serialVersionUID = 3604787588465242642L;
-
-    static final Log logger = LogFactory.getLog(DataFormatPatternFilter.class);
-
-    static final String yyyyMMddHHmmss = "yyyyMMddHHmmss";
-    static final String yyyyMMdd = "yyyyMMdd";
-    static final String yyyyMMddHH = "yyyyMMddHH";
-
-    SimpleDateFormat format1 = new SimpleDateFormat(yyyyMMdd);
-    SimpleDateFormat format2 = new SimpleDateFormat(yyyyMMddHH);
-    SimpleDateFormat format3 = new SimpleDateFormat(yyyyMMddHHmmss);
-
-    @Override
-    public boolean filter(String sourceName, String logicTableName, String tableNameSuffix) {
-
-        int len = tableNameSuffix.length();
-        boolean isFilter = false;
-
-        switch (len) {
-            case 8:
-                try {
-                    format1.parse(tableNameSuffix);
-                    isFilter = true;
-                } catch (ParseException e) {
-                    e.printStackTrace();
-                    isFilter = false;
-                }
-                break;
-            case 10:
-                try {
-                    format2.parse(tableNameSuffix);
-                    isFilter = true;
-                } catch (ParseException e) {
-                    e.printStackTrace();
-                    isFilter = false;
-                }
-                break;
-            case 14:
-                try {
-                    format3.parse(tableNameSuffix);
-                    isFilter = true;
-                } catch (ParseException e) {
-                    e.printStackTrace();
-                    isFilter = false;
-                }
-                break;
-        }
-
-        if (isFilter) {
-            logger.info(String.format("filter sourceName %s, logicTableName %s, suffix %s", sourceName, logicTableName, tableNameSuffix));
-            return true;
-        }
-        if (next != null) {
-            return next.filter(sourceName, logicTableName, tableNameSuffix);
-        }
-        return false;
-    }
-
-    @Override
-    public PatternFilter setNext(PatternFilter filter) {
-        super.setNext(filter);
-        return this;
-    }
-
-    public PatternFilter getNext() {
-        return next;
-    }
-
-    public static void main(String[] args) {
-        DataFormatPatternFilter filter = new DataFormatPatternFilter();
-//        System.out.println(filter.filter("20200101"));
-//        System.out.println(filter.filter("2020010101"));
-//        System.out.println(filter.filter("20200101010101"));
-
-    }
-
-}
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/PatternFilter.java b/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/PatternFilter.java
deleted file mode 100644
index 4236500..0000000
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/source/filter/PatternFilter.java
+++ /dev/null
@@ -1,41 +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.connectors.source.filter;
-
-/**
- * @description
- */
-public interface PatternFilter {
-
-    String yyyyMMddHHmmss = "yyyyMMddHHmmss";
-    String yyyyMMdd = "yyyyMMdd";
-    String yyyyMMddHH = "yyyyMMddHH";
-    String yyyyMMddHHmm = "yyyyMMddHHmm";
-
-
-    /**
-     * 根据sourceName和tableName判断是否符合
-     * @param sourceName
-     * @param tableName
-     * @return
-     */
-    boolean filter(String sourceName, String logicTableName, String tableName);
-
-    PatternFilter setNext(PatternFilter filter);
-
-
-}
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/DBDelegate.java b/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegate.java
deleted file mode 100644
index cc8e289..0000000
--- a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBDelegate.java
+++ /dev/null
@@ -1,26 +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;
-
-public interface DBDelegate {
-
-    public void init(String driver, String url, String userName,
-        String password);
-
-    public void init();
-}
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/DBType.java b/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBType.java
deleted file mode 100644
index 83e7978..0000000
--- a/rocketmq-streams-dbinit/src/main/java/org/apache/rocketmq/streams/dbinit/mysql/delegate/DBType.java
+++ /dev/null
@@ -1,23 +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;
-
-public class DBType {
-
-    public static final String DB_MYSQL = "MYSQL";
-}
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
index 8668e60..e69de29 100644
--- a/rocketmq-streams-dbinit/src/main/resources/tables_mysql_innodb.sql
+++ b/rocketmq-streams-dbinit/src/main/resources/tables_mysql_innodb.sql
@@ -1,204 +0,0 @@
-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,
-  `can_clear_resource` TINYINT(1) DEFAULT 0,
-  `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,
-  `can_clear_resource` TINYINT(1) DEFAULT 0,
-  `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',
-  `can_clear_resource` tinyint(1) 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-dim/pom.xml b/rocketmq-streams-dim/pom.xml
deleted file mode 100644
index d6a5a74..0000000
--- a/rocketmq-streams-dim/pom.xml
+++ /dev/null
@@ -1,51 +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">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.rocketmq</groupId>
-        <artifactId>rocketmq-streams</artifactId>
-        <version>1.0.2-preview-SNAPSHOT</version>
-    </parent>
-    <artifactId>rocketmq-streams-dim</artifactId>
-    <name>ROCKETMQ STREAMS :: dim</name>
-    <packaging>jar</packaging>
-    <properties>
-        <file_encoding>UTF-8</file_encoding>
-        <project.build.sourceEncoding>${file_encoding}</project.build.sourceEncoding>
-    </properties>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-filter</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-channel-http</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.aliyun</groupId>
-            <artifactId>aliyun-java-sdk-core</artifactId>
-            <version>3.5.0</version>
-        </dependency>
-    </dependencies>
-
-
-</project>
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/DimComponent.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/DimComponent.java
deleted file mode 100644
index fbe79f1..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/DimComponent.java
+++ /dev/null
@@ -1,62 +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.dim;
-
-import java.util.Properties;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.component.AbstractComponent;
-import org.apache.rocketmq.streams.common.component.ComponentCreator;
-import org.apache.rocketmq.streams.configurable.ConfigurableComponent;
-import org.apache.rocketmq.streams.dim.service.IDimService;
-import org.apache.rocketmq.streams.dim.service.impl.DimServiceImpl;
-
-public class DimComponent extends AbstractComponent<IDimService> {
-
-    private static final Log LOG = LogFactory.getLog(DimComponent.class);
-
-    // private transient Map<String, DBDim> nameListMap = new HashMap<>();
-
-    protected transient ConfigurableComponent configurableComponent;
-    private transient IDimService dimService;
-
-    public static DimComponent getInstance(String namespace) {
-        return ComponentCreator.getComponent(namespace, ComponentCreator.class);
-    }
-
-    @Override
-    protected boolean startComponent(String namespace) {
-        configurableComponent = ComponentCreator.getComponent(namespace, ConfigurableComponent.class);
-        dimService = new DimServiceImpl(configurableComponent);
-        return true;
-    }
-
-    @Override
-    protected boolean initProperties(Properties properties) {
-        return true;
-    }
-
-    @Override
-    public boolean stop() {
-        return true;
-    }
-
-    @Override
-    public IDimService getService() {
-        return dimService;
-    }
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/AbstractDimParser.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/AbstractDimParser.java
deleted file mode 100644
index c5e41f4..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/AbstractDimParser.java
+++ /dev/null
@@ -1,52 +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.dim.builder;
-
-import java.util.Properties;
-import org.apache.rocketmq.streams.common.metadata.MetaData;
-import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.dim.model.AbstractDim;
-
-public abstract class AbstractDimParser implements IDimSQLParser {
-    @Override
-    public AbstractDim parseDim(String namespace, Properties properties, MetaData metaData) {
-        AbstractDim dim = createDim(properties, metaData);
-        String cacheTTLMs = properties.getProperty("cacheTTLMs");
-        long pollingTime = 30;//默认更新时间是30分钟
-
-        if (StringUtil.isNotEmpty(cacheTTLMs)) {
-            pollingTime = (Long.valueOf(cacheTTLMs) / 1000 / 60);
-        }
-        dim.setNameSpace(namespace);
-        dim.setPollingTimeMinute(pollingTime);
-
-        String isLarge = properties.getProperty("isLarge");
-        if (isLarge == null || "false".equalsIgnoreCase(isLarge)) {
-            return dim;
-        }
-        dim.setLarge(Boolean.valueOf(isLarge));
-        String filePath = properties.getProperty("filePath");
-//        String tableName = metaData.getTableName();
-        if (filePath == null) {
-            throw new RuntimeException("if large table is true, must set file path args");
-        }
-        dim.setFilePath(filePath);
-        return dim;
-    }
-
-    protected abstract AbstractDim createDim(Properties properties, MetaData data);
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/DBDimSQLParser.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/DBDimSQLParser.java
deleted file mode 100644
index fbbb4dd..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/DBDimSQLParser.java
+++ /dev/null
@@ -1,84 +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.dim.builder;
-
-import com.google.auto.service.AutoService;
-import java.util.List;
-import java.util.Properties;
-import org.apache.rocketmq.streams.common.metadata.MetaData;
-import org.apache.rocketmq.streams.common.metadata.MetaDataField;
-import org.apache.rocketmq.streams.common.model.ServiceName;
-import org.apache.rocketmq.streams.dim.model.AbstractDim;
-import org.apache.rocketmq.streams.dim.model.DBDim;
-
-@AutoService(IDimSQLParser.class)
-@ServiceName(value = DBDimSQLParser.TYPE, aliasName = "rds")
-public class DBDimSQLParser extends AbstractDimParser {
-    public static final String TYPE = "db";
-
-    @Override protected AbstractDim createDim(Properties properties, MetaData metaData) {
-        String tableName = properties.getProperty("tableName");
-
-        /**
-         * 创建namelist,要起必须有pco rimary key,,否则抛出错误
-         */
-        String url = properties.getProperty("url");
-        String userName = properties.getProperty("userName");
-        String password = properties.getProperty("password");
-        String idFieldName = properties.getProperty("idFieldName");
-
-        DBDim dbNameList = new DBDim();
-        dbNameList.setUrl(url);
-        dbNameList.setUserName(userName);
-        dbNameList.setPassword(password);
-        dbNameList.setIdFieldName(idFieldName);
-
-        String selectFields = createSelectFields(metaData);
-        String sql = "select " + selectFields + " from " + tableName;
-        if (tableName.trim().toLowerCase().startsWith("from")) {
-            sql = "select " + selectFields + " " + tableName;
-        }
-
-        dbNameList.setSql(sql);
-        dbNameList.setSupportBatch(true);
-
-        return dbNameList;
-    }
-
-    /**
-     * 根据字段名,创建sql,最大加载10 w条数据,超过10w会被截断
-     *
-     * @param metaData
-     * @return
-     */
-    protected String createSelectFields(MetaData metaData) {
-        List<MetaDataField> metaDataFields = metaData.getMetaDataFields();
-        StringBuilder stringBuilder = new StringBuilder();
-        boolean isFirst = true;
-        for (MetaDataField field : metaDataFields) {
-            if (isFirst) {
-                isFirst = false;
-            } else {
-                stringBuilder.append(",");
-            }
-            stringBuilder.append(field.getFieldName());
-        }
-        String fields = stringBuilder.toString();
-        return fields;
-    }
-
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/DimBuilder.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/DimBuilder.java
deleted file mode 100644
index 45c45ce..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/DimBuilder.java
+++ /dev/null
@@ -1,94 +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.dim.builder;
-
-import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.dim.model.DBDim;
-
-public class DimBuilder {
-
-    private String url;
-    private String password;
-    private String userName;
-    protected Long pollingTime = 60L;                    // 同步数据的事件间隔
-    private String jdbcDriver = "com.mysql.jdbc.Driver";
-
-    public DimBuilder(String url, String userName, String password) {
-        this.url = url;
-        this.password = password;
-        this.userName = userName;
-    }
-
-    public DBDim createDim(String namespace, String name, String sqlOrTableName) {
-        DBDim nameList = new DBDim();
-        nameList.setNameSpace(namespace);
-        if (StringUtil.isNotEmpty(name)) {
-            nameList.setConfigureName(name);
-        }
-        String sql = sqlOrTableName;
-        if (sqlOrTableName.split(" ").length == 1) {
-            sql = "select * from " + sqlOrTableName + " limit 500000";
-        }
-        nameList.setSql(sql);
-        nameList.setJdbcdriver(jdbcDriver);
-        nameList.setPollingTimeMinute(pollingTime);
-        nameList.setUrl(url);
-        nameList.setUserName(userName);
-        nameList.setPassword(password);
-        return nameList;
-    }
-
-    public String getUrl() {
-        return url;
-    }
-
-    public void setUrl(String url) {
-        this.url = url;
-    }
-
-    public String getPassword() {
-        return password;
-    }
-
-    public void setPassword(String password) {
-        this.password = password;
-    }
-
-    public String getUserName() {
-        return userName;
-    }
-
-    public void setUserName(String userName) {
-        this.userName = userName;
-    }
-
-    public Long getPollingTime() {
-        return pollingTime;
-    }
-
-    public void setPollingTime(Long pollingTime) {
-        this.pollingTime = pollingTime;
-    }
-
-    public String getJdbcDriver() {
-        return jdbcDriver;
-    }
-
-    public void setJdbcDriver(String jdbcDriver) {
-        this.jdbcDriver = jdbcDriver;
-    }
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/FileDimSQLParser.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/FileDimSQLParser.java
deleted file mode 100644
index ae24a7f..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/FileDimSQLParser.java
+++ /dev/null
@@ -1,42 +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.dim.builder;
-
-import com.google.auto.service.AutoService;
-import java.util.Properties;
-import org.apache.rocketmq.streams.common.metadata.MetaData;
-import org.apache.rocketmq.streams.common.model.ServiceName;
-import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.dim.model.AbstractDim;
-import org.apache.rocketmq.streams.dim.model.FileDim;
-
-@AutoService(IDimSQLParser.class)
-@ServiceName(value = FileDimSQLParser.TYPE, aliasName = "FILE")
-public class FileDimSQLParser extends AbstractDimParser{
-    public static final String TYPE="file";
-
-    @Override
-    protected AbstractDim createDim(Properties properties, MetaData data) {
-        String filePath=properties.getProperty("filePath");
-        if(StringUtil.isEmpty(filePath)){
-            filePath=properties.getProperty("file_path");
-        }
-        FileDim fileDim=new FileDim();
-        fileDim.setFilePath(filePath);
-        return fileDim;
-    }
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/IDimSQLParser.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/IDimSQLParser.java
deleted file mode 100644
index e94f068..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/IDimSQLParser.java
+++ /dev/null
@@ -1,30 +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.dim.builder;
-
-import java.util.Properties;
-import org.apache.rocketmq.streams.common.metadata.MetaData;
-import org.apache.rocketmq.streams.dim.model.AbstractDim;
-
-/**
- * create dim by sql parser
- */
-public interface IDimSQLParser {
-
-    AbstractDim parseDim(String namespace,Properties properties, MetaData metaData);
-
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/SQLParserFactory.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/SQLParserFactory.java
deleted file mode 100644
index f6e04b7..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/builder/SQLParserFactory.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.dim.builder;
-
-import org.apache.rocketmq.streams.serviceloader.ServiceLoaderComponent;
-
-public class SQLParserFactory {
-
-    private static SQLParserFactory instance = new SQLParserFactory();
-
-    private SQLParserFactory() {
-    }
-
-    public static SQLParserFactory getInstance() {
-        return instance;
-    }
-
-    public IDimSQLParser create(String type) {
-        ServiceLoaderComponent serviceLoaderComponent = ServiceLoaderComponent.getInstance(IDimSQLParser.class);
-        return (IDimSQLParser) serviceLoaderComponent.loadService(type);
-    }
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/expression/InExpressionResource.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/expression/InExpressionResource.java
deleted file mode 100644
index c7a0ca4..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/expression/InExpressionResource.java
+++ /dev/null
@@ -1,78 +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.dim.function.expression;
-
-import java.util.Map;
-import org.apache.rocketmq.streams.common.context.AbstractContext;
-import org.apache.rocketmq.streams.common.context.IMessage;
-import org.apache.rocketmq.streams.dim.model.DBDim;
-import org.apache.rocketmq.streams.filter.function.expression.AbstractExpressionFunction;
-import org.apache.rocketmq.streams.filter.operator.expression.Expression;
-import org.apache.rocketmq.streams.script.annotation.Function;
-import org.apache.rocketmq.streams.script.annotation.FunctionMethod;
-import org.apache.rocketmq.streams.script.annotation.FunctionMethodAilas;
-
-@Function
-public class InExpressionResource extends AbstractExpressionFunction {
-
-    /**
-     * value格式 :resourceName.fieldName。如果只有单列,可以省略.fieldname
-     *
-     * @param expression
-     * @param context
-     * @return
-     */
-    @FunctionMethod(value = "in_expression_resouce", alias = "in_resouce")
-    @FunctionMethodAilas("in_expression_resouce(resourceName->(varName,functionName,value)&((varName,functionName,"
-        + "value)|(varName,functionName,value)))")
-    @Override
-    public Boolean doExpressionFunction(IMessage message, AbstractContext context, Expression expression) {
-        return match(message,context,expression, false);
-    }
-
-    protected Boolean match(IMessage message, AbstractContext context, Expression expression, boolean supportRegex) {
-        Object value = expression.getValue();
-        if (value == null) {
-            return null;
-        }
-        String valueStr = String.valueOf(value);
-        String[] valueArray = valueStr.split("->");
-        String dataResourceNamespace = expression.getNameSpace();
-        String dataResourceName = null;
-        String expressionStr = null;
-        if (valueArray.length == 2) {
-            dataResourceName = valueArray[0];
-            expressionStr = valueArray[1];
-        }
-        if (valueArray.length > 2) {
-            dataResourceNamespace = valueArray[0];
-            dataResourceName = valueArray[1];
-            expressionStr = valueArray[2];
-        }
-
-        DBDim dataResource =
-            (DBDim)context.getConfigurableService().queryConfigurableByIdent(DBDim.TYPE, dataResourceName);
-        if (dataResource == null) {
-            return null;
-        }
-        Map<String, Object> row = dataResource.matchExpression(expressionStr, context.getMessage().getMessageBody());
-        if (row != null && row.size() > 0) {
-            return true;
-        }
-        return false;
-    }
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/expression/NotInExpressionResource.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/expression/NotInExpressionResource.java
deleted file mode 100644
index 001c550..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/expression/NotInExpressionResource.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.dim.function.expression;
-
-import org.apache.rocketmq.streams.common.context.AbstractContext;
-import org.apache.rocketmq.streams.common.context.IMessage;
-import org.apache.rocketmq.streams.filter.operator.expression.Expression;
-import org.apache.rocketmq.streams.script.annotation.Function;
-import org.apache.rocketmq.streams.script.annotation.FunctionMethod;
-import org.apache.rocketmq.streams.script.annotation.FunctionMethodAilas;
-
-@Function
-public class NotInExpressionResource extends InExpressionResource {
-
-    /**
-     * value格式 :resourceName.fieldName。如果只有单列,可以省略.fieldname
-     *
-     * @param expression
-     * @param context
-     * @return
-     */
-    @FunctionMethod("not_in_expression_resouce")
-    @FunctionMethodAilas("not_in_expression_resouce(resourceName->(varName,functionName,value)&((varName,"
-        + "functionName,value)|(varName,functionName,value)))")
-    @Override
-    public Boolean doExpressionFunction(IMessage message, AbstractContext context, Expression expression) {
-        return !match(message,context,expression, false);
-    }
-
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/IntelligenceFunction.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/IntelligenceFunction.java
deleted file mode 100644
index 34ebc04..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/IntelligenceFunction.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.dim.function.script;
-
-import java.util.Map;
-import java.util.Map.Entry;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.component.ComponentCreator;
-import org.apache.rocketmq.streams.common.context.IMessage;
-import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.configurable.ConfigurableComponent;
-import org.apache.rocketmq.streams.dim.DimComponent;
-import org.apache.rocketmq.streams.dim.intelligence.AbstractIntelligenceCache;
-import org.apache.rocketmq.streams.script.annotation.Function;
-import org.apache.rocketmq.streams.script.annotation.FunctionMethod;
-import org.apache.rocketmq.streams.script.context.FunctionContext;
-import org.apache.rocketmq.streams.script.utils.FunctionUtils;
-
-@Function
-public class IntelligenceFunction {
-
-    private static final Log LOG = LogFactory.getLog(IntelligenceFunction.class);
-
-    private DimComponent nameListComponent;
-
-    @FunctionMethod(value = "intelligence", alias = "qingbao")
-    public void intelligence(IMessage message, FunctionContext context, String namespace, String nameListName, String intelligenceFieldName, String asName) {
-        intelligenceInner(message, context, namespace, nameListName, intelligenceFieldName, asName, true);
-    }
-
-    @FunctionMethod(value = "left_join_intelligence", alias = "left_join_qingbao")
-    public void intelligenceLeftJoin(IMessage message, FunctionContext context, String namespace, String nameListName, String intelligenceFieldName, String asName) {
-        intelligenceInner(message, context, namespace, nameListName, intelligenceFieldName, asName, false);
-    }
-
-    public void intelligenceInner(IMessage message, FunctionContext context, String namespace, String nameListName, String intelligenceFieldName, String asName, boolean isInner) {
-        String key = FunctionUtils.getValueString(message, context, intelligenceFieldName);
-        namespace = FunctionUtils.getValueString(message, context, namespace);
-        nameListName = FunctionUtils.getValueString(message, context, nameListName);
-        ConfigurableComponent configurableComponent = ComponentCreator.getComponent(namespace, ConfigurableComponent.class);
-        AbstractIntelligenceCache intelligenceCache = configurableComponent.queryConfigurable(AbstractIntelligenceCache.TYPE, nameListName);
-        if (intelligenceCache == null) {
-            throw new RuntimeException("can not query intelligence. the namespace is " + namespace + ", the name is " + nameListName);
-        }
-        Map<String, Object> row = intelligenceCache.getRow(key);
-        if (row != null) {
-            asName = FunctionUtils.getValueString(message, context, asName);
-            if (StringUtil.isNotEmpty(asName)) {
-                asName = asName + ".";
-            } else {
-                asName = "";
-            }
-            for (Entry<String, Object> entry : row.entrySet()) {
-                String elementKey = asName + entry.getKey();
-                message.getMessageBody().put(elementKey, entry.getValue());
-            }
-        } else {
-            if (isInner) {
-                context.breakExecute();
-            }
-        }
-    }
-
-}
\ No newline at end of file
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/IntelligenceNameListFunction.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/IntelligenceNameListFunction.java
deleted file mode 100644
index 02d67fb..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/IntelligenceNameListFunction.java
+++ /dev/null
@@ -1,24 +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.dim.function.script;
-
-import org.apache.rocketmq.streams.script.annotation.Function;
-
-@Function
-public class IntelligenceNameListFunction {
-
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/NameListFunction.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/NameListFunction.java
deleted file mode 100644
index 08f3bcc..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/function/script/NameListFunction.java
+++ /dev/null
@@ -1,202 +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.dim.function.script;
-
-import com.alibaba.fastjson.JSONArray;
-import com.alibaba.fastjson.JSONObject;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.component.ComponentCreator;
-import org.apache.rocketmq.streams.common.context.IMessage;
-import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.dim.DimComponent;
-import org.apache.rocketmq.streams.script.annotation.Function;
-import org.apache.rocketmq.streams.script.annotation.FunctionMethod;
-import org.apache.rocketmq.streams.script.context.FunctionContext;
-import org.apache.rocketmq.streams.script.utils.FunctionUtils;
-
-@Function
-public class NameListFunction {
-
-    private static final Log LOG = LogFactory.getLog(NameListFunction.class);
-
-    private DimComponent nameListComponent;
-
-    @FunctionMethod(value = "innerJoin", alias = "inner_join")
-    public JSONArray innerJoin(IMessage message, FunctionContext context, String namespace, String nameListName,
-        String expressionStr, String alias, String script, String... fieldNames) {
-        JSONArray rows = getRows(message, context, namespace, nameListName, expressionStr, alias, script, fieldNames);
-        if (rows == null || rows.size() == 0) {
-            context.breakExecute();
-            return null;
-        }
-        return rows;
-    }
-
-    @FunctionMethod(value = "leftJoin", alias = "left_join")
-    public JSONArray leftJoin(IMessage message, FunctionContext context, String namespace, String nameListName,
-        String expressionStr, String alias, String script, String... fieldNames) {
-        JSONArray rows = getRows(message, context, namespace, nameListName, expressionStr, alias, script, fieldNames);
-        if (rows == null || rows.size() == 0) {
-            return null;
-        }
-        return rows;
-    }
-
-    @FunctionMethod(value = "mark_rows", alias = "namelist_rows")
-    public String namelist(IMessage message, FunctionContext context, String namespace, String nameListName,
-        String expressionStr, String... fieldNames) {
-        JSONArray rows = getRows(message, context, namespace, nameListName, expressionStr, null, null, fieldNames);
-        if (rows == null || rows.size() == 0) {
-            return null;
-        }
-        return rows.toJSONString();
-    }
-
-    @FunctionMethod(value = "mark", alias = "namelist")
-    public String namelist(IMessage message, FunctionContext context, String namespace, String nameListName,
-        String expressionStr, String fieldName) {
-        String tmp = fieldName;
-        nameListName = FunctionUtils.getValueString(message, context, nameListName);
-        namespace = FunctionUtils.getValueString(message, context, namespace);
-        expressionStr = FunctionUtils.getValueString(message, context, expressionStr);
-        fieldName = FunctionUtils.getValueString(message, context, fieldName);
-        if (fieldName == null) {
-            fieldName = tmp;
-        }
-        nameListComponent = ComponentCreator.getComponent(namespace, DimComponent.class);
-        Map<String, Object> row =
-            nameListComponent.getService().match(nameListName, expressionStr, message.getMessageBody());
-        if (row != null) {
-            Object value = row.get(fieldName);
-            if (value == null) {
-                return null;
-            }
-            return value.toString();
-        }
-        return null;
-    }
-
-    @FunctionMethod(value = "mark", alias = "namelist")
-    public String namelist(IMessage message, FunctionContext context, String namespace, String nameListName,
-        String expressionStr, String fieldNames, String joinMark) {
-        nameListName = FunctionUtils.getValueString(message, context, nameListName);
-        namespace = FunctionUtils.getValueString(message, context, namespace);
-        expressionStr = FunctionUtils.getValueString(message, context, expressionStr);
-        fieldNames = FunctionUtils.getValueString(message, context, fieldNames);
-        joinMark = FunctionUtils.getValueString(message, context, joinMark);
-        nameListComponent = ComponentCreator.getComponent(namespace, DimComponent.class);
-        Map<String, Object> row =
-            nameListComponent.getService().match(nameListName, expressionStr, message.getMessageBody());
-        if (row != null) {
-            String[] fieldNameTem = fieldNames.split(",");
-            StringBuilder result = new StringBuilder();
-            for (int i = 0; i < fieldNameTem.length; i++) {
-                Object tem = row.get(fieldNameTem[i]);
-                if (tem != null) {
-                    if (result.length() == 0) {
-                        if (StringUtil.isNotEmpty(tem.toString()) && !("null".equalsIgnoreCase(tem.toString()))) {
-                            result.append(tem);
-                        }
-                    } else {
-                        if (StringUtil.isNotEmpty(tem.toString()) && !("null".equalsIgnoreCase(tem.toString()))) {
-                            result.append(joinMark + tem);
-                        }
-                    }
-                }
-
-            }
-            return result.toString();
-        }
-        return null;
-    }
-
-    @FunctionMethod(value = "in_namelist", alias = "in_namelist")
-    public Boolean inNameList(IMessage message, FunctionContext context, String namespace, String nameListName,
-        String expressionStr) {
-        nameListComponent = ComponentCreator.getComponent(namespace, DimComponent.class);
-        Map<String, Object> row =
-            nameListComponent.getService().match(nameListName, expressionStr, message.getMessageBody());
-        if (row != null && row.size() > 0) {
-            return true;
-        }
-        return false;
-    }
-
-    /**
-     * 根据表达式,从namelist中获取符合条件的数据
-     *
-     * @param message
-     * @param context
-     * @param namespace
-     * @param nameListName
-     * @param expressionStr (varname,functionName,value)&(varname,functionName,value)
-     * @param fieldNames    需要返回的字段名
-     * @return
-     */
-    protected JSONArray getRows(IMessage message, FunctionContext context, String namespace, String nameListName,
-        String expressionStr, String alias, String script, String... fieldNames) {
-        nameListName = FunctionUtils.getValueString(message, context, nameListName);
-        namespace = FunctionUtils.getValueString(message, context, namespace);
-        expressionStr = FunctionUtils.getValueString(message, context, expressionStr);
-        script = FunctionUtils.getValueString(message, context, script);
-        if (StringUtil.isEmpty(script)) {
-            script = null;
-        }
-        nameListComponent = ComponentCreator.getComponent(namespace, DimComponent.class);
-        List<Map<String, Object>> rows =
-            nameListComponent.getService().matchSupportMultiRow(nameListName, expressionStr, message.getMessageBody(), script);
-        if (rows == null || rows.size() == 0) {
-            return null;
-        }
-        JSONArray jsonArray = new JSONArray();
-        for (Map<String, Object> row : rows) {
-            JSONObject jsonObject = new JSONObject();
-            if (fieldNames == null || fieldNames.length == 0) {
-                if (StringUtil.isEmpty(FunctionUtils.getConstant(alias))) {
-                    jsonObject.putAll(row);
-                } else {
-                    Iterator<Entry<String, Object>> it = row.entrySet().iterator();
-                    while (it.hasNext()) {
-                        Entry<String, Object> entry = it.next();
-                        String fieldName = entry.getKey();
-                        if (alias != null) {
-                            fieldName = alias + "." + fieldName;
-                        }
-                        jsonObject.put(fieldName, entry.getValue());
-                    }
-                }
-
-            } else {
-                for (String fieldName : fieldNames) {
-                    String tmp = fieldName;
-                    if (alias != null) {
-                        fieldName = alias + "." + fieldName;
-                    }
-                    jsonObject.put(fieldName, row.get(tmp));
-                }
-            }
-            jsonArray.add(jsonObject);
-        }
-        return jsonArray;
-    }
-
-}
\ No newline at end of file
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/index/DimIndex.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/index/DimIndex.java
deleted file mode 100644
index ef28a24..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/index/DimIndex.java
+++ /dev/null
@@ -1,204 +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.dim.index;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.cache.compress.AbstractMemoryTable;
-import org.apache.rocketmq.streams.common.cache.compress.KVAddress;
-import org.apache.rocketmq.streams.common.cache.compress.impl.MapAddressListKV;
-import org.apache.rocketmq.streams.common.datatype.IntDataType;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-
-public class DimIndex {
-
-    private static final Log LOG = LogFactory.getLog(DimIndex.class);
-
-    /**
-     * 索引字段名,支持多个索引,每个索引一行,支持组合索引,多个字段用;拼接 name 单索引 name;age 组合索引
-     */
-    protected List<String> indexs = new ArrayList<>();
-
-    /**
-     * 如果是非唯一索引,用这个结构存储 每个索引一行,后面的map:key:索引值;value:row id 列表,rowid用字节表示
-     */
-    protected Map<String, MapAddressListKV> mutilIndex = new HashMap<>();
-
-    public DimIndex(List<String> indexs) {
-        this.indexs = formatIndexs(indexs);
-    }
-
-    public DimIndex(String index, String... indexs) {
-        if (indexs == null) {
-            return;
-        }
-        List<String> indexList = new ArrayList<>();
-        for (String idx : indexs) {
-            indexList.add(idx);
-        }
-        this.indexs = formatIndexs(indexList);
-    }
-
-    /**
-     * 组合索引,多个字段要以名称顺序排列,完成索引名称的标注化处理
-     *
-     * @param indexs
-     * @return
-     */
-    protected List<String> formatIndexs(List<String> indexs) {
-        List<String> allIndex = new ArrayList<>();
-        for (String indexName : indexs) {
-            String[] values = indexName.split(";");
-            List<String> indexList = new ArrayList<>();
-            for (String value : values) {
-                indexList.add(value);
-            }
-            Collections.sort(indexList);
-            String indexKey = MapKeyUtil.createKey(indexList);
-            allIndex.add(indexKey);
-        }
-        return allIndex;
-    }
-
-    /**
-     * 加载一行数据,如果是唯一索引,在uniqueIndex中查找,否则在mutilIndex查找
-     *
-     * @param indexName  索引名,如name
-     * @param indexValue 索引值,如chris
-     * @return
-     */
-    public List<Long> getRowIds(String indexName, String indexValue) {
-        MapAddressListKV indexs = this.mutilIndex.get(indexName);
-        if (indexs == null) {
-            return null;
-        }
-        return indexs.getLongValue(indexValue);
-    }
-
-    /**
-     * 构建索引,如果是唯一索引,构建在uniqueIndex数据结构中,否则构建在mutilIndex这个数据结构中
-     *
-     * @param tableCompress 表数据
-     */
-    public void buildIndex(AbstractMemoryTable tableCompress) {
-
-        Iterator<AbstractMemoryTable.RowElement> it = tableCompress.newIterator();
-        int i = 0;
-        while (it.hasNext()) {
-            AbstractMemoryTable.RowElement row = it.next();
-            long rowIndex = row.getRowIndex();
-//            KVAddress mapAddress = new KVAddress(new ByteArray(NumberUtils.toByte(rowIndex)));
-            KVAddress mapAddress = KVAddress.createMapAddressFromLongValue(rowIndex);
-            addRowIndex(row.getRow(), mapAddress, tableCompress.getRowCount());
-//            addRowIndex(row.getRow(),row.getRowIndex(),tableCompress.getRowCount());
-            if ((i % 100000) == 0) {
-                LOG.info("dim build continue...." + i);
-            }
-            i++;
-        }
-
-        LOG.info(" finish poll data , the row count  is " + i + ". byte is " + tableCompress
-            .getByteCount());
-    }
-
-    /**
-     * 如果想直接增加索引,可以用这个方法
-     *
-     * @param row
-     * @param rowIndex
-     * @param rowSize
-     */
-    public void addRowIndex(Map<String, Object> row, KVAddress rowIndex, int rowSize) {
-        Map<String, String> cacheValues = createRow(row);
-        if (indexs == null || indexs.size() == 0) {
-            return;
-        }
-        for (String indexName : indexs) {
-            MapAddressListKV name2RowIndexs = this.mutilIndex.get(indexName);
-            if (name2RowIndexs == null) {
-                synchronized (this) {
-                    name2RowIndexs = this.mutilIndex.get(indexName);
-                    if (name2RowIndexs == null) {
-                        name2RowIndexs = new MapAddressListKV(rowSize);
-                        this.mutilIndex.put(indexName, name2RowIndexs);
-                    }
-                }
-
-            }
-            String[] nameIndexs = indexName.split(";");
-            Arrays.sort(nameIndexs);
-            String indexValue = createIndexValue(cacheValues, nameIndexs);
-            name2RowIndexs.add(indexValue, rowIndex);
-        }
-    }
-
-    /**
-     * 对于组合索引,把各个字段的值取出来
-     *
-     * @param row
-     * @param nameIndexs
-     * @return
-     */
-    protected String createIndexValue(Map<String, String> row, String[] nameIndexs) {
-        String[] indexValues = new String[nameIndexs.length];
-        for (int i = 0; i < nameIndexs.length; i++) {
-            indexValues[i] = row.get(nameIndexs[i]);
-        }
-        if (indexValues != null && indexValues.length > 0) {
-            String indexValue = MapKeyUtil.createKey(indexValues);
-            return indexValue;
-        }
-        return null;
-    }
-
-    /**
-     * 把row 中非string的值转化成string
-     *
-     * @param row
-     * @return
-     */
-    protected Map<String, String> createRow(Map<String, Object> row) {
-        Map<String, String> cacheValues = new HashMap<String, String>();//一行数据
-        Iterator<Map.Entry<String, Object>> iterator = row.entrySet().iterator();
-        //把数据value从object转化成string
-        while (iterator.hasNext()) {
-            Map.Entry<String, Object> entry = iterator.next();
-            if (entry != null && entry.getValue() != null && entry.getKey() != null) {
-                cacheValues.put(entry.getKey(), entry.getValue().toString());
-            }
-        }
-        return cacheValues;
-    }
-
-    public static IntDataType INTDATATYPE = new IntDataType();
-
-    public List<String> getIndexs() {
-        return indexs;
-    }
-
-    public void setIndexs(List<String> indexs) {
-        this.indexs = indexs;
-    }
-
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/index/IndexExecutor.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/index/IndexExecutor.java
deleted file mode 100644
index 7a4ab7b..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/index/IndexExecutor.java
+++ /dev/null
@@ -1,209 +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.dim.index;
-
-import com.alibaba.fastjson.JSONObject;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.rocketmq.streams.common.datatype.IntDataType;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.dim.model.AbstractDim;
-import org.apache.rocketmq.streams.filter.builder.ExpressionBuilder;
-import org.apache.rocketmq.streams.filter.function.expression.Equals;
-import org.apache.rocketmq.streams.filter.operator.Rule;
-import org.apache.rocketmq.streams.filter.operator.expression.Expression;
-import org.apache.rocketmq.streams.filter.operator.expression.RelationExpression;
-
-/**
- * 执行索引的查询和构建。主要是完成表达式的解析,对于等值的表达式字段,如果有索引,根据索引查询,然后执行非等值部分的判断
- */
-public class IndexExecutor {
-
-    private String expressionStr;//表达式
-
-    private boolean isSupport = false;//是否支持索引,比如表达式等值部分无所以,则不能走索引逻辑
-
-    private String namespace;
-
-    private String indexNameKey;//索引的名字,多个字段";"分隔
-
-    private List<String> msgNames;//
-
-    private Rule rule;//表达式会被编译成rule
-
-    private List<String> index; //标准化后的索引name
-
-    private Set<String> indexNames = new HashSet<>();
-
-    private Set<String> columnNames;
-
-    public IndexExecutor(String expressionStr, String namespace, List<String> index, Set<String> columns) {
-        this.expressionStr = expressionStr;
-        this.namespace = namespace;
-        this.columnNames = columns;
-        List<String> allIndex = new ArrayList<>();
-        for (String indexName : index) {
-            String[] values = indexName.split(";");
-            List<String> indexList = new ArrayList<>();
-            for (String value : values) {
-                indexNames.add(value);
-                indexList.add(value);
-            }
-            Collections.sort(indexList);
-            String indexKey = MapKeyUtil.createKey(indexList);
-            allIndex.add(indexKey);
-        }
-        this.index = allIndex;
-        parse();
-    }
-
-    /**
-     * 解析表达式,找出等值字段和非等值字段 如果有索引走索引,否则走全表扫描
-     */
-    protected void parse() {
-        List<Expression> expressions = new ArrayList<>();
-        List<RelationExpression> relationExpressions = new ArrayList<>();
-        Expression expression = ExpressionBuilder.createOptimizationExpression(namespace, "tmp", expressionStr,
-            expressions, relationExpressions);
-
-        RelationExpression relationExpression = null;
-        if (RelationExpression.class.isInstance(expression)) {
-            relationExpression = (RelationExpression) expression;
-            if (!"and".equals(relationExpression.getRelation())) {
-                isSupport = false;
-                return;
-            }
-        }
-
-        this.isSupport = true;
-        List<Expression> indexExpressions = new ArrayList<>();
-        List<Expression> otherExpressions = new ArrayList<>();
-        if (relationExpression != null) {
-            Map<String, Expression> map = new HashMap<>();
-            for (Expression tmp : expressions) {
-                map.put(tmp.getConfigureName(), tmp);
-            }
-            for (Expression tmp : relationExpressions) {
-                map.put(tmp.getConfigureName(), tmp);
-            }
-            List<String> expressionNames = relationExpression.getValue();
-            relationExpression.setValue(new ArrayList<>());
-            for (String expressionName : expressionNames) {
-                Expression subExpression = map.get(expressionName);
-                if (subExpression != null && !RelationExpression.class.isInstance(subExpression) && this.indexNames.contains(subExpression.getValue())) {
-                    indexExpressions.add(subExpression);
-                } else {
-                    otherExpressions.add(subExpression);
-                    relationExpression.getValue().add(subExpression.getConfigureName());
-                }
-            }
-
-        } else {
-            indexExpressions.add(expression);
-        }
-
-        List<String> fieldNames = new ArrayList<>();
-        Map<String, String> msgNames = new HashMap<>();
-
-        for (Expression expre : indexExpressions) {
-            if (RelationExpression.class.isInstance(expre)) {
-                continue;
-            }
-            String indexName = expre.getValue().toString();
-            if (Equals.isEqualFunction(expre.getFunctionName()) && indexNames.contains(indexName)) {
-
-                fieldNames.add(indexName);
-                msgNames.put(indexName, expre.getVarName());
-            }
-        }
-        Collections.sort(fieldNames);
-        indexNameKey = MapKeyUtil.createKey(fieldNames);
-        if (!this.index.contains(indexNameKey)) {
-            this.isSupport = false;
-            return;
-        }
-        this.msgNames = createMsgNames(fieldNames, msgNames);
-        if (otherExpressions.size() == 0) {
-            return;
-        }
-        Rule rule = ExpressionBuilder.createRule("tmp", "tmp", expressionStr);
-
-        this.rule = rule;
-
-    }
-
-    /**
-     * 创建索引字段的索引值
-     *
-     * @param fieldNames
-     * @param msgNames
-     * @return
-     */
-    protected List<String> createMsgNames(List<String> fieldNames, Map<String, String> msgNames) {
-        List<String> msgNameList = new ArrayList<>();
-        for (String fieldName : fieldNames) {
-            msgNameList.add(msgNames.get(fieldName));
-        }
-        return msgNameList;
-    }
-
-    public boolean isSupport() {
-        return isSupport;
-    }
-
-    private static IntDataType intDataType = new IntDataType();
-
-    public List<Map<String, Object>> match(JSONObject msg, AbstractDim nameList, boolean needAll, String script) {
-        List<Map<String, Object>> rows = new ArrayList<>();
-        String msgValue = createValue(msg);
-        List<Long> rowIds = nameList.getNameListIndex() == null ? Collections.emptyList() : nameList.getNameListIndex().getRowIds(indexNameKey, msgValue);
-        ;
-        if (rowIds == null) {
-            return null;
-        }
-        for (Long rowId : rowIds) {
-            Map<String, Object> oldRow = nameList.getDataCache().getRow(rowId);
-            Map<String, Object> newRow = AbstractDim.isMatch(this.rule, oldRow, msg, script, this.columnNames);
-            if (newRow != null) {
-                rows.add(newRow);
-                if (needAll == false) {
-                    return rows;
-                }
-            }
-        }
-        return rows;
-    }
-
-    /**
-     * 按顺序创建msg的key
-     *
-     * @param msg
-     * @return
-     */
-    private String createValue(JSONObject msg) {
-        List<String> value = new ArrayList<>();
-        for (String msgName : msgNames) {
-            value.add(msg.getString(msgName));
-        }
-        return MapKeyUtil.createKey(value);
-    }
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/AbstractIntelligenceCache.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/AbstractIntelligenceCache.java
deleted file mode 100644
index eafd34a..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/AbstractIntelligenceCache.java
+++ /dev/null
@@ -1,405 +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.dim.intelligence;
-
-import com.alibaba.fastjson.JSON;
-import com.alibaba.fastjson.JSONObject;
-import com.aliyuncs.CommonRequest;
-import com.aliyuncs.CommonResponse;
-import com.aliyuncs.DefaultAcsClient;
-import com.aliyuncs.IAcsClient;
-import com.aliyuncs.profile.DefaultProfile;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV;
-import org.apache.rocketmq.streams.common.channel.sink.ISink;
-import org.apache.rocketmq.streams.common.component.ComponentCreator;
-import org.apache.rocketmq.streams.common.configurable.BasedConfigurable;
-import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener;
-import org.apache.rocketmq.streams.common.configurable.IConfigurableService;
-import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence;
-import org.apache.rocketmq.streams.common.configure.ConfigureFileKey;
-import org.apache.rocketmq.streams.common.dboperator.IDBDriver;
-import org.apache.rocketmq.streams.common.utils.NumberUtils;
-import org.apache.rocketmq.streams.common.utils.SQLUtil;
-import org.apache.rocketmq.streams.db.driver.JDBCDriver;
-
-public abstract class AbstractIntelligenceCache extends BasedConfigurable implements
-    IAfterConfigurableRefreshListener {
-
-    private static final Log LOG = LogFactory.getLog(AbstractIntelligenceCache.class);
-
-    public static final String TYPE = "intelligence";
-
-    protected static final int FILE_MAX_LINE = 50000;//每个文件最大行数
-
-    protected transient IntValueKV intValueKV = new IntValueKV(0) {
-        @Override
-        public Integer get(String key) {
-            return null;
-        }
-
-        @Override
-        public void put(String key, Integer value) {
-
-        }
-    };
-
-    protected String idFieldName = "id";//必须有id字段
-
-    protected int batchSize = 3000;
-
-    @ENVDependence
-    protected Long pollingTimeMintue = 30L;
-
-    protected String datasourceName;//情报对应的存储
-
-    protected transient IDBDriver outputDataSource;
-
-    protected static ExecutorService executorService;
-
-    protected transient ScheduledExecutorService scheduledExecutorService;
-
-    public AbstractIntelligenceCache() {
-        setType(TYPE);
-        executorService = new ThreadPoolExecutor(20, 20,
-            0L, TimeUnit.MILLISECONDS,
-            new LinkedBlockingQueue<Runnable>(1000));
-        scheduledExecutorService = new ScheduledThreadPoolExecutor(3);
-    }
-
-    public IntValueKV startLoadData(String sql, IDBDriver resource) {
-        try {
-            String statisticalSQL = sql;
-            int startIndex = sql.toLowerCase().indexOf("from");
-            statisticalSQL = "select count(1) as c, min(" + idFieldName + ") as min, max(" + idFieldName + ") as max "
-                + sql.substring(startIndex);
-            List<Map<String, Object>> rows = resource.queryForList(statisticalSQL);
-            Map<String, Object> row = rows.get(0);
-            int count = Integer.valueOf(row.get("c").toString());
-            IntValueKV intValueKV = new IntValueKV(count);
-            //int maxBatch=count/maxSyncCount;//每1w条数据,一个并发。如果数据量比较大,为了提高性能,并行执行
-            if (count == 0) {
-                return new IntValueKV(0) {
-                    @Override
-                    public Integer get(String key) {
-                        return null;
-                    }
-
-                    @Override
-                    public void put(String key, Integer value) {
-
-                    }
-                };
-            }
-            long min = Long.valueOf(row.get("min").toString());
-            long max = Long.valueOf(row.get("max").toString());
-            int maxSyncCount = count / FILE_MAX_LINE + 1;
-            long step = (max - min + 1) / maxSyncCount;
-            CountDownLatch countDownLatch = new CountDownLatch(maxSyncCount + 1);
-            AtomicInteger finishedCount = new AtomicInteger(0);
-            String taskSQL = null;
-            if (sql.indexOf(" where ") != -1) {
-                taskSQL = sql + " and " + idFieldName + ">#{startIndex} and " + idFieldName + "<=#{endIndex} order by "
-                    + idFieldName + " limit " + batchSize;
-            } else {
-                taskSQL = sql + " where " + idFieldName + ">#{startIndex} and " + idFieldName
-                    + "<=#{endIndex} order by " + idFieldName + " limit " + batchSize;
-            }
-
-            int i = 0;
-            for (; i < maxSyncCount; i++) {
-                FetchDataTask fetchDataTask = new FetchDataTask(taskSQL, (min - 1) + step * i,
-                    (min - 1) + step * (i + 1), countDownLatch, finishedCount, resource, i, intValueKV, this, count);
-                executorService.execute(fetchDataTask);
-            }
-            FetchDataTask fetchDataTask = new FetchDataTask(taskSQL, (min - 1) + step * i, (min - 1) + step * (i + 1),
-                countDownLatch, finishedCount, resource, i, intValueKV, this, count);
-            executorService.execute(fetchDataTask);
-
-            countDownLatch.await();
-
-            LOG.info(getClass().getSimpleName() + " load data finish, load data line  size is " + intValueKV.getSize());
-            return intValueKV;
-        } catch (Exception e) {
-            LOG.error("failed loading intelligence data!", e);
-            return new IntValueKV(0) {
-                @Override
-                public Integer get(String key) {
-                    return null;
-                }
-
-                @Override
-                public void put(String key, Integer value) {
-
-                }
-            };
-        }
-    }
-
-    protected transient AtomicBoolean hasInit = new AtomicBoolean(false);
-
-    @Override
-    public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) {
-        this.outputDataSource = configurableService.queryConfigurable(ISink.TYPE, datasourceName);
-    }
-
-    public void startIntelligence() {
-        boolean success = dbInit();
-        if (success) {
-            startIntelligenceInner();
-        } else {
-            Thread thread = new Thread(new Runnable() {
-                @Override
-                public void run() {
-                    boolean success = false;
-                    while (!success) {
-                        success = dbInit();
-                        try {
-                            Thread.sleep(60 * 1000);
-                        } catch (InterruptedException e) {
-                            e.printStackTrace();
-                        }
-                    }
-                    startIntelligenceInner();
-                }
-            });
-            thread.start();
-        }
-    }
-
-    public void startIntelligenceInner() {
-        String sql = getSQL();
-        if (hasInit.compareAndSet(false, true)) {
-            this.intValueKV = startLoadData(sql, outputDataSource);
-            scheduledExecutorService.scheduleWithFixedDelay(new Runnable() {
-                @Override
-                public void run() {
-                    intValueKV = startLoadData(sql, outputDataSource);
-                }
-            }, pollingTimeMintue, pollingTimeMintue, TimeUnit.MINUTES);
-        }
-    }
-
-    public abstract Map<String, Object> getRow(String key);
-
-    /**
-     * 查询情报需要的sql
-     *
-     * @return
-     */
-    protected abstract String getSQL();
-
-    /**
-     * 情报中的 情报字段名
-     *
-     * @return
-     */
-    public abstract String getKeyName();
-
-    /**
-     * 情报对应的表名
-     *
-     * @return
-     */
-    public abstract String getTableName();
-
-    protected class FetchDataTask implements Runnable {
-        IntValueKV intValueKV;
-        long startIndex;
-        long endIndex;
-        String sql;
-        CountDownLatch countDownLatch;
-        int index;
-        IDBDriver resource;
-        AtomicInteger finishedCount;//完成了多少条
-        AbstractIntelligenceCache cache;
-        int totalSize;//一共有多少条数据
-
-        public FetchDataTask(String sql, long startIndex, long endIndex, CountDownLatch countDownLatch,
-            AtomicInteger finishedCount, IDBDriver resource, int i, IntValueKV intValueKV,
-            AbstractIntelligenceCache cache, int totalSize) {
-            this.startIndex = startIndex;
-            this.endIndex = endIndex;
-            this.countDownLatch = countDownLatch;
-            this.sql = sql;
-            this.finishedCount = finishedCount;
-            this.resource = resource;
-            this.index = i;
-            this.intValueKV = intValueKV;
-            this.cache = cache;
-            this.totalSize = totalSize;
-        }
-
-        @Override
-        public void run() {
-            long currentIndex = startIndex;
-            JSONObject msg = new JSONObject();
-            msg.put("endIndex", endIndex);
-            while (true) {
-                try {
-
-                    msg.put("startIndex", currentIndex);
-
-                    String sql = SQLUtil.parseIbatisSQL(msg, this.sql);
-                    List<Map<String, Object>> rows = resource.queryForList(sql);
-                    if (rows == null || rows.size() == 0) {
-                        break;
-                    }
-                    currentIndex = Long.valueOf(rows.get(rows.size() - 1).get(idFieldName).toString());
-
-                    int size = rows.size();
-                    int count = finishedCount.addAndGet(size);
-                    double progress = (double) count / (double) totalSize;
-                    progress = progress * 100;
-                    System.out.println(cache.getClass().getSimpleName() + ", finished count is " + count + " the total count is " + totalSize + ", the progress is " + String.format("%.2f", progress) + "%");
-                    if (size < batchSize) {
-                        if (size > 0) {
-                            doProccRows(intValueKV, rows, index);
-                        }
-                        break;
-                    }
-                    doProccRows(intValueKV, rows, index);
-                } catch (Exception e) {
-                    throw new RuntimeException("put data error ", e);
-                }
-            }
-
-            countDownLatch.countDown();
-        }
-    }
-
-    public boolean dbInit() {
-        try {
-            int successCode = 200;
-            String region = ComponentCreator.getProperties().getProperty(ConfigureFileKey.INTELLIGENCE_REGION);
-            String ak = ComponentCreator.getProperties().getProperty(
-                ConfigureFileKey.INTELLIGENCE_AK);
-            String sk = ComponentCreator.getProperties().getProperty(
-                ConfigureFileKey.INTELLIGENCE_SK);
-            String endpoint = ComponentCreator.getProperties().getProperty(
-                ConfigureFileKey.INTELLIGENCE_TIP_DB_ENDPOINT);
-            if (StringUtils.isNotBlank(region) && StringUtils.isNotBlank(ak) && StringUtils.isNotBlank(sk) && StringUtils.isNotBlank(endpoint)) {
-                DefaultProfile profile = DefaultProfile.getProfile(region, ak, sk);
-                IAcsClient client = new DefaultAcsClient(profile);
-                CommonRequest request = new CommonRequest();
-                request.setDomain(endpoint);
-                request.setVersion("2016-03-16");
-                request.setAction("DescribeDataSource");
-                CommonResponse response = client.getCommonResponse(request);
-                int code = response.getHttpStatus();
-                if (successCode == code) {
-                    String content = response.getData();
-                    if (StringUtils.isNotBlank(content)) {
-                        JSONObject obj = JSON.parseObject(content);
-                        JSONObject dbInfo = obj.getJSONObject("dBInfo");
-                        if (dbInfo != null) {
-                            String dbUrl = "jdbc:mysql://" + dbInfo.getString("dbConnection") + ":" + dbInfo.getInteger(
-                                "port") + "/" + dbInfo.getString("dBName");
-                            String dbUserName = dbInfo.getString("userName");
-                            String dbPassword = dbInfo.getString("passWord");
-                            JDBCDriver dataSource = (JDBCDriver) this.outputDataSource;
-                            dataSource.setUrl(dbUrl);
-                            dataSource.setPassword(dbPassword);
-                            dataSource.setUserName(dbUserName);
-                            dataSource.setHasInit(false);
-                            dataSource.init();
-                            LOG.debug("succeed in getting db information from tip service!");
-                            return true;
-                        }
-                    }
-                }
-            }
-            LOG.error("failed in getting db information from tip service!");
-            return false;
-        } catch (Exception e) {
-            LOG.error("failed in getting db information from tip service!", e);
-            return false;
-        }
-    }
-
-    /**
-     * 把存储0/1字符串的值,转化成bit
-     *
-     * @param values
-     * @return
-     */
-    protected int createInt(List<String> values) {
-        return NumberUtils.createBitMapInt(values);
-    }
-
-    /**
-     * 获取某位的值,如果是1,返回字符串1,否则返回null
-     *
-     * @param num
-     * @param i
-     * @return
-     */
-    protected String getNumBitValue(int num, int i) {
-        boolean exist = NumberUtils.getNumFromBitMapInt(num, i);
-        if (exist) {
-            return "1";
-        }
-        return null;
-    }
-
-    protected abstract void doProccRows(IntValueKV intValueKV, List<Map<String, Object>> rows, int index);
-
-    public String getIdFieldName() {
-        return idFieldName;
-    }
-
-    public void setIdFieldName(String idFieldName) {
-        this.idFieldName = idFieldName;
-    }
-
-    public int getBatchSize() {
-        return batchSize;
-    }
-
-    public void setBatchSize(int batchSize) {
-        this.batchSize = batchSize;
-    }
-
-    public Long getPollingTimeMintue() {
-        return pollingTimeMintue;
-    }
-
-    public void setPollingTimeMintue(Long pollingTimeMintue) {
-        this.pollingTimeMintue = pollingTimeMintue;
-    }
-
-    public String getDatasourceName() {
-        return datasourceName;
-    }
-
-    public void setDatasourceName(String datasourceName) {
-        this.datasourceName = datasourceName;
-    }
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/AccountIntelligenceCache.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/AccountIntelligenceCache.java
deleted file mode 100644
index 1cbda46..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/AccountIntelligenceCache.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.dim.intelligence;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV;
-import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener;
-
-/**
- * table: ads_yunsec_abnormal_account
- */
-public class AccountIntelligenceCache extends AbstractIntelligenceCache implements IAfterConfigurableRefreshListener {
-
-    private static final Log LOG = LogFactory.getLog(AccountIntelligenceCache.class);
-
-    /**
-     * 情报域名
-     */
-    protected transient String keyName = "account";
-
-    @Override
-    public Map<String, Object> getRow(String account) {
-        Integer value = intValueKV.get(account);
-        if (value == null) {
-            return null;
-        }
-        Map<String, Object> row = new HashMap<String, Object>() {{
-            put("account", account);
-        }};
-        return row;
-    }
-
-    @Override
-    protected String getSQL() {
-        return "SELECT id, `account` FROM `ads_yunsec_abnormal_account`";
-    }
-
-    @Override
-    public String getKeyName() {
-        return this.keyName;
-    }
-
-    @Override
-    public String getTableName() {
-        return "ads_yunsec_abnormal_account";
-    }
-
-    @Override
-    protected void doProccRows(IntValueKV intValueKV, List<Map<String, Object>> rows, int index) {
-        rows.forEach(row -> {
-            String account = (String)row.get(keyName);
-            if (account != null) {
-                intValueKV.put(account, 1);
-            }
-        });
-    }
-
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/DomainIntelligenceCache.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/DomainIntelligenceCache.java
deleted file mode 100644
index 1926a67..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/DomainIntelligenceCache.java
+++ /dev/null
@@ -1,82 +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.dim.intelligence;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV;
-import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener;
-
-public class DomainIntelligenceCache extends AbstractIntelligenceCache implements IAfterConfigurableRefreshListener {
-    private static final Log LOG = LogFactory.getLog(DomainIntelligenceCache.class);
-    protected transient String keyName = "domain";
-
-    @Override
-    protected String getSQL() {
-        return "SELECT id, `domain` , `is_malicious_source` , `is_phishing` , `is_c2` , `is_mining_pool`  FROM `ads_yunsec_ti_domain_all_df` where curdate() < date_add(modify_time, interval expire_time day) ";
-    }
-
-    @Override
-    public Map<String, Object> getRow(String ip) {
-        Integer value = intValueKV.get(ip);
-        if (value == null) {
-            return null;
-        }
-        Map<String, Object> row = new HashMap<>();
-        row.put("is_malicious_source", getNumBitValue(value, 0));
-        row.put("is_phishing", getNumBitValue(value, 1));
-        row.put("is_c2", getNumBitValue(value, 2));
-        row.put("is_mining_pool", getNumBitValue(value, 3));
-        return row;
-    }
-
-    @Override
-    public String getKeyName() {
-        return this.keyName;
-    }
-
-    @Override
-    public String getTableName() {
-        return "ads_yunsec_ti_domain_all_df";
-    }
-
-    @Override
-    protected void doProccRows(IntValueKV intValueKV, List<Map<String, Object>> rows, int index) {
-        for (Map<String, Object> row : rows) {
-            String ip = (String)row.get(keyName);
-            if (ip == null) {
-                LOG.warn("load Intelligence exception ,the ip is null");
-                continue;
-            }
-            List<String> values = new ArrayList<>();
-            values.add((String)row.get("is_malicious_source"));
-            values.add((String)row.get("is_phishing"));
-            values.add((String)row.get("is_c2"));
-            values.add((String)row.get("is_mining_pool"));
-            int value = createInt(values);
-            synchronized (this) {
-                intValueKV.put(ip, value);
-            }
-
-        }
-    }
-
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/IPIntelligenceCache.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/IPIntelligenceCache.java
deleted file mode 100644
index 61ba2f3..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/IPIntelligenceCache.java
+++ /dev/null
@@ -1,107 +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.dim.intelligence;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV;
-import org.apache.rocketmq.streams.common.component.ComponentCreator;
-import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener;
-import org.apache.rocketmq.streams.common.dboperator.IDBDriver;
-import org.apache.rocketmq.streams.db.driver.DriverBuilder;
-
-public class IPIntelligenceCache extends AbstractIntelligenceCache implements IAfterConfigurableRefreshListener {
-    private static final Log LOG = LogFactory.getLog(IPIntelligenceCache.class);
-    protected transient String keyName = "ip";
-
-
-    @Override
-    protected String getSQL() {
-        return "SELECT id,ip, `is_web_attack` , `is_tor` , `is_proxy` , `is_nat` , `is_mining_pool` , `is_c2` , "
-            + "`is_malicious_source` , `is_3rd` , `is_idc` , `is_malicious_login`  FROM `ads_yunsec_ti_ip_all_df` where curdate() < date_add(modify_time, interval expire_time day) ";
-    }
-
-    @Override
-    public String getKeyName() {
-        return this.keyName;
-    }
-
-    @Override
-    public String getTableName() {
-        return "ads_yunsec_ti_ip_all_df";
-    }
-
-    @Override
-    public Map<String, Object> getRow(String ip) {
-
-        Integer value = intValueKV.get(ip);
-        if (value == null) {
-            return null;
-        }
-        Map<String, Object> row = new HashMap<>();
-
-        row.put("is_web_attack", getNumBitValue(value, 0));
-        row.put("is_tor", getNumBitValue(value, 1));
-        row.put("is_proxy", getNumBitValue(value, 2));
-        row.put("is_nat", getNumBitValue(value, 3));
-        row.put("is_mining_pool", getNumBitValue(value, 4));
-        row.put("is_c2", getNumBitValue(value, 5));
-        row.put("is_malicious_source", getNumBitValue(value, 6));
-        row.put("is_3rd", getNumBitValue(value, 7));
-        row.put("is_idc", getNumBitValue(value, 8));
-        row.put("is_malicious_login", getNumBitValue(value, 9));
-        return row;
-    }
-
-    @Override
-    protected void doProccRows(IntValueKV intValueKV, List<Map<String, Object>> rows, int index) {
-        for (Map<String, Object> row : rows) {
-            String ip = (String)row.get(keyName);
-            if (ip == null) {
-                LOG.warn("load Intelligence exception ,the ip is null");
-                continue;
-            }
-            List<String> values = new ArrayList<>();
-            values.add((String)row.get("is_web_attack"));
-            values.add((String)row.get("is_tor"));
-            values.add((String)row.get("is_proxy"));
-            values.add((String)row.get("is_nat"));
-            values.add((String)row.get("is_mining_pool"));
-            values.add((String)row.get("is_c2"));
-            values.add((String)row.get("is_malicious_source"));
-            values.add((String)row.get("is_3rd"));
-            values.add((String)row.get("is_idc"));
-            values.add((String)row.get("is_malicious_login"));
-            int value = createInt(values);
-            synchronized (this) {
-                intValueKV.put(ip, value);
-            }
-        }
-    }
-
-    public static void main(String[] args) {
-        ComponentCreator.setProperties(
-            "siem.properties");
-        IPIntelligenceCache ipIntelligenceCache = new IPIntelligenceCache();
-        IDBDriver outputDataSource = DriverBuilder.createDriver();
-        ipIntelligenceCache.startLoadData(ipIntelligenceCache.getSQL(), outputDataSource);
-    }
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/URLIntelligenceCache.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/URLIntelligenceCache.java
deleted file mode 100644
index b1263da..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/intelligence/URLIntelligenceCache.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.dim.intelligence;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.cache.compress.impl.IntValueKV;
-import org.apache.rocketmq.streams.common.configurable.IAfterConfigurableRefreshListener;
-
-public class URLIntelligenceCache extends AbstractIntelligenceCache implements IAfterConfigurableRefreshListener {
-
-    private static final Log LOG = LogFactory.getLog(URLIntelligenceCache.class);
-
-    protected transient String keyName = "url";
-
-    @Override
-    protected String getSQL() {
-        return "SELECT id, url, `is_malicious_source`  FROM `ads_yunsec_ti_url_all_df` where curdate() < date_add(modify_time, interval expire_time day) ";
-    }
-
-    @Override
-    public Map<String, Object> getRow(String ip) {
-        Integer value = intValueKV.get(ip);
-        if (value == null) {
-            return null;
-        }
-        Map<String, Object> row = new HashMap<>();
-
-        row.put("is_malicious_source", getNumBitValue(value, 0));
-        return row;
-    }
-
-    @Override
-    public String getKeyName() {
-        return this.keyName;
-    }
-
-    @Override
-    public String getTableName() {
-        return "ads_yunsec_ti_url_all_df";
-    }
-
-    @Override
-    protected void doProccRows(IntValueKV intValueKV, List<Map<String, Object>> rows, int index) {
-        for (Map<String, Object> row : rows) {
-            String ip = (String)row.get(keyName);
-            if (ip == null) {
-                LOG.warn("load Intelligence exception ,the ip is null");
-                continue;
-            }
-            List<String> values = new ArrayList<>();
-            values.add((String)row.get("is_malicious_source"));
-            int value = createInt(values);
-            synchronized (this) {
-                intValueKV.put(ip, value);
-            }
-
-        }
-    }
-
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/AbstractDim.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/AbstractDim.java
deleted file mode 100644
index fb76395..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/AbstractDim.java
+++ /dev/null
@@ -1,489 +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.dim.model;
-
-import com.alibaba.fastjson.JSONObject;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.cache.ByteArrayMemoryTable;
-import org.apache.rocketmq.streams.common.cache.MappedByteBufferTable;
-import org.apache.rocketmq.streams.common.cache.compress.AbstractMemoryTable;
-import org.apache.rocketmq.streams.common.cache.softreference.ICache;
-import org.apache.rocketmq.streams.common.cache.softreference.impl.SoftReferenceCache;
-import org.apache.rocketmq.streams.common.component.ComponentCreator;
-import org.apache.rocketmq.streams.common.configurable.BasedConfigurable;
-import org.apache.rocketmq.streams.common.configure.ConfigureFileKey;
-import org.apache.rocketmq.streams.common.utils.DataTypeUtil;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.dim.index.DimIndex;
-import org.apache.rocketmq.streams.dim.index.IndexExecutor;
-import org.apache.rocketmq.streams.filter.builder.ExpressionBuilder;
-import org.apache.rocketmq.streams.filter.function.expression.Equals;
-import org.apache.rocketmq.streams.filter.operator.Rule;
-import org.apache.rocketmq.streams.filter.operator.expression.Expression;
-import org.apache.rocketmq.streams.filter.operator.expression.RelationExpression;
-import org.apache.rocketmq.streams.script.ScriptComponent;
-
-/**
- * 这个结构代表一张表 存放表的全部数据和索引
- */
-public abstract class AbstractDim extends BasedConfigurable {
-
-    private static final Log LOG = LogFactory.getLog(AbstractDim.class);
-
-    public static final String TYPE = "nameList";
-
-    /**
-     * 同步数据的事件间隔,单位是分钟
-     */
-    protected Long pollingTimeMinute = 60L;
-
-    /**
-     * 支持多组索引,如果一个索引是组合索引,需要拼接成一个string,用;分割 建立索引后,会创建索引的数据结构,类似Map<String,List<RowId>,可以快速定位,无索引会全表扫描,不建议使用 如有两组索引:1.name 2. ip;address
-     */
-    protected List<String> indexs = new ArrayList<>();
-
-    protected boolean isLarge = false;//if isLarge=true use MapperByteBufferTable 内存结构
-
-    protected String filePath;
-    /**
-     * 把表数据转化成二进制存储在CompressTable中
-     */
-    protected transient volatile AbstractMemoryTable dataCache;
-
-    /**
-     * 建立名单的时候,可以指定多组索引,索引的值当作key,row在datacache的index当作value,可以快速匹配索引对应的row key:索引的值 value:row在dataCache的index当作value,可以快速匹配索引对应的row
-     */
-    protected transient DimIndex nameListIndex;
-    protected transient Set<String> columnNames;
-    //定时加载表数据到内存
-    protected transient ScheduledExecutorService executorService;
-
-    public AbstractDim() {
-        this.setType(TYPE);
-    }
-
-    //protected String index;//只是做标记,为了是简化indexs的赋值
-
-    public String addIndex(String... fieldNames) {
-        return addIndex(this.indexs, fieldNames);
-    }
-
-    @Override
-    protected boolean initConfigurable() {
-        boolean success = super.initConfigurable();
-        if (Boolean.TRUE.equals(Boolean.valueOf(ComponentCreator.getProperties().getProperty(ConfigureFileKey.DIPPER_RUNNING_STATUS, ConfigureFileKey.DIPPER_RUNNING_STATUS_DEFAULT)))) {
-            loadNameList();
-            executorService = new ScheduledThreadPoolExecutor(3);
-            executorService.scheduleWithFixedDelay(new Runnable() {
-                @Override
-                public void run() {
-                    loadNameList();
-                }
-            }, pollingTimeMinute, pollingTimeMinute, TimeUnit.MINUTES);
-        }
-
-        return success;
-    }
-
-    /**
-     * 加载维表数据 创建索引
-     */
-    protected void loadNameList() {
-        try {
-            LOG.info(getConfigureName() + " begin polling data");
-            //全表数据
-            AbstractMemoryTable dataCacheVar = loadData();
-            this.dataCache = dataCacheVar;
-            this.nameListIndex = buildIndex(dataCacheVar);
-            this.columnNames = this.dataCache.getCloumnName2Index().keySet();
-        } catch (Exception e) {
-            LOG.error("Load configurables error:" + e.getMessage(), e);
-        }
-    }
-
-    /**
-     * 给维表生成索引数据结构
-     *
-     * @param dataCacheVar 维表
-     * @return
-     */
-    protected DimIndex buildIndex(AbstractMemoryTable dataCacheVar) {
-        DimIndex dimIndex = new DimIndex(this.indexs);
-        dimIndex.buildIndex(dataCacheVar);
-        return dimIndex;
-    }
-
-    /**
-     * 软引用缓存,最大可能保存索引执行器,避免频繁创建,带来额外开销 同时会保护内存不被写爆,当内存不足时自动回收内存
-     */
-    private static ICache<String, IndexExecutor> cache = new SoftReferenceCache<>();
-
-    /**
-     * 先找索引,如果有索引,通过索引匹配。如果没有,全表扫表.
-     *
-     * @param expressionStr 表达式
-     * @param msg           消息
-     * @return 只返回匹配的第一行
-     */
-    public Map<String, Object> matchExpression(String expressionStr, JSONObject msg) {
-        List<Map<String, Object>> rows = matchExpression(expressionStr, msg, true, null);
-        if (rows != null && rows.size() > 0) {
-            return rows.get(0);
-        }
-        return null;
-    }
-
-    /**
-     * 先找索引,如果有索引,通过索引匹配。如果没有,全表扫表
-     *
-     * @param expressionStr 表达式
-     * @param msg           消息
-     * @return 返回全部匹配的行
-     */
-    public List<Map<String, Object>> matchExpression(String expressionStr, JSONObject msg, boolean needAll,
-        String script) {
-        IndexExecutor indexNamelistExecutor = cache.get(expressionStr);
-        if (indexNamelistExecutor == null) {
-            indexNamelistExecutor = new IndexExecutor(expressionStr, getNameSpace(), this.indexs, dataCache.getCloumnName2DatatType().keySet());
-            cache.put(expressionStr, indexNamelistExecutor);
-        }
-        if (indexNamelistExecutor.isSupport()) {
-            return indexNamelistExecutor.match(msg, this, needAll, script);
-        } else {
-            return matchExpressionByLoop(dataCache.rowIterator(), expressionStr, msg, needAll, script, columnNames);
-        }
-    }
-
-    /**
-     * 全表扫描,做表达式匹配,返回全部匹配结果
-     *
-     * @param expressionStr
-     * @param msg
-     * @param needAll
-     * @return
-     */
-    protected List<Map<String, Object>> matchExpressionByLoop(String expressionStr, JSONObject msg, boolean needAll) {
-        AbstractMemoryTable dataCache = this.dataCache;
-        List<Map<String, Object>> rows = matchExpressionByLoop(dataCache.rowIterator(), expressionStr, msg, needAll, null, columnNames);
-        return rows;
-    }
-
-    /**
-     * 全表扫描,做表达式匹配,返回全部匹配结果。join中有使用
-     *
-     * @param expressionStr
-     * @param msg
-     * @param needAll
-     * @return
-     */
-    public static List<Map<String, Object>> matchExpressionByLoop(Iterator<Map<String, Object>> it,
-        String expressionStr, JSONObject msg, boolean needAll) {
-        return matchExpressionByLoop(it, expressionStr, msg, needAll, null, new HashSet<>());
-    }
-
-    /**
-     * 全表扫描,做表达式匹配,返回全部匹配结果。join中有使用
-     *
-     * @param expressionStr
-     * @param msg
-     * @param needAll
-     * @return
-     */
-    public static List<Map<String, Object>> matchExpressionByLoop(Iterator<Map<String, Object>> it,
-        String expressionStr, JSONObject msg, boolean needAll, String script, Set<String> colunmNames) {
-        List<Map<String, Object>> rows = new ArrayList<>();
-        Rule ruleTemplete = ExpressionBuilder.createRule("tmp", "tmpRule", expressionStr);
-        while (it.hasNext()) {
-            Map<String, Object> oldRow = it.next();
-            Map<String, Object> newRow = isMatch(ruleTemplete, oldRow, msg, script, colunmNames);
-            if (newRow != null) {
-                rows.add(newRow);
-                if (!needAll) {
-                    return rows;
-                }
-            }
-        }
-        return rows;
-    }
-
-    /**
-     * 和维表的一行数据进行匹配,如果维表中有函数,先执行函数
-     *
-     * @param ruleTemplete
-     * @param dimRow
-     * @param msgRow
-     * @param script
-     * @param colunmNames
-     * @return
-     */
-    public static Map<String, Object> isMatch(Rule ruleTemplete, Map<String, Object> dimRow, JSONObject msgRow,
-        String script, Set<String> colunmNames) {
-        Map<String, Object> oldRow = dimRow;
-        Map<String, Object> newRow = executeScript(oldRow, script);
-        if (ruleTemplete == null) {
-            return newRow;
-        }
-        Rule rule = ruleTemplete.copy();
-        Map<String, Expression> expressionMap = new HashMap<>();
-        String dimAsName = null;
-        ;
-        for (Expression expression : rule.getExpressionMap().values()) {
-            expressionMap.put(expression.getConfigureName(), expression);
-            if (expression instanceof RelationExpression) {
-                continue;
-            }
-            Object object = expression.getValue();
-            if (object != null && DataTypeUtil.isString(object.getClass())) {
-                String fieldName = (String) object;
-                Object value = newRow.get(fieldName);
-                if (value != null) {
-                    Expression e = expression.copy();
-                    e.setValue(value.toString());
-                    expressionMap.put(e.getConfigureName(), e);
-                }
-            }
-            if (expression.getVarName().contains(".")) {
-                String[] values = expression.getVarName().split("\\.");
-                if (values.length == 2) {
-                    String asName = values[0];
-                    String varName = values[1];
-                    if (colunmNames.contains(varName)) {
-                        dimAsName = asName;
-                    }
-                }
-
-            }
-        }
-        rule.setExpressionMap(expressionMap);
-        rule.initElements();
-        JSONObject copyMsg = msgRow;
-        if (StringUtil.isNotEmpty(dimAsName)) {
-            copyMsg = new JSONObject(msgRow);
-            for (String key : newRow.keySet()) {
-                copyMsg.put(dimAsName + "." + key, newRow.get(key));
-            }
-        }
-        boolean matched = rule.execute(copyMsg);
-        if (matched) {
-            return newRow;
-        }
-        return null;
-    }
-
-    public static interface IDimField {
-        boolean isDimField(Object fieldName);
-    }
-
-    /**
-     * 根据join条件设置索引
-     */
-    public void createIndexByJoinCondition(String expressionStr, IDimField dimField) {
-        List<Expression> expressions = new ArrayList<>();
-        List<RelationExpression> relationExpressions = new ArrayList<>();
-        Expression expression = ExpressionBuilder.createOptimizationExpression("tmp", "tmp", expressionStr, expressions, relationExpressions);
-
-        RelationExpression relationExpression = null;
-        if (expression instanceof RelationExpression) {
-            relationExpression = (RelationExpression) expression;
-            if (!"and".equals(relationExpression.getRelation())) {
-                return;
-            }
-        }
-
-        List<Expression> indexExpressions = new ArrayList<>();
-        List<Expression> otherExpressions = new ArrayList<>();
-        if (relationExpression != null) {
-            Map<String, Expression> map = new HashMap<>();
-            for (Expression tmp : expressions) {
-                map.put(tmp.getConfigureName(), tmp);
-            }
-            for (Expression tmp : relationExpressions) {
-                map.put(tmp.getConfigureName(), tmp);
-            }
-            List<String> expressionNames = relationExpression.getValue();
-            relationExpression.setValue(new ArrayList<>());
-            for (String expressionName : expressionNames) {
-                Expression subExpression = map.get(expressionName);
-                if (subExpression != null && !RelationExpression.class.isInstance(subExpression) && dimField.isDimField(subExpression.getValue())) {
-                    indexExpressions.add(subExpression);
-                } else {
-                    otherExpressions.add(subExpression);
-                    relationExpression.getValue().add(subExpression.getConfigureName());
-                }
-            }
-
-        } else {
-            indexExpressions.add(expression);
-        }
-
-        List<String> fieldNames = new ArrayList<>();
-
-        for (Expression expre : indexExpressions) {
-            if (expre instanceof RelationExpression) {
-                continue;
-            }
-            String indexName = expre.getValue().toString();
-            if (Equals.isEqualFunction(expre.getFunctionName()) && dimField.isDimField(expre.getValue())) {
-
-                fieldNames.add(indexName);
-
-            }
-        }
-
-        String[] indexFieldNameArray = new String[fieldNames.size()];
-        int i = 0;
-        for (String fieldName : fieldNames) {
-            indexFieldNameArray[i] = fieldName;
-            i++;
-        }
-        Arrays.sort(indexFieldNameArray);
-        String index = MapKeyUtil.createKey(indexFieldNameArray);
-        if (this.getIndexs().contains(index)) {
-            return;
-        }
-        if (indexFieldNameArray.length > 0) {
-            this.addIndex(indexFieldNameArray);
-        }
-    }
-
-    protected static Map<String, Object> executeScript(Map<String, Object> oldRow, String script) {
-        if (script == null) {
-            return oldRow;
-        }
-        ScriptComponent scriptComponent = ScriptComponent.getInstance();
-        JSONObject msg = new JSONObject();
-        msg.putAll(oldRow);
-        scriptComponent.getService().executeScript(msg, script);
-        return msg;
-    }
-
-    protected AbstractMemoryTable loadData() {
-        AbstractMemoryTable memoryTable = null;
-        if (!isLarge) {
-            LOG.info(String.format("init ByteArrayMemoryTable."));
-            memoryTable = new ByteArrayMemoryTable();
-            loadData2Memory(memoryTable);
-        } else {
-            LOG.info(String.format("init MappedByteBufferTable."));
-//            memoryTable = new MappedByteBufferTable();
-//            loadData2Memory(memoryTable);
-            Date date = new Date();
-            try {
-                memoryTable = MappedByteBufferTable.Creator.newCreator(filePath, date, pollingTimeMinute.intValue()).create(table -> loadData2Memory(table));
-            } catch (IOException e) {
-                e.printStackTrace();
-            }
-
-        }
-        return memoryTable;
-    }
-
-    protected abstract void loadData2Memory(AbstractMemoryTable table);
-
-    @Override
-    public void destroy() {
-        super.destroy();
-        executorService.shutdown();
-    }
-
-    /**
-     * 设置索引
-     *
-     * @param indexs 字段名称,多个字段";"分隔
-     */
-    public void setIndex(String indexs) {
-        if (StringUtil.isEmpty(indexs)) {
-            return;
-        }
-        List<String> tmp = new ArrayList<>();
-        String[] values = indexs.split(";");
-        this.addIndex(tmp, values);
-        this.indexs = tmp;
-    }
-
-    /**
-     * 建议指定索引,会基于索引建立map,对于等值的判断,可以快速匹配
-     *
-     * @param fieldNames
-     */
-    private String addIndex(List<String> indexs, String... fieldNames) {
-        if (fieldNames == null) {
-            return null;
-        }
-        Arrays.sort(fieldNames);
-        String index = MapKeyUtil.createKey(fieldNames);
-        if (StringUtil.isNotEmpty(index)) {
-            indexs.add(index);
-        }
-        return index;
-    }
-
-    public Long getPollingTimeMinute() {
-        return pollingTimeMinute;
-    }
-
-    public void setPollingTimeMinute(Long pollingTimeMinute) {
-        this.pollingTimeMinute = pollingTimeMinute;
-    }
-
-    public List<String> getIndexs() {
-        return indexs;
-    }
-
-    public void setIndexs(List<String> indexs) {
-        this.indexs = indexs;
-    }
-
-    public AbstractMemoryTable getDataCache() {
-        return dataCache;
-    }
-
-    public boolean isLarge() {
-        return isLarge;
-    }
-
-    public void setLarge(boolean large) {
-        isLarge = large;
-    }
-
-    public DimIndex getNameListIndex() {
-        return nameListIndex;
-    }
-
-    public String getFilePath() {
-        return filePath;
-    }
-
-    public void setFilePath(String filePath) {
-        this.filePath = filePath;
-    }
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/AbstractProcShareDim.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/AbstractProcShareDim.java
deleted file mode 100644
index 41d56f1..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/AbstractProcShareDim.java
+++ /dev/null
@@ -1,109 +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.dim.model;
-
-import java.io.File;
-import java.nio.channels.FileChannel;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.cache.compress.AbstractMemoryTable;
-import org.apache.rocketmq.streams.dim.index.DimIndex;
-
-/**
- * @description 基于本地文件存储的多进程共享的dim
- */
-public abstract class AbstractProcShareDim extends AbstractDim {
-
-    static final Log logger = LogFactory.getLog(AbstractProcShareDim.class);
-
-    /**
-     * 多进程文件锁
-     */
-    private transient File fileLock;
-    /**
-     * done标, 多进程通过done文件通信
-     */
-    private transient File doneFile;
-    private transient FileChannel lockChannel;
-    String filePath;
-
-    static final Object lock = new Object();
-    static volatile boolean isFinishCreate = false;
-    static AbstractMemoryTable table;
-    static ScheduledExecutorService executorService;
-    static DimIndex dimIndex;
-    static String cycleStr;
-
-    public AbstractProcShareDim(String filePath) {
-        super();
-        this.filePath = filePath;
-    }
-
-    @Override
-    protected boolean initConfigurable() {
-        if (executorService == null) {
-            synchronized (lock) {
-                if (executorService == null) {
-                    executorService = new ScheduledThreadPoolExecutor(1);
-                    executorService.scheduleAtFixedRate(new Runnable() {
-                        @Override
-                        public void run() {
-                            loadNameList();
-                        }
-                    }, pollingTimeMinute, pollingTimeMinute, TimeUnit.MINUTES);
-                }
-            }
-        }
-//        boolean old = Boolean.parseBoolean(ComponentCreator.getProperties().getProperty(ConfigureFileKey.DIPPER_RUNNING_STATUS));
-//        ComponentCreator.getProperties().put(ConfigureFileKey.DIPPER_RUNNING_STATUS, false);
-//        boolean success = super.initConfigurable();
-//        ComponentCreator.getProperties().put(ConfigureFileKey.DIPPER_RUNNING_STATUS, old);
-        return true;
-    }
-
-    protected void loadNameList() {
-        try {
-            logger.info(getConfigureName() + " begin polling data");
-            //全表数据
-            AbstractMemoryTable dataCacheVar = loadData();
-            table = dataCacheVar;
-            this.nameListIndex = buildIndex(dataCacheVar);
-            this.columnNames = this.dataCache.getCloumnName2Index().keySet();
-        } catch (Exception e) {
-            logger.error("Load configurables error:" + e.getMessage(), e);
-        }
-    }
-
-    private void getCycleStr(Date date) {
-        if (pollingTimeMinute >= 1 && pollingTimeMinute < 60) {
-            SimpleDateFormat format = new SimpleDateFormat("yyyyMMddHHmm");
-            cycleStr = format.format(date);
-        } else if (pollingTimeMinute >= 60 && pollingTimeMinute < (60 * 24)) {
-            SimpleDateFormat format = new SimpleDateFormat("yyyyMMddHH");
-            cycleStr = format.format(date);
-        } else if (pollingTimeMinute >= (60 * 24)) {
-            SimpleDateFormat format = new SimpleDateFormat("yyyyMMdd");
-            cycleStr = format.format(date);
-        }
-    }
-
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/DBDim.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/DBDim.java
deleted file mode 100644
index 48b2bd9..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/DBDim.java
+++ /dev/null
@@ -1,160 +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.dim.model;
-
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.cache.compress.AbstractMemoryTable;
-import org.apache.rocketmq.streams.common.configurable.annotation.ENVDependence;
-import org.apache.rocketmq.streams.common.utils.IPUtil;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.db.driver.DriverBuilder;
-import org.apache.rocketmq.streams.db.driver.JDBCDriver;
-import org.apache.rocketmq.streams.db.driver.batchloader.BatchRowLoader;
-import org.apache.rocketmq.streams.db.driver.batchloader.IRowOperator;
-
-public class DBDim extends AbstractDim {
-
-    private static final Log LOG = LogFactory.getLog(DBDim.class);
-
-    private String jdbcdriver = "com.mysql.jdbc.Driver";
-
-    @ENVDependence
-    private String url;
-
-    @ENVDependence
-    private String userName;
-
-    @ENVDependence
-    private String password;
-
-    private String sql;//sql 会被定时执行
-
-    protected String idFieldName;
-
-    private static transient AtomicInteger nameCreator = new AtomicInteger(0);
-
-    /**
-     * 是否支持批量查找
-     */
-    protected transient Boolean supportBatch = false;
-
-    public DBDim() {
-        this.setConfigureName(MapKeyUtil.createKey(IPUtil.getLocalIdentification(), System.currentTimeMillis() + "",
-            nameCreator.incrementAndGet() + ""));
-        this.setType(TYPE);
-    }
-
-    @Override
-    protected void loadData2Memory(AbstractMemoryTable tableCompress) {
-        if (StringUtil.isNotEmpty(idFieldName)) {
-            BatchRowLoader batchRowLoader = new BatchRowLoader(idFieldName, sql, new IRowOperator() {
-                @Override
-                public synchronized void doProcess(Map<String, Object> row) {
-                    tableCompress.addRow(row);
-                }
-            });
-            batchRowLoader.startLoadData();
-            return;
-        }
-        List<Map<String, Object>> rows = executeQuery();
-
-        for (Map<String, Object> row : rows) {
-            tableCompress.addRow(row);
-        }
-    }
-
-    protected List<Map<String, Object>> executeQuery() {
-        JDBCDriver resource = createResouce();
-        try {
-            List<Map<String, Object>> result = resource.queryForList(sql);
-            ;
-            LOG.info("load configurable's count is " + result.size());
-            return result;
-        } finally {
-            if (resource != null) {
-                resource.destroy();
-            }
-        }
-
-    }
-
-    protected JDBCDriver createResouce() {
-        return DriverBuilder.createDriver(jdbcdriver, url, userName, password);
-    }
-
-    public void setJdbcdriver(String jdbcdriver) {
-        this.jdbcdriver = jdbcdriver;
-    }
-
-    public void setUrl(String url) {
-        this.url = url;
-    }
-
-    public void setUserName(String userName) {
-        this.userName = userName;
-    }
-
-    public void setPassword(String password) {
-        this.password = password;
-    }
-
-    public void setSql(String sql) {
-        this.sql = sql;
-    }
-
-    public String getJdbcdriver() {
-        return jdbcdriver;
-    }
-
-    public String getUrl() {
-        return url;
-    }
-
-    public String getUserName() {
-        return userName;
-    }
-
-    public String getPassword() {
-        return password;
-    }
-
-    public String getSql() {
-        return sql;
-    }
-
-    public Boolean getSupportBatch() {
-        return supportBatch;
-    }
-
-    public String getIdFieldName() {
-        return idFieldName;
-    }
-
-    public void setIdFieldName(String idFieldName) {
-        this.idFieldName = idFieldName;
-    }
-
-    public void setSupportBatch(Boolean supportBatch) {
-        this.supportBatch = supportBatch;
-    }
-
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/FileDim.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/FileDim.java
deleted file mode 100644
index 325ba36..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/FileDim.java
+++ /dev/null
@@ -1,58 +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.dim.model;
-
-import com.alibaba.fastjson.JSON;
-import com.alibaba.fastjson.JSONObject;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.rocketmq.streams.common.cache.compress.AbstractMemoryTable;
-import org.apache.rocketmq.streams.common.utils.FileUtil;
-
-public class FileDim extends AbstractDim {
-    protected String filePath;
-
-    @Override
-    protected void loadData2Memory(AbstractMemoryTable tableCompress) {
-        List<String> rows = FileUtil.loadFileLine(filePath);
-        for (String row : rows) {
-            JSONObject jsonObject = JSON.parseObject(row);
-            Map<String, Object> values = new HashMap<>();
-            for (String key : jsonObject.keySet()) {
-                values.put(key, jsonObject.getString(key));
-            }
-            tableCompress.addRow(values);
-        }
-    }
-
-    public static void main(String[] args) {
-        List<String> lines = FileUtil.loadFileLine("/tmp/data_model_extractor_config.txt");
-        for (String row : lines) {
-            JSONObject jsonObject = JSON.parseObject(row);
-            System.out.println(jsonObject);
-        }
-    }
-
-    public String getFilePath() {
-        return filePath;
-    }
-
-    public void setFilePath(String filePath) {
-        this.filePath = filePath;
-    }
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/IDataCache.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/IDataCache.java
deleted file mode 100644
index ba3a71e..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/IDataCache.java
+++ /dev/null
@@ -1,26 +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.dim.model;
-
-/**
- * @author zengyu.cw
- * @program rocketmq-streams-apache
- * @create 2021-11-17 09:43
- * @description
- */
-public interface IDataCache {
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/IDimSource.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/IDimSource.java
deleted file mode 100644
index 5b1d949..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/model/IDimSource.java
+++ /dev/null
@@ -1,37 +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.dim.model;
-
-import java.util.List;
-
-/**
- * @author zengyu.cw
- * @program rocketmq-streams-apache
- * @create 2021-11-17 09:42:20
- * @description
- */
-public interface IDimSource<InputRecord> {
-
-    public boolean put(InputRecord record) throws Exception;
-
-    public int batchPut(List<InputRecord> recordList) throws Exception;
-
-    public IDataCache initCache();
-
-    public boolean buildIndex(IDataCache cache);
-
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/service/IDimService.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/service/IDimService.java
deleted file mode 100644
index 02921df..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/service/IDimService.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.dim.service;
-
-import java.util.List;
-import java.util.Map;
-
-public interface IDimService {
-
-    /**
-     * 做维表join,关系通过表达式表示,返回所有匹配的行。因为msg没有key,表达式中,以下标表示key,如0,1,2。
-     *
-     * @param dimName       维表的名称
-     * @param expressionStr 表达式(0,functionName,filedName)&(1,functionName,filedName)|(2,functionName,filedName)
-     * @param msgs          流数据
-     * @return 符合匹配条件的所有行
-     */
-    Map<String, Object> match(String dimName, String expressionStr, Object... msgs);
-
-    /**
-     * 做维表join,关系通过表达式表示,返回所有匹配的行。
-     *
-     * @param dimName       维表的名称
-     * @param expressionStr 表达式,varName是msg中的key名称(varName,functionName,filedName)&(varName,functionName,filedName)|(varName,functionName,filedName)
-     * @param msgs          流数据
-     * @return 符合匹配条件的所有行
-     */
-    List<Map<String, Object>> matchSupportMultiRow(String dimName, String expressionStr, Map<String, Object> msgs);
-
-    /**
-     * 做维表join,关系通过表达式表示,返回匹配的一行数据,如果有多行匹配,只返回第一行。
-     *
-     * @param dimName       维表的名称
-     * @param expressionStr 表达式,varName是msg中的key名称(varName,functionName,filedName)&(varName,functionName,filedName)|(varName,functionName,filedName)
-     * @param msgs          流数据
-     * @return 返回匹配的一行数据,如果有多行匹配,只返回第一行。
-     */
-    Map<String, Object> match(String dimName, String expressionStr, Map<String, Object> msgs);
-
-    /**
-     * 做维表join,关系通过表达式表示,返回匹配的全部数据。
-     *
-     * @param dimName       维表的名称
-     * @param expressionStr 表达式,varName是msg中的key名称(varName,functionName,filedName)&(varName,functionName,filedName)|(varName,functionName,filedName)
-     * @param msgs          流数据
-     * @param script        对维表字段做处理的函数,在执行表达式前需要先对维表字段做处理,如fiedlName=trim(fieldName)
-     * @return 返回匹配的一行数据,如果有多行匹配,只返回第一行。
-     */
-    List<Map<String, Object>> matchSupportMultiRow(String dimName,
-        String expressionStr, Map<String, Object> msgs, String script);
-}
diff --git a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/service/impl/DimServiceImpl.java b/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/service/impl/DimServiceImpl.java
deleted file mode 100644
index 30201d2..0000000
--- a/rocketmq-streams-dim/src/main/java/org/apache/rocketmq/streams/dim/service/impl/DimServiceImpl.java
+++ /dev/null
@@ -1,90 +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.dim.service.impl;
-
-import com.alibaba.fastjson.JSONObject;
-import java.util.List;
-import java.util.Map;
-import org.apache.rocketmq.streams.configurable.ConfigurableComponent;
-import org.apache.rocketmq.streams.dim.model.AbstractDim;
-import org.apache.rocketmq.streams.dim.service.IDimService;
-
-public class DimServiceImpl implements IDimService {
-    protected ConfigurableComponent configurableComponent;
-
-    public DimServiceImpl(ConfigurableComponent configurableComponent) {
-        this.configurableComponent = configurableComponent;
-    }
-
-    /**
-     * 传入要比对的字段,进行规则匹配。字段和名单的比对逻辑,写在规则中
-     *
-     * @param msgs 字段名默认为数组的索引,如1,2,3
-     * @return
-     */
-    @Override
-    public Map<String, Object> match(String dimName, String expressionStr, Object... msgs) {
-        if (msgs == null || msgs.length == 0) {
-            return null;
-        }
-        int i = 0;
-        JSONObject jsonObject = new JSONObject();
-        for (Object o : msgs) {
-            jsonObject.put(i + "", o);
-            i++;
-        }
-        return match(dimName, expressionStr, jsonObject);
-    }
-
-    @Override
-    public List<Map<String, Object>> matchSupportMultiRow(String dimName, String expressionStr, Map<String, Object> msgs) {
-        return matchSupportMultiRow(dimName, expressionStr, msgs, null);
-    }
-
-    @Override
-    public List<Map<String, Object>> matchSupportMultiRow(String dimName, String expressionStr, Map<String, Object> msgs, String script) {
-        JSONObject jsonObject = createJsonable(msgs);
-        AbstractDim nameList = configurableComponent.queryConfigurable(AbstractDim.TYPE, dimName);
-        if (nameList != null) {
-            return nameList.matchExpression(expressionStr, jsonObject, true, script);
-        } else {
-            return null;
-        }
-    }
-
-    @Override
-    public Map<String, Object> match(String nameListName, String expressionStr, Map<String, Object> parameters) {
-        JSONObject jsonObject = createJsonable(parameters);
-        AbstractDim nameList = configurableComponent.queryConfigurable(AbstractDim.TYPE, nameListName);
-        if (nameList != null) {
-            return nameList.matchExpression(expressionStr, jsonObject);
-        } else {
-            return null;
-        }
-    }
-
-    private JSONObject createJsonable(Map<String, Object> parameters) {
-        JSONObject jsonObject = null;
-        if (parameters instanceof JSONObject) {
-            jsonObject = (JSONObject)parameters;
-        } else {
-            jsonObject.putAll(parameters);
-        }
-        return jsonObject;
-    }
-
-}
diff --git a/rocketmq-streams-dim/src/test/java/com/aliyun/service/ConfigureLoaderTest.java b/rocketmq-streams-dim/src/test/java/com/aliyun/service/ConfigureLoaderTest.java
deleted file mode 100644
index b671bc9..0000000
--- a/rocketmq-streams-dim/src/test/java/com/aliyun/service/ConfigureLoaderTest.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 com.aliyun.service;
-
-import java.io.IOException;
-import org.junit.Test;
-
-public class ConfigureLoaderTest {
-
-    @Test
-    public void testLoadResource() throws IOException {
-        //        BufferedReader br = new BufferedReader(new InputStreamReader(RuleEngineRunner.class.getClassLoader
-        // ().getResourceAsStream(".")));
-        //        String line = br.readLine();
-        //        while (line != null) {
-        //            System.out.println(line);
-        //            line = br.readLine();
-        //        }
-    }
-}
diff --git a/rocketmq-streams-dim/src/test/java/com/aliyun/service/ExpressionExecutorTest.java b/rocketmq-streams-dim/src/test/java/com/aliyun/service/ExpressionExecutorTest.java
deleted file mode 100644
index 92b10eb..0000000
--- a/rocketmq-streams-dim/src/test/java/com/aliyun/service/ExpressionExecutorTest.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 com.aliyun.service;
-
-import com.alibaba.fastjson.JSONObject;
-import java.io.File;
-import org.apache.rocketmq.streams.filter.FilterComponent;
-import org.apache.rocketmq.streams.filter.builder.ExpressionBuilder;
-import org.apache.rocketmq.streams.filter.operator.expression.SimpleExpression;
-import org.junit.Test;
-
-public class ExpressionExecutorTest {
-    private static final String CREDIBLE_PROPERTIES = "credible" + File.separator + "credible.properties";
-    private FilterComponent filterComponent;
-
-    private String namespace = "test.credible.net.vistor";
-    private String ruleNameSpace = "credible.rule.net.vistor";
-    private String selectorName = "credible.selector.net.vistor";
-    private String selectorExpression = "(host_uuid,=,0a2153e2-e45c-403f-8d5f-d811f400c3fb)";
-    private String procWriteList = "credible.namelist.proc";
-    private String netWriteList = "credible.namelist.net.vistor";
-
-    private String ruleExpression =
-        "(proc_path,in_resouce," + namespace + "->" + procWriteList + ")&(inner_message,not_in_expression_resouce,'"
-            + namespace + "->" + netWriteList
-            + "->(visitor_ip,=,dest_ip)&(visitor_port,=,dest_port)&(proc_path,=,program_path)')";
-
-    //    @Test
-    //    public void parseExpression() {
-    //        List<Expression> expressions = new ArrayList<>();
-    //        List<RelationExpression> relationExpressions = new ArrayList<>();
-    //        Expression expression = ExpressionBuilder.createExpression("namespace", ruleExpression,
-    // expressions,
-    //            relationExpressions);
-    //    }
-
-    public ExpressionExecutorTest() {
-        //        FilterComponent filterComponent= new FilterComponent();
-        //        filterComponent.init(CREDIBLE_PROPERTIES);
-        //        filterComponent.start(null);
-        //        this.filterComponent=filterComponent;
-    }
-
-    @Test
-    public void testExecutor() {
-        System.out.println("hello wolrd");
-        JSONObject msg = new JSONObject();
-        msg.put("ip", "1.1.1.1");
-        boolean match = ExpressionBuilder.executeExecute(new SimpleExpression("ip", "=", "1.1.1.1"), msg);
-        System.out.println(match);
-    }
-
-    @Test
-    public void testRelationExecutor() {
-        JSONObject jsonObject = new JSONObject();
-        jsonObject.put("ip", "1.2.2.3");
-        jsonObject.put("uid", "1224");
-        jsonObject.put("vmip", "1.1.1.1");
-
-        boolean value =
-            ExpressionBuilder.executeExecute("namespace", "(ip,=,1.2.2.3)&((uid,=,12214)|(vmip,=,1.1.11.1))",
-                jsonObject);
-        System.out.println(value);
-    }
-}
diff --git a/rocketmq-streams-dim/src/test/java/com/aliyun/service/JsonParserTest.java b/rocketmq-streams-dim/src/test/java/com/aliyun/service/JsonParserTest.java
deleted file mode 100644
index 8f58a04..0000000
--- a/rocketmq-streams-dim/src/test/java/com/aliyun/service/JsonParserTest.java
+++ /dev/null
@@ -1,40 +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 com.aliyun.service;
-
-import com.alibaba.fastjson.JSON;
-import com.alibaba.fastjson.JSONArray;
-import com.alibaba.fastjson.JSONObject;
-import org.junit.Test;
-
-public class JsonParserTest {
-
-    @Test
-    public void testJson() {
-        String array =
-            "[{\"value\":\"groupname\",\"key\":\"group_name\"},{\"value\":\"username\",\"key\":\"user_name\"},"
-                + "{\"value\":\"seq\",\"key\":\"index\"},{\"value\":\"egroupid\",\"key\":\"egroup_id\"},"
-                + "{\"value\":\"filepath\",\"key\":\"file_path\"},{\"value\":\"groupid\",\"key\":\"group_id\"},"
-                + "{\"value\":\"pfilename\",\"key\":\"pfile_path\"},{\"value\":\"safe_mode\",\"key\":\"perm\"},"
-                + "{\"value\":\"cmdline\",\"key\":\"cmd_line\"}]";
-        String jsonStr =
-            "{\"className\":\"com.aliyun.filter.result.FieldReNameScript\",\"oldField2NewFiled\":" + array + "}";
-        JSONArray jsonObject = JSON.parseArray(array);
-        JSONObject js = JSON.parseObject(jsonStr);
-        System.out.println(js.toJSONString());
-    }
-}
diff --git a/rocketmq-streams-dim/src/test/java/com/aliyun/service/NameListFunctionTest.java b/rocketmq-streams-dim/src/test/java/com/aliyun/service/NameListFunctionTest.java
deleted file mode 100644
index f08aa45..0000000
--- a/rocketmq-streams-dim/src/test/java/com/aliyun/service/NameListFunctionTest.java
+++ /dev/null
@@ -1,99 +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 com.aliyun.service;
-
-import com.alibaba.fastjson.JSONObject;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.rocketmq.streams.common.cache.compress.KVAddress;
-import org.apache.rocketmq.streams.common.utils.NumberUtils;
-import org.apache.rocketmq.streams.dim.model.AbstractDim;
-import org.apache.rocketmq.streams.dim.model.DBDim;
-import org.junit.Test;
-
-public class NameListFunctionTest {
-
-    @Test
-    public void testInterge() {
-        KVAddress mapAddress = new KVAddress(128, 0);
-        byte[] bytes = mapAddress.createBytesIngoreFirstBit();
-        long rowIndex = NumberUtils.toLong(bytes);
-        KVAddress mapAddress1 = KVAddress.createMapAddressFromLongValue(rowIndex);
-        System.out.println(rowIndex);
-    }
-
-    @Test
-    public void testNameList() {
-        AbstractDim nameList = create();
-        JSONObject msg = new JSONObject();
-        msg.put("ip", "47.105.77.144");
-        msg.put("vpcId", "1");
-        msg.put("now", "2019-07-18 17:33:29");
-        long start = System.currentTimeMillis();
-    }
-
-    @Test
-    public void testNameList2() {
-        AbstractDim nameList = createMapping();
-        JSONObject msg = new JSONObject();
-        msg.put("levelFile", "aegis-vul_record:level");
-        msg.put("levelValue", "high");
-        msg.put("now", "2019-07-18 17:33:29");
-        long start = System.currentTimeMillis();
-    }
-
-    private AbstractDim create() {
-        DBDim dbNameList = new DBDim();
-        dbNameList.setNameSpace("soc");
-        dbNameList.setConfigureName("isoc_field_mappings");
-        dbNameList.setUrl("");
-        dbNameList.setUserName("");
-        dbNameList.setPassword("");
-        dbNameList.setSql("SELECT * FROM `ecs_info` WHERE STATUS=1 LIMIT 1");
-        List<String> ipFieldNames = new ArrayList<>();
-        ipFieldNames.add("public_ips");
-        ipFieldNames.add("inner_ips");
-        ipFieldNames.add("eip");
-        ipFieldNames.add("private_ips");
-        dbNameList.init();
-        return dbNameList;
-    }
-
-    @Test
-    public void testNameListAllRow() {
-        AbstractDim nameList = createMapping();
-        JSONObject msg = new JSONObject();
-        msg.put("levelFile", "aegis-vul_record:level");
-        msg.put("levelValue", "high");
-        msg.put("now", "2019-07-18 17:33:29");
-        long start = System.currentTimeMillis();
-
-    }
-
-    private AbstractDim createMapping() {
-        DBDim dbNameList = new DBDim();
-        dbNameList.setNameSpace("soc");
-        dbNameList.setConfigureName("isoc_field_mappings");
-        dbNameList.setUrl("");
-        dbNameList.setUserName("");
-        dbNameList.setPassword("");
-        dbNameList.setSql("select * from ads_yunsec_ti_url_all_df limit 100000");
-        dbNameList.init();
-        return dbNameList;
-    }
-
-}
diff --git a/rocketmq-streams-dim/src/test/java/com/aliyun/service/TableCompressTest.java b/rocketmq-streams-dim/src/test/java/com/aliyun/service/TableCompressTest.java
deleted file mode 100644
index e2ccf99..0000000
--- a/rocketmq-streams-dim/src/test/java/com/aliyun/service/TableCompressTest.java
+++ /dev/null
@@ -1,145 +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 com.aliyun.service;
-
-import com.alibaba.fastjson.JSONObject;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import org.apache.rocketmq.streams.common.cache.ByteArrayMemoryTable;
-import org.apache.rocketmq.streams.common.cache.compress.AbstractMemoryTable;
-import org.apache.rocketmq.streams.common.cache.compress.impl.MapAddressListKV;
-import org.junit.Test;
-
-import static org.junit.Assert.assertFalse;
-
-public class TableCompressTest {
-
-    @Test
-    public void testNameList() throws InterruptedException {
-        long startTime = System.currentTimeMillis();
-        ByteArrayMemoryTable table = new ByteArrayMemoryTable();
-        int rowSize = 40000000;
-        //加载数据
-        for (int i = 0; i < rowSize; i++) {
-            JSONObject msg = new JSONObject();
-            msg.put("name", "chris" + i);
-            msg.put("age", i);
-            msg.put("address", "fddfdsfdsffddsdfsfdsfFDFDFDSDFSFDDFDSFSfddsffdsdsffdsfdfdsfdsfd" + i);
-            long rowIndex = table.addRow(msg);
-        }
-
-        //建立索引和验证索引
-        MapAddressListKV index = new MapAddressListKV(table.getRowCount());
-        Iterator<AbstractMemoryTable.RowElement> it = table.newIterator();
-        int count = 0;
-        int countIndex = 0;
-        while (it.hasNext()) {
-            AbstractMemoryTable.RowElement rowElement = it.next();
-            Map<String, Object> row = rowElement.getRow();
-            index.addLongValue((String) row.get("name"), rowElement.getRowIndex());
-            String key = (String) row.get("name");
-            List<Long> rowIds = null;
-
-            try {
-
-                rowIds = index.getLongValue(key);
-            } catch (Exception e) {
-                System.out.println(countIndex);
-                e.printStackTrace();
-            }
-
-            long rowIndex = rowElement.getRowIndex();
-            Map<String, Object> map = null;
-            try {
-                map = table.getRow(rowIndex);
-            } catch (Exception e) {
-                System.out.println(countIndex + "   " + rowIndex);
-                e.printStackTrace();
-            }
-
-            if (!map.get("name").equals(row.get("name")) || !map.get("age").equals(row.get("age")) || !map.get("address").equals(row.get("address"))) {
-                assertFalse("索引可能存在问题,基于索引获取的值和原数据不匹配 " + count, true);
-            }
-            countIndex++;
-        }
-        long start = System.currentTimeMillis();
-        for (int i = 0; i < rowSize; i++) {
-            String name = "chris" + i;
-            List<Long> rowIds = index.getLongValue(name);
-            for (Long rowId : rowIds) {
-                table.getRow(rowId);
-            }
-        }
-        System.out.println("query time cost is " + (System.currentTimeMillis() - start));
-
-        //空间占用
-        double indexSize = ((double) index.calMemory()) / 1024;
-        System.out.println("原始数据大小(G):" + (((double) table.getByteCount()) / 1024 / 1024 / 1024 + indexSize) + "G");
-        double size = ((double) table.getCache().byteSize()) / 1024;
-        System.out.println("存储空间大小(G)" + (size + indexSize) + "G");
-        System.out.println("insert and query cost is " + (System.currentTimeMillis() - startTime));
-    }
-
-    /**
-     * 1000w:25333
-     * 2000w:55489
-     */
-    @Test
-    public void testTable() {
-        ByteArrayMemoryTable table = new ByteArrayMemoryTable();
-        int size = 10;
-        MapAddressListKV index = new MapAddressListKV(size);
-        long start = System.currentTimeMillis();
-
-        //加载数据
-        for (int i = 0; i < size; i++) {
-            JSONObject msg = new JSONObject();
-            msg.put("name", "chris" + i);
-            msg.put("age", i);
-            msg.put("address", "address-" + i);
-            long rowIndex = table.addRow(msg);
-            index.addLongValue("chris" + i, rowIndex);
-        }
-        Random random = new Random();
-        for (int i = 0; i < size; i++) {
-            int j = random.nextInt(50000000);
-            List<Long> rowIds = index.getLongValue("chris" + j);
-            if (rowIds == null || rowIds.size() == 0) {
-                continue;
-            }
-            Long rowIndex = rowIds.get(0);
-            table.getRow(rowIndex);
-        }
-        System.out.println("finish cost is " + (System.currentTimeMillis() - start));
-    }
-
-    @Test
-    public void testColumNull() throws InterruptedException {
-        ByteArrayMemoryTable table = new ByteArrayMemoryTable();
-        JSONObject msg = new JSONObject();
-        msg.put("name", "chris" + 0);
-        msg.put("age", 0);
-        msg.put("address", "fddfdsfdsffddsdfsfdsfFDFDFDSDFSFDDFDSFSfddsffdsdsffdsfdfdsfdsfd");
-        table.addRow(msg);
-        msg.put("address", null);
-        long rowIndex = table.addRow(msg);
-        Map<String, Object> row = table.getRow(rowIndex);
-        System.out.println(row.size());
-    }
-}
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 3685ff1..7a0cdd6 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 82d51d9..cc6611e 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;
 
 /**
  * 消费 rocketmq 中的数据,10s一个窗口,并按照 ProjectName 和 LogStore 两个字段联合分组统计,两个字段的值相同,分为一组。
@@ -38,7 +39,8 @@
      * 2、rocketmq allow create topic automatically.
      */
     public static void main(String[] args) {
-        ProducerFromFile.produce("data.txt",NAMESRV_ADDRESS, "windowTopic");
+
+        ProducerFromFile.produce("data.txt",NAMESRV_ADDRESS, "windowTopic", true);
 
         try {
             Thread.sleep(1000 * 3);
@@ -63,12 +65,12 @@
                 })
                 //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)
+                .waterMark(2)
                 .setLocalStorageOnly(true)
                 .toDataStream()
                 .toPrint(1)
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 2a4abdf..49857bc 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 62d760d..0000000
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/checkpoint/RemoteCheckpointExample.java
+++ /dev/null
@@ -1,97 +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.examples.aggregate.Constant.NAMESRV_ADDRESS;
-
-
-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 = "";
-
-    private static final String TOPIC_NAME = "mysql_checkpoint_topic";
-
-    private static final String GROUP_NAME = "mysql_checkpoint_group_id";
-
-    static  {
-        ComponentCreator.getProperties().put(ConfigureFileKey.CONNECT_TYPE, "DB");
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, URL);
-        //if you mysql version large than 8.0, use com.mysql.cj.jdbc.Driver as a driver.
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_DRIVER, "com.mysql.cj.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, TOPIC_NAME);
-        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(
-                TOPIC_NAME,
-                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)
-                .toDataStream()
-                .toPrint(1)
-                .start();
-    }
-
-}
diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqDimJoinExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqDimJoinExample.java
deleted file mode 100644
index 8bc87de..0000000
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqDimJoinExample.java
+++ /dev/null
@@ -1,33 +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.join;
-
-import org.apache.rocketmq.streams.client.StreamBuilder;
-
-public class RocketmqDimJoinExample {
-
-    public static void main(String[] args) {
-        StreamBuilder.dataStream("tmp", "tmp")
-            .fromRocketmq("TopicTest", "groupA", true, "localhost:9876")
-            .dimJoin("classpath://dim.txt", 10000L)
-            .on("ProjectName, =, project")
-            .toDataStream()
-            .toPrint()
-            .start();
-    }
-
-}
diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqJoinExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqJoinExample.java
index 8a149ac..f10306f 100644
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqJoinExample.java
+++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/join/RocketmqJoinExample.java
@@ -22,9 +22,9 @@
 public class RocketmqJoinExample {
     public static void main(String[] args) {
         DataStream left = StreamBuilder.dataStream("tmp", "tmp")
-            .fromRocketmq("TopicTest", "groupA", true, "localhost:9876");
-        DataStream right = StreamBuilder.dataStream("tmp", "tmp")
-            .fromRocketmq("TopicTest", "groupB", true, "localhost:9876");
+            .fromRocketmq("TopicTestJoin", "groupA", true, "localhost:9876", null);
+        DataStream right = StreamBuilder.dataStream("tmp", "tmp22")
+            .fromRocketmq("TopicTestJoin", "groupB", true, "localhost:9876", null);
 
         left.join(right)
             .on("(ProjectName,=,ProjectName)")
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 888fc69..673e9e8 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
@@ -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 java.util.Random;
 import java.util.concurrent.ExecutorService;
@@ -44,7 +45,7 @@
         producerPool.submit(new Runnable() {
             @Override
             public void run() {
-                Producer.produceInLoop(topic,"data.txt");
+                ProducerFromFile.produceInLoop("data.txt", NAMESRV_ADDRESS, topic, 100);
             }
         });
 
@@ -71,7 +72,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 33b0269..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 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 java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS;
-
-public class Producer {
-    private static final AtomicInteger count = new AtomicInteger(0);
-
-    /**
-     * total produce 1000 data.
-     *
-     * @param fileName
-     */
-    public static void produceInLoop(String topic, 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(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-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBoundedSource.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/SendMessageContinuously.java
similarity index 66%
rename from rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBoundedSource.java
rename to rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/SendMessageContinuously.java
index 1a38343..564313a 100644
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBoundedSource.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
@@ -14,19 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.streams.connectors;
 
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS;
+import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_TOPIC;
 
-/**
- * @description
- */
-public interface IBoundedSource{
-
-    /**
-     * reader完成时调用
-     * @param iSplit
-     */
-    void boundedFinishedCallBack(ISplit iSplit);
-
+public class SendMessageContinuously {
+    public static void main(String[] args) {
+        ProducerFromFile.produceInLoop("data.txt",NAMESRV_ADDRESS, RMQ_TOPIC, 100);
+    }
 }
diff --git a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBoundedSource.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/SendMessageOnce.java
similarity index 65%
copy from rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBoundedSource.java
copy to rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/send/SendMessageOnce.java
index 1a38343..70e6fc2 100644
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/IBoundedSource.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
@@ -14,19 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.streams.connectors;
 
-import org.apache.rocketmq.streams.common.channel.split.ISplit;
+import static org.apache.rocketmq.streams.examples.aggregate.Constant.NAMESRV_ADDRESS;
+import static org.apache.rocketmq.streams.examples.aggregate.Constant.RMQ_TOPIC;
 
-/**
- * @description
- */
-public interface IBoundedSource{
-
-    /**
-     * reader完成时调用
-     * @param iSplit
-     */
-    void boundedFinishedCallBack(ISplit iSplit);
-
+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/FileSourceExample.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/FileSourceExample.java
index 339dc91..be46e77 100644
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/FileSourceExample.java
+++ b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/FileSourceExample.java
@@ -33,7 +33,7 @@
                 .map(message -> message)
                 .filter(message -> ((JSONObject) message).getInteger("score") > 90)
                 .selectFields("name", "subject")
-                .toFile("./rocketmq-streams-examples/src/main/resources/result.txt")
+                .toPrint()
                 .start();
 
     }
diff --git a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/FileSourceExample1.java b/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/FileSourceExample1.java
deleted file mode 100644
index 1254f7c..0000000
--- a/rocketmq-streams-examples/src/main/java/org/apache/rocketmq/streams/examples/source/FileSourceExample1.java
+++ /dev/null
@@ -1,32 +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.source;
-
-import com.alibaba.fastjson.JSONObject;
-import org.apache.rocketmq.streams.client.StreamBuilder;
-import org.apache.rocketmq.streams.client.source.DataStreamSource;
-
-public class FileSourceExample1 {
-    public static void main(String[] args) {
-        DataStreamSource source = StreamBuilder.dataStream("namespace", "pipeline");
-        source.fromFile("score.txt", true)
-                .filter(message -> ((JSONObject)message).getInteger("score") > 90)
-                .selectFields("name", "class", "subject")
-                .toPrint()
-                .start();
-    }
-}
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 487beb5..2af89ed 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;
 
@@ -28,7 +28,9 @@
     private static String groupName = "groupName-1";
 
     public static void main(String[] args) {
-        ProducerFromFile.produce("data.txt",NAMESRV_ADDRESS, topicName);
+
+        ProducerFromFile.produce("data.txt",NAMESRV_ADDRESS, topicName, 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 4b27302..a77cbf4 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
@@ -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 java.util.Arrays;
 
@@ -31,7 +31,7 @@
      * 1、make sure your rocketmq server has been started.
      */
     public static void main(String[] args) {
-        ProducerFromFile.produce("data.txt", NAMESRV_ADDRESS, topicName);
+        ProducerFromFile.produce("data.txt", NAMESRV_ADDRESS, topicName, 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 fad9d2f..09f0572 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
@@ -20,7 +20,7 @@
 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.examples.aggregate.ProducerFromFile;
+import org.apache.rocketmq.streams.examples.send.ProducerFromFile;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -37,7 +37,8 @@
      * 1、make sure your rocketmq server has been started.
      */
     public static void main(String[] args) {
-        ProducerFromFile.produce("data.txt", NAMESRV_ADDRESS, topicName);
+        ProducerFromFile.produce("data.txt", NAMESRV_ADDRESS, topicName, 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 d117fb1..902692b 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,7 +20,7 @@
 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;
 
@@ -30,9 +30,10 @@
 
     public static void main(String[] args) {
         System.out.println("send data to rocketmq");
-        ProducerFromFile.produce("joinData-1.txt", NAMESRV_ADDRESS, topicName);
+        ProducerFromFile.produce("joinData-1.txt", NAMESRV_ADDRESS, topicName, false);
 
-        ProducerFromFile.produce("joinData-2.txt", NAMESRV_ADDRESS, topicName + 2);
+        ProducerFromFile.produce("joinData-2.txt", NAMESRV_ADDRESS, topicName + 2, true);
+
 
         try {
             Thread.sleep(1000 * 3);
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-examples/src/main/resources/score.txt b/rocketmq-streams-examples/src/main/resources/score.txt
deleted file mode 100644
index b2e90a1..0000000
--- a/rocketmq-streams-examples/src/main/resources/score.txt
+++ /dev/null
@@ -1,18 +0,0 @@
-{"name":"zhangsan","class":"3","subject":"math","score":90}
-{"name":"zhangsan","class":"3","subject":"history","score":81}
-{"name":"zhangsan","class":"3","subject":"english","score":91}
-{"name":"zhangsan","class":"3","subject":"chinese","score":70}
-{"name":"zhangsan","class":"3","subject":"political","score":84}
-{"name":"zhangsan","class":"3","subject":"geographic","score":99}
-{"name":"lisi","class":"3","subject":"math","score":76}
-{"name":"lisi","class":"3","subject":"history","score":83}
-{"name":"lisi","class":"3","subject":"english","score":82}
-{"name":"lisi","class":"3","subject":"chinese","score":92}
-{"name":"lisi","class":"3","subject":"political","score":97}
-{"name":"lisi","class":"3","subject":"geographic","score":89}
-{"name":"wangwu","class":"3","subject":"math","score":86}
-{"name":"wangwu","class":"3","subject":"history","score":88}
-{"name":"wangwu","class":"3","subject":"english","score":86}
-{"name":"wangwu","class":"3","subject":"chinese","score":93}
-{"name":"wangwu","class":"3","subject":"political","score":99}
-{"name":"wangwu","class":"3","subject":"geographic","score":88}
\ No newline at end of file
diff --git a/rocketmq-streams-filter/pom.xml b/rocketmq-streams-filter/pom.xml
index 8071852..e273d09 100755
--- a/rocketmq-streams-filter/pom.xml
+++ b/rocketmq-streams-filter/pom.xml
@@ -36,10 +36,6 @@
             <groupId>org.apache.rocketmq</groupId>
             <artifactId>rocketmq-streams-script</artifactId>
         </dependency>
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-db-operator</artifactId>
-        </dependency>
     </dependencies>
 
 
diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/ExpressionBuilder.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/ExpressionBuilder.java
index 96ead53..1d7ed40 100644
--- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/ExpressionBuilder.java
+++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/ExpressionBuilder.java
@@ -223,7 +223,7 @@
         addConfigurable2Map(rule.getVarMap(), ruleCreator.getVarList());
         addConfigurable2Map(rule.getExpressionMap(), ruleCreator.getExpressionList());
         addConfigurable2Map(rule.getMetaDataMap(), ruleCreator.getMetaDataList());
-        addConfigurable2Map(rule.getDataSourceMap(), ruleCreator.getDataSourceList());
+
         addConfigurable2Map(rule.getActionMap(), ruleCreator.getActionList());
         ruleCreator.setRootExpression(expression);
         rule = ruleCreator.createRule();
diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/RuleBuilder.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/RuleBuilder.java
index 50daf3a..3988ea1 100644
--- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/RuleBuilder.java
+++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/builder/RuleBuilder.java
@@ -16,14 +16,6 @@
  */
 package org.apache.rocketmq.streams.filter.builder;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.rocketmq.streams.common.channel.sink.ISink;
 import org.apache.rocketmq.streams.common.configurable.IConfigurable;
 import org.apache.rocketmq.streams.common.configurable.IConfigurableService;
 import org.apache.rocketmq.streams.common.datatype.DataType;
@@ -32,7 +24,6 @@
 import org.apache.rocketmq.streams.common.model.NameCreator;
 import org.apache.rocketmq.streams.common.utils.DataTypeUtil;
 import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.db.driver.JDBCDriver;
 import org.apache.rocketmq.streams.filter.contants.RuleElementType;
 import org.apache.rocketmq.streams.filter.operator.Rule;
 import org.apache.rocketmq.streams.filter.operator.action.Action;
@@ -44,6 +35,14 @@
 import org.apache.rocketmq.streams.filter.operator.var.InnerVar;
 import org.apache.rocketmq.streams.filter.operator.var.Var;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 /**
  * 通过这个工具可以快速创建一条规则。这个工具默认消息流的字段名=metadata的字段名
  */
@@ -54,7 +53,6 @@
     private List<MetaData> metaDataList = new ArrayList<>();//包含所有的metadata
     private List<Expression> expressionList = new ArrayList<>();//包含所有的表达式
     private List<Action> actionList = new ArrayList<>();//包含所有的action
-    private List<JDBCDriver> dataSourceList = new ArrayList<>();//包含所有的datasource
     private MetaData metaData;//输入消息的metadata
     private String namespace;//规则的命名空间
     private String ruleName;//规则的名字
@@ -62,8 +60,6 @@
     private String ruleCode;
     private String ruleTitle;
     private String ruleDescription;
-    private transient NameCreator actionNameCreator = new NameCreator();
-    private transient NameCreator dataSourceNameCreator = new NameCreator();
     private transient NameCreator metaDataNameCreator = new NameCreator();
 
     /**
@@ -164,8 +160,6 @@
             ruleEngineConfigurableService.queryConfigurableByType(RuleElementType.EXPRESSION.getType());
         this.metaDataList = ruleEngineConfigurableService.queryConfigurableByType(RuleElementType.METADATA.getType());
         this.actionList = ruleEngineConfigurableService.queryConfigurableByType(RuleElementType.ACTION.getType());
-        this.dataSourceList =
-            ruleEngineConfigurableService.queryConfigurableByType(RuleElementType.DATASOURCE.getType());
     }
 
     /**
@@ -397,7 +391,6 @@
         insertOrUpdate(ruleEngineConfigurableService, varList, Var.TYPE);
         insertOrUpdate(ruleEngineConfigurableService, expressionList, Expression.TYPE);
         insertOrUpdate(ruleEngineConfigurableService, actionList, Action.TYPE);
-        insertOrUpdate(ruleEngineConfigurableService, dataSourceList, ISink.TYPE);
         if (ruleEngineConfigurableService != null) {
             ruleEngineConfigurableService.insert(rule);
         }
@@ -520,10 +513,6 @@
         return actionList;
     }
 
-    public List<JDBCDriver> getDataSourceList() {
-        return dataSourceList;
-    }
-
     public String getNamespace() {
         return namespace;
     }
diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/RuleContext.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/RuleContext.java
index bd7d9b9..89a398e 100644
--- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/RuleContext.java
+++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/context/RuleContext.java
@@ -18,6 +18,22 @@
 
 import com.alibaba.fastjson.JSONObject;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.rocketmq.streams.common.configurable.IConfigurableService;
+import org.apache.rocketmq.streams.common.context.AbstractContext;
+import org.apache.rocketmq.streams.common.context.IMessage;
+import org.apache.rocketmq.streams.common.context.Message;
+import org.apache.rocketmq.streams.common.monitor.IMonitor;
+import org.apache.rocketmq.streams.common.monitor.TopologyFilterMonitor;
+import org.apache.rocketmq.streams.filter.function.expression.ExpressionFunction;
+import org.apache.rocketmq.streams.filter.operator.Rule;
+import org.apache.rocketmq.streams.filter.operator.action.Action;
+import org.apache.rocketmq.streams.filter.operator.expression.Expression;
+import org.apache.rocketmq.streams.filter.operator.var.Var;
+import org.apache.rocketmq.streams.script.function.model.FunctionConfigure;
+import org.apache.rocketmq.streams.script.function.service.impl.ScanFunctionService;
+
 import java.io.Serializable;
 import java.util.Properties;
 import java.util.Vector;
@@ -28,24 +44,6 @@
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.configurable.IConfigurableService;
-import org.apache.rocketmq.streams.common.context.AbstractContext;
-import org.apache.rocketmq.streams.common.context.IMessage;
-import org.apache.rocketmq.streams.common.context.Message;
-import org.apache.rocketmq.streams.common.metadata.MetaData;
-import org.apache.rocketmq.streams.common.metadata.MetaDataAdapter;
-import org.apache.rocketmq.streams.common.monitor.IMonitor;
-import org.apache.rocketmq.streams.common.monitor.TopologyFilterMonitor;
-import org.apache.rocketmq.streams.db.driver.JDBCDriver;
-import org.apache.rocketmq.streams.filter.function.expression.ExpressionFunction;
-import org.apache.rocketmq.streams.filter.operator.Rule;
-import org.apache.rocketmq.streams.filter.operator.action.Action;
-import org.apache.rocketmq.streams.filter.operator.expression.Expression;
-import org.apache.rocketmq.streams.filter.operator.var.Var;
-import org.apache.rocketmq.streams.script.function.model.FunctionConfigure;
-import org.apache.rocketmq.streams.script.function.service.impl.ScanFunctionService;
 
 public class RuleContext extends AbstractContext<Message> implements Serializable {
 
@@ -233,33 +231,7 @@
 
     }
 
-    public MetaData getMetaData(String name) {
-        MetaData metaData = rule.getMetaDataMap().get(name);
-        return metaData;
-    }
 
-    public JDBCDriver getDataSource(String name) {
-        return rule.getDataSourceMap().get(name);
-    }
-
-    /**
-     * @param name
-     * @return
-     */
-    public MetaDataAdapter getMetaDataAdapter(String name) {
-        MetaData md = getMetaData(name);
-        JDBCDriver dataSource = this.getDataSource(md.getDataSourceName());
-        MetaDataAdapter mda = new MetaDataAdapter(md, dataSource);
-        return mda;
-
-    }
-
-    public boolean containsVarName(String varName) {
-        if (varValueMap.containsKey(varName)) {
-            return true;
-        }
-        return false;
-    }
 
     /**
      * 获取变量值,内部使用,不能直接使用,获取变量的值需要用var.getVarValue()
diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/Rule.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/Rule.java
index 631756d..ab7ed69 100644
--- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/Rule.java
+++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/Rule.java
@@ -44,7 +44,6 @@
 import org.apache.rocketmq.streams.common.topology.model.AbstractRule;
 import org.apache.rocketmq.streams.common.topology.stages.FilterChainStage;
 import org.apache.rocketmq.streams.common.utils.TraceUtil;
-import org.apache.rocketmq.streams.db.driver.JDBCDriver;
 import org.apache.rocketmq.streams.filter.FilterComponent;
 import org.apache.rocketmq.streams.filter.operator.action.Action;
 import org.apache.rocketmq.streams.filter.operator.action.impl.SinkAction;
@@ -64,7 +63,6 @@
     @Deprecated
     private transient Map<String, Action> actionMap = new HashMap<>();
     private transient Map<String, MetaData> metaDataMap = new HashMap<>();
-    private transient volatile Map<String, JDBCDriver> dataSourceMap = new HashMap<>();
     private String expressionStr;//表达式
     protected transient Expression rootExpression;
 
@@ -84,7 +82,6 @@
         this.actionMap = rule.getActionMap();
         this.expressionMap = rule.getExpressionMap();
         this.metaDataMap = rule.getMetaDataMap();
-        this.dataSourceMap = rule.getDataSourceMap();
         this.setMsgMetaDataName(rule.getMsgMetaDataName());
         this.setExpressionStr(rule.getExpressionStr());
         this.setVarNames(rule.getVarNames());
@@ -104,7 +101,6 @@
         rule.expressionMap = expressionMap;
         rule.actionMap = actionMap;
         rule.metaDataMap = metaDataMap;
-        rule.dataSourceMap = dataSourceMap;
         rule.setActionNames(actionNames);
         rule.setVarNames(varNames);
         rule.setExpressionName(expressionName);
@@ -122,11 +118,6 @@
 
     @Override
     public void doProcessAfterRefreshConfigurable(IConfigurableService configurableService) {
-
-        if (isFinishVarAndExpression == false) {
-            this.dataSourceMap = configurableService.queryConfigurableMapByType(ISink.TYPE);
-
-        }
         initVar(configurableService);
         initExpression(configurableService);
         initAction(configurableService);
@@ -147,7 +138,6 @@
             }
         }
         for (Action action : this.actionMap.values()) {
-            action.setDataSourceMap(this.dataSourceMap);
             action.setMetaDataMap(this.metaDataMap);
         }
     }
@@ -271,8 +261,6 @@
             actionMap.put(configurable.getConfigureName(), (Action) configurable);
         } else if (MetaData.TYPE.equals(type)) {
             metaDataMap.put(configurable.getConfigureName(), (MetaData) configurable);
-        } else if (ISink.TYPE.equals(type)) {
-            dataSourceMap.put(configurable.getConfigureName(), (JDBCDriver) configurable);
         }
     }
 
@@ -300,9 +288,7 @@
         if (metaDataMap != null) {
             configurableList.addAll(metaDataMap.values());
         }
-        if (dataSourceMap != null) {
-            configurableList.addAll(dataSourceMap.values());
-        }
+
         return configurableList;
     }
 
@@ -338,13 +324,6 @@
         this.metaDataMap = metaDataMap;
     }
 
-    public Map<String, JDBCDriver> getDataSourceMap() {
-        return dataSourceMap;
-    }
-
-    public void setDataSourceMap(Map<String, JDBCDriver> dataSourceMap) {
-        this.dataSourceMap = dataSourceMap;
-    }
 
     private transient static FilterComponent filterComponent = FilterComponent.getInstance();
 
@@ -441,9 +420,6 @@
         if (metaDataMap.values() != null) {
             pipelineBuilder.addConfigurables(metaDataMap.values());
         }
-        if (dataSourceMap.values() != null) {
-            pipelineBuilder.addConfigurables(dataSourceMap.values());
-        }
     }
 
     /**
diff --git a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/Action.java b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/Action.java
index 8cd0402..ff4b9f9 100644
--- a/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/Action.java
+++ b/rocketmq-streams-filter/src/main/java/org/apache/rocketmq/streams/filter/operator/action/Action.java
@@ -21,14 +21,12 @@
 import org.apache.rocketmq.streams.common.configurable.BasedConfigurable;
 import org.apache.rocketmq.streams.common.configurable.IConfigurable;
 import org.apache.rocketmq.streams.common.metadata.MetaData;
-import org.apache.rocketmq.streams.db.driver.JDBCDriver;
 import org.apache.rocketmq.streams.filter.operator.var.Var;
 
 public abstract class Action<T> extends BasedConfigurable implements IConfigurableAction<T>, IConfigurable {
     public static final String TYPE = "action";
 
     private transient Map<String, MetaData> metaDataMap = new HashMap<>();
-    private transient volatile Map<String, JDBCDriver> dataSourceMap = new HashMap<>();
     private transient volatile Map<String, Var> varMap = new HashMap<>();
     public Action() {
         setType(TYPE);
@@ -42,13 +40,6 @@
         this.metaDataMap = metaDataMap;
     }
 
-    public Map<String, JDBCDriver> getDataSourceMap() {
-        return dataSourceMap;
-    }
-
-    public void setDataSourceMap(Map<String, JDBCDriver> dataSourceMap) {
-        this.dataSourceMap = dataSourceMap;
-    }
 
     public Map<String, Var> getVarMap() {
         return varMap;
diff --git a/rocketmq-streams-filter/src/test/java/org/apache/rocketmq/streams/filter/operator/RuleTest.java b/rocketmq-streams-filter/src/test/java/org/apache/rocketmq/streams/filter/operator/RuleTest.java
deleted file mode 100644
index ca7900a..0000000
--- a/rocketmq-streams-filter/src/test/java/org/apache/rocketmq/streams/filter/operator/RuleTest.java
+++ /dev/null
@@ -1,39 +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.filter.operator;
-
-import com.alibaba.fastjson.JSONObject;
-import org.apache.rocketmq.streams.configurable.ConfigurableComponent;
-import org.apache.rocketmq.streams.filter.builder.RuleBuilder;
-import org.junit.Test;
-
-import static junit.framework.TestCase.assertTrue;
-
-public class RuleTest {
-    @Test
-    public void testRule() {
-        ConfigurableComponent configurableComponent = ConfigurableComponent.getInstance("namespace");
-        JSONObject jsonObject = new JSONObject();
-        jsonObject.put("ip", "1.2.2.3");
-        jsonObject.put("uid", 1224);
-        jsonObject.put("vmip", "1.1.1.1");
-
-        RuleBuilder ruleBuilder = new RuleBuilder("namespace", "ruleName", "(ip,==,1.2.2.3)&((uid,=,1224)|(vmip,=,1.1.11.1))", "uid;int");
-        boolean isMatch = ruleBuilder.generateRule(configurableComponent.getService()).execute(jsonObject);
-        assertTrue(isMatch);
-    }
-}
diff --git a/rocketmq-streams-lease/pom.xml b/rocketmq-streams-lease/pom.xml
deleted file mode 100755
index 8921c8e..0000000
--- a/rocketmq-streams-lease/pom.xml
+++ /dev/null
@@ -1,38 +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">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.rocketmq</groupId>
-        <artifactId>rocketmq-streams</artifactId>
-        <version>1.0.2-preview-SNAPSHOT</version>
-    </parent>
-    <artifactId>rocketmq-streams-lease</artifactId>
-    <name>ROCKETMQ STREAMS :: lease</name>
-    <packaging>jar</packaging>
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-channel-db</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-db-operator</artifactId>
-        </dependency>
-    </dependencies>
-</project>
diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/LeaseComponent.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/LeaseComponent.java
deleted file mode 100644
index 614b5aa..0000000
--- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/LeaseComponent.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.lease;
-
-import java.util.Properties;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.component.AbstractComponent;
-import org.apache.rocketmq.streams.common.component.ComponentCreator;
-import org.apache.rocketmq.streams.common.component.ConfigureDescriptor;
-import org.apache.rocketmq.streams.common.configure.ConfigureFileKey;
-import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.configurable.service.ConfigurableServcieType;
-import org.apache.rocketmq.streams.lease.service.ILeaseService;
-import org.apache.rocketmq.streams.lease.service.ILeaseStorage;
-import org.apache.rocketmq.streams.lease.service.impl.LeaseServiceImpl;
-import org.apache.rocketmq.streams.lease.service.impl.MockLeaseImpl;
-import org.apache.rocketmq.streams.lease.service.storages.DBLeaseStorage;
-import org.apache.rocketmq.streams.serviceloader.ServiceLoaderComponent;
-
-/**
- * 通过db实现租约和锁,可以更轻量级,减少其他中间件的依赖 使用主备场景,只有一个实例运行,当当前实例挂掉,在一定时间内,会被其他实例接手 也可以用于全局锁
- *
- * @date 1/9/19
- */
-public class LeaseComponent extends AbstractComponent<ILeaseService> {
-
-    private static LeaseComponent leaseComponent = null;
-    private static final Log LOG = LogFactory.getLog(LeaseComponent.class);
-    private ILeaseService leaseService;
-
-    public LeaseComponent() {
-        initConfigurableServiceDescriptor();
-        addConfigureDescriptor(
-            new ConfigureDescriptor(CONNECT_TYPE, false, ConfigurableServcieType.DEFAULT_SERVICE_NAME));
-    }
-
-    public static LeaseComponent getInstance() {
-        if (leaseComponent == null) {
-            synchronized (LeaseComponent.class) {
-                if (leaseComponent == null) {
-                    leaseComponent = ComponentCreator.getComponent(null, LeaseComponent.class);
-                }
-            }
-        }
-        return leaseComponent;
-    }
-
-    @Override
-    public boolean stop() {
-        return true;
-    }
-
-    @Override
-    public ILeaseService getService() {
-        return leaseService;
-    }
-
-    @Override
-    protected boolean startComponent(String namespace) {
-        return true;
-    }
-
-    @Override
-    protected boolean initProperties(Properties properties) {
-        String connectType = properties.getProperty(JDBC_URL);
-        if (StringUtil.isEmpty(connectType)) {
-            this.leaseService = new MockLeaseImpl();
-            return true;
-        }
-
-        LeaseServiceImpl leaseService = new LeaseServiceImpl();
-        String storageName = ComponentCreator.getProperties().getProperty(ConfigureFileKey.LEASE_STORAGE_NAME);
-        ILeaseStorage storasge = null;
-        if (StringUtil.isEmpty(storageName)) {
-            String jdbc = properties.getProperty(AbstractComponent.JDBC_DRIVER);
-            String url = properties.getProperty(AbstractComponent.JDBC_URL);
-            String userName = properties.getProperty(AbstractComponent.JDBC_USERNAME);
-            String password = properties.getProperty(AbstractComponent.JDBC_PASSWORD);
-            storasge = new DBLeaseStorage(jdbc, url, userName, password);
-        } else {
-            storasge = (ILeaseStorage)ServiceLoaderComponent.getInstance(ILeaseStorage.class).loadService(storageName);
-        }
-        leaseService.setLeaseStorage(storasge);
-        this.leaseService = leaseService;
-        return true;
-    }
-}
diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/model/LeaseInfo.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/model/LeaseInfo.java
deleted file mode 100644
index 469a711..0000000
--- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/model/LeaseInfo.java
+++ /dev/null
@@ -1,127 +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.lease.model;
-
-/**
- * 租约对象,需要创建租约表。
- */
-
-import java.io.Serializable;
-import java.util.Date;
-
-public class LeaseInfo implements Serializable {
-    private static final long serialVersionUID = 665608838255753618L;
-    private Long id;
-    private Date createTime;
-    private Date updateTime;
-    private String leaseName;//租约名称,多个进程共享一个租约,只要名称相同即可
-    private String leaseUserIp;//区分不同的租约实体,以前默认用ip,但一个机器多个进程的情况下,用ip会区分不开,后续会加上进程号
-    private Date leaseEndDate;//租约到期时间
-    private int status;//租约的有效状态
-    private long version;//版本,通过版本保证更新原子性
-
-    public LeaseInfo() {
-    }
-
-    /**
-     * 建表语句
-     *
-     * @return
-     */
-    public static String createTableSQL() {
-        return "CREATE TABLE  IF NOT EXISTS  `lease_info` (\n"
-            + "  `id` bigint(20) unsigned NOT NULL AUTO_INCREMENT COMMENT '主键',\n"
-            + "  `gmt_create` datetime NOT NULL COMMENT '创建时间',\n"
-            + "  `gmt_modified` datetime NOT NULL COMMENT '修改时间',\n"
-            + "  `lease_name` varchar(255) NOT NULL COMMENT '租约名称',\n"
-            + "  `lease_user_ip` varchar(255) NOT NULL COMMENT '租者IP',\n"
-            + "  `lease_end_time` varchar(255) NOT NULL COMMENT '租约到期时间',\n"
-            + "  `status` int(11) NOT NULL DEFAULT '1' COMMENT '状态',\n"
-            + "  `version` bigint(20) NOT NULL COMMENT '版本',\n"
-            + "  `candidate_lease_ip` varchar(255) DEFAULT NULL COMMENT '候选租约ip',\n"
-            + "  PRIMARY KEY (`id`),\n"
-            + "  UNIQUE KEY `uk_name` (`lease_name`)\n"
-            + ") ENGINE=InnoDB AUTO_INCREMENT=8150 DEFAULT CHARSET=utf8 COMMENT='租约信息'\n"
-            + ";";
-    }
-
-    public Long getId() {
-        return this.id;
-    }
-
-    public void setId(Long id) {
-        this.id = id;
-    }
-
-    public Date getCreateTime() {
-        return this.createTime;
-    }
-
-    public void setCreateTime(Date createTime) {
-        this.createTime = createTime;
-    }
-
-    public Date getUpdateTime() {
-        return this.updateTime;
-    }
-
-    public void setUpdateTime(Date updateTime) {
-        this.updateTime = updateTime;
-    }
-
-    public String getLeaseUserIp() {
-        return this.leaseUserIp;
-    }
-
-    public void setLeaseUserIp(String leaseUserIp) {
-        this.leaseUserIp = leaseUserIp;
-    }
-
-    public Date getLeaseEndDate() {
-        return this.leaseEndDate;
-    }
-
-    public void setLeaseEndDate(Date leaseEndDate) {
-        this.leaseEndDate = leaseEndDate;
-    }
-
-    public int getStatus() {
-        return this.status;
-    }
-
-    public void setStatus(int status) {
-        this.status = status;
-    }
-
-    public String getLeaseName() {
-        return this.leaseName;
-    }
-
-    public void setLeaseName(String leaseName) {
-        this.leaseName = leaseName;
-    }
-
-    public long getVersion() {
-        return this.version;
-    }
-
-    public void setVersion(long version) {
-        this.version = version;
-    }
-
-}
-
diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseGetCallback.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseGetCallback.java
deleted file mode 100644
index ad63dde..0000000
--- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseGetCallback.java
+++ /dev/null
@@ -1,30 +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.lease.service;
-
-import java.util.Date;
-
-public interface ILeaseGetCallback {
-
-    /**
-     * 当成功获取租约时,回调接口
-     *
-     * @param nextLeaseDate 租约到期时间
-     */
-    void callback(Date nextLeaseDate);
-
-}
diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseService.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseService.java
deleted file mode 100644
index 56c0981..0000000
--- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseService.java
+++ /dev/null
@@ -1,134 +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.lease.service;
-
-import java.util.List;
-import org.apache.rocketmq.streams.lease.model.LeaseInfo;
-
-/**
- * 通过db实现租约和锁,可以更轻量级,减少其他中间件的依赖 使用主备场景,只有一个实例运行,当当前实例挂掉,在一定时间内,会被其他实例接手 也可以用于全局锁
- */
-public interface ILeaseService {
-
-    /**
-     * 默认锁定时间
-     */
-    static final int DEFALUT_LOCK_TIME = 60 * 5;
-
-    /**
-     * 检查某用户当前时间是否具有租约。这个方法是纯内存操作,无性能开销
-     *
-     * @return true,租约有效;false,租约无效
-     */
-    boolean hasLease(String name);
-
-    /**
-     * 申请租约,会启动一个线程,不停申请租约,直到申请成功。 申请成功后,每 租期/2 续约。 如果目前被其他租户获取租约,只有在对方租约失效,后才允许新的租户获取租约
-     *
-     * @param name 租约名称,无特殊要求,相同名称会竞争租约
-     */
-    void startLeaseTask(String name);
-
-    /**
-     * 申请租约,会启动一个线程,不停申请租约,直到申请成功。 申请成功后,每 租期/2 续约。 如果目前被其他租户获取租约,只有在对方租约失效,后才允许新的租户获取租约
-     *
-     * @param name     租约名称,无特殊要求,相同名称会竞争租约
-     * @param callback 当第一获取租约时,回调此函数
-     */
-    void startLeaseTask(final String name, ILeaseGetCallback callback);
-
-    /**
-     * 申请租约,会启动一个线程,不停申请租约,直到申请成功。 申请成功后,每 租期/2 续约。 如果目前被其他租户获取租约,只有在对方租约失效,后才允许新的租户获取租约
-     *
-     * @param name            租约名称,无特殊要求,相同名称会竞争租约
-     * @param leaseTermSecond 租期,在租期内可以做业务处理,单位是秒
-     * @param callback        当第一获取租约时,回调此函数
-     */
-    void startLeaseTask(final String name, int leaseTermSecond, ILeaseGetCallback callback);
-
-    /**
-     * 申请锁,无论成功与否,立刻返回。如果不释放,最大锁定时间是5分钟
-     *
-     * @param name       业务名称
-     * @param lockerName 锁名称
-     * @return 是否枷锁成功
-     */
-    boolean lock(String name, String lockerName);
-
-    /**
-     * 申请锁,无论成功与否,立刻返回。默认锁定时间是5分钟
-     *
-     * @param name           业务名称
-     * @param lockerName     锁名称
-     * @param lockTimeSecond 如果不释放,锁定的最大时间,单位是秒
-     * @return 是否枷锁成功
-     * @return
-     */
-    boolean lock(String name, String lockerName, int lockTimeSecond);
-
-    /**
-     * 申请锁,如果没有则等待,等待时间可以指定,如果是-1 则无限等待。如果不释放,最大锁定时间是5分钟
-     *
-     * @param name       业务名称
-     * @param lockerName 锁名称
-     * @param waitTime   没获取锁时,最大等待多长时间,如果是-1 则无限等待
-     * @return 是否枷锁成功
-     */
-    boolean tryLocker(String name, String lockerName, long waitTime);
-
-    /**
-     * 申请锁,如果没有则等待,等待时间可以指定,如果是-1 则无限等待。如果不释放,最大锁定时间是lockTimeSecond
-     *
-     * @param name           业务名称
-     * @param lockerName     锁名称
-     * @param waitTime       没获取锁时,最大等待多长时间,如果是-1 则无限等待
-     * @param lockTimeSecond 如果不释放,锁定的最大时间,单位是秒
-     * @return 是否枷锁成功
-     */
-    boolean tryLocker(String name, String lockerName, long waitTime, int lockTimeSecond);
-
-    /**
-     * 释放锁
-     *
-     * @param name
-     * @param lockerName
-     * @return
-     */
-    boolean unlock(String name, String lockerName);
-
-    /**
-     * 对于已经获取锁的,可以通过这个方法,一直持有锁。 和租约的区别是,当释放锁后,无其他实例抢占。无法实现主备模式
-     *
-     * @param name           业务名称
-     * @param lockerName     锁名称
-     * @param lockTimeSecond 租期,这个方法会自动续约,如果不主动释放,会一直持有锁
-     * @return 是否成功获取锁
-     */
-    boolean holdLock(String name, String lockerName, int lockTimeSecond);
-
-    /**
-     * 是否持有锁,不会申请锁。如果以前申请过,且未过期,返回true,否则返回false
-     *
-     * @param name       业务名称
-     * @param lockerName 锁名称
-     * @return
-     */
-    boolean hasHoldLock(String name, String lockerName);
-
-    List<LeaseInfo> queryLockedInstanceByNamePrefix(String name, String lockerNamePrefix);
-
-}
diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseStorage.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseStorage.java
deleted file mode 100644
index a85d3a6..0000000
--- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseStorage.java
+++ /dev/null
@@ -1,71 +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.lease.service;
-
-import java.util.List;
-import org.apache.rocketmq.streams.lease.model.LeaseInfo;
-
-public interface ILeaseStorage {
-
-    /**
-     * 更新lease info,需要是原子操作,存储保障多线程操作的原子性
-     *
-     * @param leaseInfo 租约表数据
-     * @return
-     */
-    boolean updateLeaseInfo(LeaseInfo leaseInfo);
-
-    /**
-     * 统计这个租约名称下,LeaseInfo对象个数
-     *
-     * @param leaseName 租约名称,无特殊要求,相同名称会竞争租约
-     * @return
-     */
-    Integer countLeaseInfo(String leaseName);
-
-    /**
-     * 查询无效的的租约
-     *
-     * @param leaseName 租约名称,无特殊要求,相同名称会竞争租约
-     * @return
-     */
-    LeaseInfo queryInValidateLease(String leaseName);
-
-    /**
-     * 查询有效的的租约
-     *
-     * @param leaseName 租约名称,无特殊要求,相同名称会竞争租约
-     * @return
-     */
-    LeaseInfo queryValidateLease(String leaseName);
-
-    /**
-     * 按前缀查询有效的租约信息
-     *
-     * @param namePrefix
-     * @return
-     */
-    List<LeaseInfo> queryValidateLeaseByNamePrefix(String namePrefix);
-
-    /**
-     * 增加租约
-     *
-     * @param leaseInfo 租约名称,无特殊要求,相同名称会竞争租约
-     */
-    void addLeaseInfo(LeaseInfo leaseInfo);
-
-}
diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseStorasge.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseStorasge.java
deleted file mode 100644
index cbfe26e..0000000
--- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/ILeaseStorasge.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.lease.service;
-
-import org.apache.rocketmq.streams.lease.model.LeaseInfo;
-
-public interface ILeaseStorasge {
-
-
-
-    /**
-     * 更新lease info,需要是原子操作,存储保障多线程操作的原子性
-     * @param leaseInfo 租约表数据
-     * @return
-     */
-    boolean updateLeaseInfo(LeaseInfo leaseInfo);
-
-    /**
-     * 统计这个租约名称下,LeaseInfo对象个数
-     * @param leaseName 租约名称,无特殊要求,相同名称会竞争租约
-     * @return
-     */
-    Integer countLeaseInfo(String leaseName);
-
-    /**
-     * 查询无效的的租约
-     * @param leaseName 租约名称,无特殊要求,相同名称会竞争租约
-     * @return
-     */
-    LeaseInfo queryInValidateLease(String leaseName);
-
-
-    /**
-     * 查询无效的的租约
-     * @param leaseName 租约名称,无特殊要求,相同名称会竞争租约
-     * @return
-     */
-    LeaseInfo queryValidateLease(String leaseName);
-
-    /**
-     * 增加租约
-     * @param leaseInfo 租约名称,无特殊要求,相同名称会竞争租约
-     */
-    void addLeaseInfo(LeaseInfo leaseInfo);
-
-
-
-}
diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/BasedLesaseImpl.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/BasedLesaseImpl.java
deleted file mode 100644
index 9584a46..0000000
--- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/BasedLesaseImpl.java
+++ /dev/null
@@ -1,402 +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.lease.service.impl;
-
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.utils.DateUtil;
-import org.apache.rocketmq.streams.common.utils.IPUtil;
-import org.apache.rocketmq.streams.common.utils.RuntimeUtil;
-import org.apache.rocketmq.streams.lease.model.LeaseInfo;
-import org.apache.rocketmq.streams.lease.service.ILeaseGetCallback;
-import org.apache.rocketmq.streams.lease.service.ILeaseService;
-import org.apache.rocketmq.streams.lease.service.ILeaseStorage;
-
-public abstract class BasedLesaseImpl implements ILeaseService {
-    private static final Log LOG = LogFactory.getLog(BasedLesaseImpl.class);
-
-    private static final String CONSISTENT_HASH_PREFIX = "consistent_hash_";
-    private static final AtomicBoolean syncStart = new AtomicBoolean(false);
-    private static final int synTime = 120;  // 5分钟的一致性hash同步时间太久了,改为2分钟
-    protected ScheduledExecutorService taskExecutor = null;
-    protected int leaseTerm = 300 * 2;                                  // 租约时间
-
-    // protected transient JDBCDriver jdbcDataSource = null;
-    protected ILeaseStorage leaseStorage;
-    protected volatile Map<String, Date> leaseName2Date = new ConcurrentHashMap<>();    // 每个lease name对应的租约到期时间
-
-    public BasedLesaseImpl() {
-
-        taskExecutor = new ScheduledThreadPoolExecutor(10);
-    }
-
-    /**
-     * lease_name: consistent_hash_ip, lease_user_ip: ip,定时刷新lease_info表,检查一致性hash环的节点情况
-     *
-     * @param name
-     * @return
-     */
-    @Override
-    public boolean hasLease(String name) {
-        // 内存中没有租约信息则表示 没有租约
-        Date leaseEndTime = leaseName2Date.get(name);
-        if (leaseEndTime == null) {
-            // LOG.info("内存中根据 " + name + "没有查询到租约信息,表示没有租约");
-            return false;
-        }
-        // LOG.info("查询是否有租约 name:" + name + " ,当前时间:" + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(new Date())
-        // + " 租约到期时间 " + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(leaseEndTime));
-        // 有租约时间,并且租约时间大于当前时间,表示有租约信息
-        if (new Date().before(leaseEndTime)) {
-            return true;
-        }
-
-        return false;
-    }
-
-    private final Map<String, AtomicBoolean> startLeaseMap = new HashMap<>();
-
-    @Override
-    public void startLeaseTask(final String name) {
-        startLeaseTask(name, this.leaseTerm, null);
-    }
-
-    @Override
-    public void startLeaseTask(final String name, ILeaseGetCallback callback) {
-        startLeaseTask(name, this.leaseTerm, callback);
-    }
-
-    @Override
-    public void startLeaseTask(final String name, int leaseTerm, ILeaseGetCallback callback) {
-        ApplyTask applyTask = new ApplyTask(leaseTerm, name, callback);
-        startLeaseTask(name, applyTask, leaseTerm / 2, true);
-    }
-
-    /**
-     * 启动定时器,定时执行任务,确保任务可重入
-     *
-     * @param name
-     * @param runnable     具体任务
-     * @param scheduleTime 调度时间
-     * @param startNow     是否立刻启动一次
-     */
-    protected void startLeaseTask(final String name, Runnable runnable, int scheduleTime, boolean startNow) {
-        AtomicBoolean isStartLease = startLeaseMap.get(name);//多次调用,只启动一次定时任务
-        if (isStartLease == null) {
-            synchronized (this) {
-                isStartLease = startLeaseMap.get(name);
-                if (isStartLease == null) {
-                    isStartLease = new AtomicBoolean(false);
-                    startLeaseMap.put(name, isStartLease);
-                }
-            }
-        }
-        if (isStartLease.compareAndSet(false, true)) {
-            if (startNow) {
-                runnable.run();
-            }
-            taskExecutor.scheduleWithFixedDelay(runnable, 0, scheduleTime, TimeUnit.SECONDS);
-        }
-    }
-
-    /**
-     * 续约任务
-     */
-    protected class ApplyTask implements Runnable {
-
-        protected String name;
-        protected int leaseTerm;
-        protected ILeaseGetCallback callback;
-
-        public ApplyTask(int leaseTerm, String name) {
-            this(leaseTerm, name, null);
-        }
-
-        public ApplyTask(int leaseTerm, String name, ILeaseGetCallback callback) {
-            this.name = name;
-            this.leaseTerm = leaseTerm;
-            this.callback = callback;
-        }
-
-        @Override
-        public void run() {
-            try {
-                // LOG.info("LeaseServiceImpl name: " + name + "开始获取租约...");
-                AtomicBoolean newApplyLease = new AtomicBoolean(false);
-                Date leaseDate = applyLeaseTask(leaseTerm, name, newApplyLease);
-                if (leaseDate != null) {
-                    leaseName2Date.put(name, leaseDate);
-                    LOG.info("LeaseServiceImpl, name: " + name + " " + getSelfUser() + " 获取租约成功, 租约到期时间为 "
-                        + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(leaseDate));
-                } else {
-                    // fix.2020.08.13 这时name对应的租约可能还在有效期内,或者本机还持有租约,需要remove
-                    //  leaseName2Date.remove(name);
-                    LOG.info("LeaseServiceImpl name: " + name + " " + getSelfUser() + " 获取租约失败 ");
-                }
-                if (newApplyLease.get() && callback != null) {
-                    callback.callback(leaseDate);
-                }
-            } catch (Exception e) {
-                LOG.error(" LeaseServiceImpl name: " + name + "  " + getSelfUser() + " 获取租约出现异常 ", e);
-            }
-
-        }
-    }
-
-    /**
-     * 申请租约,如果当期租约有效,直接更新一个租约周期,如果当前租约无效,先查询是否有有效的租约,如果有申请失败,否则直接申请租约
-     */
-    protected Date applyLeaseTask(int leaseTerm, String name, AtomicBoolean newApplyLease) {
-
-        // 计算下一次租约时间 = 当前时间 + 租约时长
-        Date nextLeaseDate = DateUtil.addSecond(new Date(), leaseTerm);
-
-        // 1 如果已经有租约,则更新租约时间(内存和数据库)即可
-        if (hasLease(name)) {
-            // LOG.info("用户已有租约,更新数据库和内存中的租约信息");
-            // 更新数据库
-            LeaseInfo leaseInfo = queryValidateLease(name);
-            if (leaseInfo == null) {
-                LOG.error("LeaseServiceImpl applyLeaseTask leaseInfo is null");
-                return null;
-            }
-            // fix.2020.08.13,与本机ip相等且满足一致性hash分配策略,才续约,其他情况为null
-            String leaseUserIp = leaseInfo.getLeaseUserIp();
-            if (!leaseUserIp.equals(getSelfUser())) {
-                return null;
-            }
-            leaseInfo.setLeaseEndDate(nextLeaseDate);
-            updateLeaseInfo(leaseInfo);
-            return nextLeaseDate;
-        }
-
-        // 2 没有租约情况 判断是否可以获取租约,只要租约没有被其他人获取,则说明有有效租约
-        boolean success = canGetLease(name);
-        if (!success) { // 表示被其他机器获取到了有效的租约
-            // LOG.info("其他机器获取到了有效的租约");
-            return null;
-        }
-
-        // 3 没有租约而且可以获取租约的情况,则尝试使用数据库原子更新的方式获取租约,保证只有一台机器成功获取租约,而且可以运行
-        boolean flag = tryGetLease(name, nextLeaseDate);
-        if (flag) { // 获取租约成功
-            newApplyLease.set(true);
-            return nextLeaseDate;
-        }
-        return null;
-
-    }
-
-    /**
-     * 查询数据库,自己是否在租期内或没有被其他人租用
-     *
-     * @return
-     */
-    protected boolean canGetLease(String name) {
-        LeaseInfo leaseInfo = queryValidateLease(name);
-        if (leaseInfo == null) {
-            return true;
-        }
-        // fix.2020.08.13,租约ip为本机ip,且与一致性hash分配ip一致,才是有效租约
-        String leaseUserIp = leaseInfo.getLeaseUserIp();
-        if (leaseUserIp.equals(getSelfUser())) {
-            return true;
-        }
-        return false;
-    }
-
-    /**
-     * 更新数据库,占用租期并更新租期时间
-     *
-     * @param time
-     */
-    protected boolean tryGetLease(String name, Date time) {
-        // LOG.info("尝试获取租约 lease name is : " + name + " 下次到期时间: "
-        // + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(time));
-        LeaseInfo validateLeaseInfo = queryValidateLease(name);
-
-        if (validateLeaseInfo == null) {// 这里有两种情况 1 数据库里面没有租约信息 2 数据库里面有租约信息但是已经过期
-            Integer count = countLeaseInfo(name);
-            if (count == null || count == 0) {// 表示现在数据库里面没有任何租约信息,插入租约成功则表示获取成功,失败表示在这一时刻其他机器获取了租约
-                // LOG.info("数据库中暂时没有租约信息,尝试原子插入租约:" + name);
-                // fix.2020.08.13,经过一致性hash计算,该名字的任务不应该在本机执行,直接返回,无需插入。只有分配到hash执行权限的机器才可以插入并获取租约
-                if (!getSelfUser().equals(getConsistentHashHost(name))) {
-                    return false;
-                }
-                validateLeaseInfo = new LeaseInfo();
-                validateLeaseInfo.setLeaseName(name);
-                validateLeaseInfo.setLeaseUserIp(getSelfUser());
-                validateLeaseInfo.setLeaseEndDate(time);
-                validateLeaseInfo.setStatus(1);
-                validateLeaseInfo.setVersion(1);
-                if (insert(validateLeaseInfo)) {
-                    LOG.info("数据库中暂时没有租约信息,原子插入成功,获取租约成功:" + name);
-                    return true;
-                } else {
-                    LOG.info("数据库中暂时没有租约信息,原子插入失败,已经被其他机器获取租约:" + name);
-                    return false;
-                }
-            } else { // 表示数据库里面有一条但是无效,这里需要两台机器按照version进行原子更新,更新成功的获取租约
-                // LOG.info("数据库中有一条无效的租约信息,尝试根据版本号去原子更新租约信息:" + name);
-                LeaseInfo inValidateLeaseInfo = queryInValidateLease(name);
-                if (inValidateLeaseInfo == null) {// 说明这个时候另外一台机器获取成功了
-                    LOG.info("另外一台机器获取成功了租约:" + name);
-                    return false;
-                }
-                // fix.2020.08.13,机器重启之后,该名字的任务已经不分配在此机器上执行,直接返回,无需更新数据库
-                if (!getSelfUser().equals(getConsistentHashHost(name))) {
-                    return false;
-                }
-                inValidateLeaseInfo.setLeaseName(name);
-                inValidateLeaseInfo.setLeaseUserIp(getSelfUser());
-                inValidateLeaseInfo.setLeaseEndDate(time);
-                inValidateLeaseInfo.setStatus(1);
-                boolean success = updateDBLeaseInfo(inValidateLeaseInfo);
-                if (success) {
-                    LOG.info("LeaseServiceImpl 原子更新租约成功,当前机器获取到了租约信息:" + name);
-                } else {
-                    LOG.info("LeaseServiceImpl 原子更新租约失败,租约被其他机器获取:" + name);
-                }
-                return success;
-            }
-
-        } else { // 判断是否是自己获取了租约,如果是自己获取了租约则更新时间(内存和数据库),
-            // 这里是为了解决机器重启的情况,机器重启,内存中没有租约信息,但是实际上该用户是有租约权限的
-            // fix.2020.08.13,租约的ip与本机ip相等,且满足一致性hash策略,才会被本机执行
-            String leaseUserIp = validateLeaseInfo.getLeaseUserIp();
-            if (leaseUserIp.equals(getSelfUser())) {
-                // 如果当期用户有租约信息,则更新数据库
-                validateLeaseInfo.setLeaseEndDate(time);
-                boolean hasUpdate = updateLeaseInfo(validateLeaseInfo);
-                if (hasUpdate) {
-                    LOG.info(
-                        "LeaseServiceImpl机器重启情况,当前用户有租约信息,并且更新数据库成功,租约信息为 name :" + validateLeaseInfo.getLeaseName()
-                            + " ip : " + validateLeaseInfo.getLeaseUserIp() + " 到期时间 : " + new SimpleDateFormat(
-                            "yyyy-MM-dd HH:mm:ss").format(validateLeaseInfo.getLeaseEndDate()));
-                    return true;
-                } else {
-                    LOG.info("LeaseServiceImpl 机器重启情况,当前用户有租约信息,并且更新数据库失败,表示失去租约:" + name);
-                    return false;
-                }
-            }
-            // LOG.info("LeaseServiceImpl 租约被其他机器获取,租约信息为 name :" + validateLeaseInfo.getLeaseName() + " ip : "
-            // + validateLeaseInfo.getLeaseUserIp() + " 到期时间 : "
-            // + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(validateLeaseInfo.getLeaseEndDate()));
-            return false;
-        }
-
-    }
-
-    protected LeaseInfo queryValidateLease(String name) {
-        //String sql = "SELECT * FROM lease_info WHERE lease_name ='" + name + "' and status=1 and lease_end_time>now()";
-        //// LOG.info("LeaseServiceImpl query validate lease sql:" + sql);
-        //return queryLease(name, sql);
-        return leaseStorage.queryValidateLease(name);
-    }
-
-    protected List<LeaseInfo> queryValidateLeaseByNamePrefix(String namePrefix) {
-        return leaseStorage.queryValidateLeaseByNamePrefix(namePrefix);
-    }
-
-    /**
-     * 如果发生唯一索引冲突返回失败
-     *
-     * @param leaseInfo
-     * @return
-     */
-    private boolean insert(LeaseInfo leaseInfo) {
-        try {
-            addLeaseInfo(leaseInfo);
-            return true;
-        } catch (Exception e) {
-            LOG.error("LeaseServiceImpl insert error", e);
-            return false;
-        }
-    }
-
-    /**
-     * 更新时需要加version=当前version,如果更新数据条数为0,返回false
-     *
-     * @param leaseInfo
-     * @return
-     */
-    protected boolean updateDBLeaseInfo(LeaseInfo leaseInfo) {
-        return updateLeaseInfo(leaseInfo);
-    }
-
-    protected boolean updateLeaseInfo(LeaseInfo leaseInfo) {
-
-        return leaseStorage.updateLeaseInfo(leaseInfo);
-    }
-
-    protected Integer countLeaseInfo(String name) {
-
-        return leaseStorage.countLeaseInfo(name);
-    }
-
-    protected LeaseInfo queryInValidateLease(String name) {
-
-        return leaseStorage.queryInValidateLease(name);
-    }
-
-    protected void addLeaseInfo(LeaseInfo leaseInfo) {
-
-        leaseStorage.addLeaseInfo(leaseInfo);
-
-    }
-
-    /**
-     * 本地ip地址作为自己的唯一标识
-     *
-     * @return
-     */
-    public static String getLocalName() {
-        return IPUtil.getLocalIdentification() + ":" + Optional.ofNullable(RuntimeUtil.getPid()).orElse("UNKNOWN");
-    }
-
-    /**
-     * 本地ip地址作为自己的唯一标识
-     *
-     * @return
-     */
-    public String getSelfUser() {
-        return getLocalName();
-    }
-
-    private String getConsistentHashHost(String name) {
-        //if (StringUtil.isEmpty(leaseConsistentHashSuffix)) {
-        //    return getSelfUser();
-        //}
-        //return consistentHashInstance.getCandidateNode(name);
-        return getSelfUser();
-    }
-
-    public void setLeaseStorage(ILeaseStorage leaseStorage) {
-        this.leaseStorage = leaseStorage;
-    }
-}
diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/LeaseServiceImpl.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/LeaseServiceImpl.java
deleted file mode 100644
index f10ad0a..0000000
--- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/LeaseServiceImpl.java
+++ /dev/null
@@ -1,279 +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.lease.service.impl;
-
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.utils.DateUtil;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.lease.model.LeaseInfo;
-import org.apache.rocketmq.streams.lease.service.ILeaseService;
-
-public class LeaseServiceImpl extends BasedLesaseImpl {
-
-    private static final Log LOG = LogFactory.getLog(LeaseServiceImpl.class);
-
-    private transient ConcurrentHashMap<String, HoldLockTask> holdLockTasks = new ConcurrentHashMap();
-
-    protected ConcurrentHashMap<String, HoldLockFunture> seizeLockingFuntures = new ConcurrentHashMap<>();
-    //如果是抢占锁状态中,则不允许申请锁
-
-    public LeaseServiceImpl() {
-        super();
-    }
-
-    /**
-     * 尝试获取锁,可以等待waitTime,如果到点未返回,则直接返回。如果是-1,则一直等待
-     *
-     * @param name       业务名称
-     * @param lockerName 锁名称
-     * @param waitTime   等待时间,是微秒单位
-     * @return
-     */
-    @Override
-    public boolean tryLocker(String name, String lockerName, long waitTime) {
-        return tryLocker(name, lockerName, waitTime, ILeaseService.DEFALUT_LOCK_TIME);
-    }
-
-    @Override
-    public boolean tryLocker(String name, String lockerName, long waitTime, int lockTimeSecond) {
-        long now = System.currentTimeMillis();
-        boolean success = lock(name, lockerName, lockTimeSecond);
-        while (!success) {
-            if (waitTime > -1 && (System.currentTimeMillis() - now > waitTime)) {
-                break;
-            }
-            success = lock(name, lockerName, lockTimeSecond);
-            if (success) {
-                return success;
-            }
-            try {
-                Thread.sleep(100);
-            } catch (InterruptedException e) {
-                LOG.error("LeaseServiceImpl try locker error", e);
-            }
-        }
-        return success;
-
-    }
-
-    @Override
-    public boolean lock(String name, String lockerName) {
-        return lock(name, lockerName, ILeaseService.DEFALUT_LOCK_TIME);
-    }
-
-    @Override
-    public boolean lock(String name, String lockerName, int leaseSecond) {
-        lockerName = createLockName(name, lockerName);
-        Future future = seizeLockingFuntures.get(lockerName);
-        if (future != null && ((HoldLockFunture)future).isDone == false) {
-            return false;
-        }
-        Date nextLeaseDate =
-            DateUtil.addSecond(new Date(), leaseSecond);// 默认锁定5分钟,用完需要立刻释放.如果时间不同步,可能导致锁失败
-        return tryGetLease(lockerName, nextLeaseDate);
-    }
-
-    @Override
-    public boolean unlock(String name, String lockerName) {
-        // LOG.info("LeaseServiceImpl unlock,name:" + name);
-        lockerName = createLockName(name, lockerName);
-        LeaseInfo validateLeaseInfo = queryValidateLease(lockerName);
-        if (validateLeaseInfo == null) {
-            LOG.warn("LeaseServiceImpl unlock,validateLeaseInfo is null,lockerName:" + lockerName);
-        }
-        if (validateLeaseInfo != null && validateLeaseInfo.getLeaseUserIp().equals(getSelfUser())) {
-            validateLeaseInfo.setStatus(0);
-            updateDBLeaseInfo(validateLeaseInfo);
-        }
-        HoldLockTask holdLockTask = holdLockTasks.remove(lockerName);
-        if (holdLockTask != null) {
-            holdLockTask.close();
-        }
-        leaseName2Date.remove(lockerName);
-        return false;
-    }
-
-    /**
-     * 如果有锁,则一直持有,如果不能获取,则结束。和租约不同,租约是没有也会尝试重试,一备对方挂机,自己可以接手工作
-     *
-     * @param name
-     * @param secondeName
-     * @param lockTimeSecond 获取锁的时间
-     * @return
-     */
-    @Override
-    public boolean holdLock(String name, String secondeName, int lockTimeSecond) {
-        if (hasHoldLock(name, secondeName)) {
-            return true;
-        }
-        synchronized (this) {
-            if (hasHoldLock(name, secondeName)) {
-                return true;
-            }
-            String lockerName = createLockName(name, secondeName);
-            Date nextLeaseDate =
-                DateUtil.addSecond(new Date(), lockTimeSecond);
-            boolean success = tryGetLease(lockerName, nextLeaseDate);// 申请锁,锁的时间是leaseTerm
-            if (!success) {
-                return false;
-            }
-            leaseName2Date.put(lockerName, nextLeaseDate);
-
-            if (!holdLockTasks.containsKey(lockerName)) {
-                HoldLockTask holdLockTask = new HoldLockTask(lockTimeSecond, lockerName, this);
-                holdLockTask.start();
-                holdLockTasks.putIfAbsent(lockerName, holdLockTask);
-            }
-        }
-
-        return true;
-    }
-
-    /**
-     * 是否持有锁,不访问数据库,直接看本地
-     *
-     * @param name
-     * @param secondeName
-     * @return
-     */
-    @Override
-    public boolean hasHoldLock(String name, String secondeName) {
-        String lockerName = createLockName(name, secondeName);
-        return hasLease(lockerName);
-    }
-
-    @Override
-    public List<LeaseInfo> queryLockedInstanceByNamePrefix(String name, String lockerNamePrefix) {
-        String leaseNamePrefix = MapKeyUtil.createKey(name, lockerNamePrefix);
-        return queryValidateLeaseByNamePrefix(leaseNamePrefix);
-    }
-
-    private String createLockName(String name, String lockerName) {
-        return MapKeyUtil.createKey(name, lockerName);
-    }
-
-    private class HoldLockTask extends ApplyTask {
-        protected volatile boolean isContinue = true;
-        protected LeaseServiceImpl leaseService;
-        protected ScheduledExecutorService scheduledExecutor;
-
-        public HoldLockTask(int leaseTerm, String name, LeaseServiceImpl leaseService) {
-            super(leaseTerm, name);
-            this.leaseService = leaseService;
-            scheduledExecutor = new ScheduledThreadPoolExecutor(1);
-
-        }
-
-        public void start() {
-            scheduledExecutor.scheduleWithFixedDelay(this, leaseTerm / 2, leaseTerm / 2, TimeUnit.SECONDS);
-        }
-
-        public void close() {
-            isContinue = false;
-            if (scheduledExecutor != null) {
-                scheduledExecutor.shutdown();
-            }
-        }
-
-        public boolean isContinue() {
-            return isContinue;
-        }
-
-        @Override
-        public void run() {
-            try {
-                if (!isContinue) {
-                    return;
-                }
-                Date leaseDate = applyLeaseTask(leaseTerm, name, new AtomicBoolean(false));
-                if (leaseDate != null) {
-                    leaseName2Date.put(name, leaseDate);
-                    LOG.debug("LeaseServiceImpl, name: " + name + " " + getSelfUser() + " 续约锁成功, 租约到期时间为 "
-                        + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(leaseDate));
-                } else {
-                    isContinue = false;
-                    synchronized (leaseService) {
-                        holdLockTasks.remove(name);
-                    }
-                    LOG.info("LeaseServiceImpl name: " + name + " " + getSelfUser() + " 续约锁失败,续锁程序会停止");
-                }
-            } catch (Exception e) {
-                isContinue = false;
-                LOG.error(" LeaseServiceImpl name: " + name + "  " + getSelfUser() + " 续约锁出现异常,续锁程序会停止", e);
-            }
-
-        }
-
-    }
-
-    /**
-     * 抢占锁的future,必须等锁超时才能继续获取锁
-     */
-    protected class HoldLockFunture implements Future<Boolean> {
-        private volatile boolean isDone = false;
-        private volatile Date date = null;
-
-        @Override
-        public boolean cancel(boolean mayInterruptIfRunning) {
-            throw new RuntimeException("can not cancel");
-        }
-
-        @Override
-        public boolean isCancelled() {
-            return false;
-        }
-
-        @Override
-        public boolean isDone() {
-            if (date != null && System.currentTimeMillis() - date.getTime() >= 0) {
-                isDone = true;
-                return isDone;
-            }
-            return false;
-        }
-
-        @Override
-        public Boolean get() throws InterruptedException, ExecutionException {
-            while (!isDone()) {
-                Thread.sleep(1000);
-            }
-            return true;
-        }
-
-        private long startTime = System.currentTimeMillis();
-
-        @Override
-        public Boolean get(long timeout, TimeUnit unit)
-            throws InterruptedException, ExecutionException, TimeoutException {
-
-            throw new RuntimeException("can not support timeout ");
-        }
-
-    }
-}
diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/MockLeaseImpl.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/MockLeaseImpl.java
deleted file mode 100644
index 7354956..0000000
--- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/impl/MockLeaseImpl.java
+++ /dev/null
@@ -1,90 +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.lease.service.impl;
-
-import java.util.Date;
-import java.util.List;
-import org.apache.rocketmq.streams.common.utils.DateUtil;
-import org.apache.rocketmq.streams.lease.model.LeaseInfo;
-import org.apache.rocketmq.streams.lease.service.ILeaseGetCallback;
-import org.apache.rocketmq.streams.lease.service.ILeaseService;
-
-/**
- * 在内存和文件模式下使用,所有的申请都会返回true,主要用来做业务测试
- */
-public class MockLeaseImpl implements ILeaseService {
-    @Override
-    public boolean hasLease(String name) {
-        return true;
-    }
-
-    @Override
-    public void startLeaseTask(String name) {
-
-    }
-
-    @Override
-    public void startLeaseTask(String name, ILeaseGetCallback callback) {
-        callback.callback(DateUtil.addMinute(new Date(), 1));
-    }
-
-    @Override
-    public void startLeaseTask(String name, int leaseTerm, ILeaseGetCallback callback) {
-
-    }
-
-    @Override
-    public boolean lock(String name, String lockerName) {
-        return true;
-    }
-
-    @Override
-    public boolean lock(String name, String lockerName, int lockTimeSecond) {
-        return true;
-    }
-
-    @Override
-    public boolean tryLocker(String name, String lockerName, long waitTime) {
-        return true;
-    }
-
-    @Override
-    public boolean tryLocker(String name, String lockerName, long waitTime, int lockTimeSecond) {
-        return true;
-    }
-
-    @Override
-    public boolean unlock(String name, String lockerName) {
-        return true;
-    }
-
-    @Override
-    public boolean holdLock(String name, String lockerName, int lockTimeSecond) {
-        return true;
-    }
-
-    @Override
-    public boolean hasHoldLock(String name, String lockerName) {
-        return true;
-    }
-
-    @Override
-    public List<LeaseInfo> queryLockedInstanceByNamePrefix(String name, String lockerNamePrefix) {
-        return null;
-    }
-
-}
diff --git a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/storages/DBLeaseStorage.java b/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/storages/DBLeaseStorage.java
deleted file mode 100644
index d3d16b9..0000000
--- a/rocketmq-streams-lease/src/main/java/org/apache/rocketmq/streams/lease/service/storages/DBLeaseStorage.java
+++ /dev/null
@@ -1,228 +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.lease.service.storages;
-
-import java.math.BigInteger;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.utils.DateUtil;
-import org.apache.rocketmq.streams.common.utils.SQLUtil;
-import org.apache.rocketmq.streams.common.utils.StringUtil;
-import org.apache.rocketmq.streams.db.driver.DriverBuilder;
-import org.apache.rocketmq.streams.db.driver.JDBCDriver;
-import org.apache.rocketmq.streams.lease.model.LeaseInfo;
-import org.apache.rocketmq.streams.lease.service.ILeaseStorage;
-
-public class DBLeaseStorage implements ILeaseStorage {
-    private static final Log LOG = LogFactory.getLog(DBLeaseStorage.class);
-    protected JDBCDriver jdbcDataSource;
-    private String url;
-    protected String userName;
-    protected String password;
-    protected String jdbc;
-
-    public DBLeaseStorage(String jdbc, String url, String userName, String password) {
-        this.jdbc = jdbc;
-        this.url = url;
-        this.userName = userName;
-        this.password = password;
-        jdbcDataSource = DriverBuilder.createDriver(jdbc, url, userName, password);
-    }
-
-    @Override
-    public boolean updateLeaseInfo(LeaseInfo leaseInfo) {
-        String sql = "UPDATE lease_info SET version=version+1,status=#{status},gmt_modified=now()";
-        String whereSQL = " WHERE id=#{id} and version=#{version}";
-
-        if (StringUtil.isNotEmpty(leaseInfo.getLeaseName())) {
-            sql += ",lease_name=#{leaseName}";
-        }
-        if (StringUtil.isNotEmpty(leaseInfo.getLeaseUserIp())) {
-            sql += ",lease_user_ip=#{leaseUserIp}";
-        }
-        if (leaseInfo.getLeaseEndDate() != null) {
-            sql += ",lease_end_time=#{leaseEndDate}";
-        }
-        sql += whereSQL;
-        sql = SQLUtil.parseIbatisSQL(leaseInfo, sql);
-        try {
-            int count = getOrCreateJDBCDataSource().update(sql);
-            boolean success = count > 0;
-            if (success) {
-                synchronized (this) {
-                    leaseInfo.setVersion(leaseInfo.getVersion() + 1);
-                }
-            } else {
-                System.out.println(count);
-            }
-            return success;
-        } catch (Exception e) {
-            LOG.error("LeaseServiceImpl updateLeaseInfo excuteUpdate error", e);
-            throw new RuntimeException("execute sql error " + sql, e);
-        }
-    }
-
-    @Override
-    public Integer countLeaseInfo(String leaseName) {
-        String sql = "SELECT count(*) as c FROM lease_info  WHERE lease_name = '" + leaseName + "' and status = 1";
-        try {
-
-            List<Map<String, Object>> rows = getOrCreateJDBCDataSource().queryForList(sql);
-            if (rows == null || rows.size() == 0) {
-                return null;
-            }
-            Long value = (Long) rows.get(0).get("c");
-            return value.intValue();
-        } catch (Exception e) {
-            throw new RuntimeException("execute sql error " + sql, e);
-        }
-    }
-
-    @Override
-    public LeaseInfo queryInValidateLease(String leaseName) {
-        String sql = "SELECT * FROM lease_info WHERE lease_name ='" + leaseName + "' and status=1 and lease_end_time<'" + DateUtil.getCurrentTimeString() + "'";
-        LOG.info("LeaseServiceImpl queryInValidateLease builder:" + sql);
-        return queryLease(leaseName, sql);
-    }
-
-    @Override
-    public LeaseInfo queryValidateLease(String leaseName) {
-        String sql = "SELECT * FROM lease_info WHERE lease_name ='" + leaseName + "' and status=1 and lease_end_time>now()";
-        return queryLease(leaseName, sql);
-    }
-
-    @Override
-    public List<LeaseInfo> queryValidateLeaseByNamePrefix(String namePrefix) {
-        String sql = "SELECT * FROM lease_info WHERE lease_name like '" + namePrefix + "%' and status=1 and lease_end_time>now()";
-        try {
-            List<LeaseInfo> leaseInfos = new ArrayList<>();
-            List<Map<String, Object>> rows = getOrCreateJDBCDataSource().queryForList(sql);
-            if (rows == null || rows.size() == 0) {
-                return null;
-            }
-            for (Map<String, Object> row : rows) {
-                LeaseInfo leaseInfo = convert(row);
-                leaseInfos.add(leaseInfo);
-            }
-
-            return leaseInfos;
-        } catch (Exception e) {
-            throw new RuntimeException("execute sql error " + sql, e);
-        }
-    }
-
-    @Override
-    public void addLeaseInfo(LeaseInfo leaseInfo) {
-        String sql =
-            " REPLACE INTO lease_info(lease_name,lease_user_ip,lease_end_time,status,version,gmt_create,gmt_modified)"
-                + " VALUES (#{leaseName},#{leaseUserIp},#{leaseEndDate},#{status},#{version},now(),now())";
-        sql = SQLUtil.parseIbatisSQL(leaseInfo, sql);
-        try {
-
-            getOrCreateJDBCDataSource().execute(sql);
-        } catch (Exception e) {
-            LOG.error("LeaseServiceImpl execute sql error,sql:" + sql, e);
-            throw new RuntimeException("execute sql error " + sql, e);
-        }
-    }
-
-    protected JDBCDriver getOrCreateJDBCDataSource() {
-        if (this.jdbcDataSource == null || !this.jdbcDataSource.isValidate()) {
-            synchronized (this) {
-                if (this.jdbcDataSource == null || !this.jdbcDataSource.isValidate()) {
-                    this.jdbcDataSource =
-                        DriverBuilder.createDriver(this.jdbc, this.url, this.userName, this.password);
-                }
-            }
-        }
-        return jdbcDataSource;
-    }
-
-    protected LeaseInfo queryLease(String name, String sql) {
-        try {
-            List<Map<String, Object>> rows = getOrCreateJDBCDataSource().queryForList(sql);
-            if (rows == null || rows.size() == 0) {
-                return null;
-            }
-            return convert(rows.get(0));
-        } catch (Exception e) {
-            throw new RuntimeException("execute sql error " + sql, e);
-        }
-    }
-
-    protected LeaseInfo convert(Map<String, Object> map) {
-        LeaseInfo leaseInfo = new LeaseInfo();
-        leaseInfo.setId(getMapLongValue("id", map));
-        leaseInfo.setCreateTime(getMapDateValue("gmt_create", map));
-        leaseInfo.setLeaseEndDate(getMapDateValue("lease_end_time", map));
-        leaseInfo.setLeaseName(getMapValue("lease_name", map, String.class));
-        leaseInfo.setLeaseUserIp(getMapValue("lease_user_ip", map, String.class));
-        Integer status = getMapValue("status", map, Integer.class);
-        if (status != null) {
-            leaseInfo.setStatus(status);
-        }
-        leaseInfo.setUpdateTime(getMapDateValue("gmt_modified", map));
-        Long version = getMapLongValue("version", map);
-        if (version != null) {
-            leaseInfo.setVersion(version);
-        }
-        return leaseInfo;
-    }
-
-    @SuppressWarnings("unchecked")
-    private <T> T getMapValue(String fieldName, Map<String, Object> map, Class<T> integerClass) {
-        Object value = map.get(fieldName);
-        if (value == null) {
-            return null;
-        }
-        return (T) value;
-    }
-
-    private Long getMapLongValue(String fieldName, Map<String, Object> map) {
-        Object value = map.get(fieldName);
-        if (value == null) {
-            return null;
-        }
-        if (value instanceof Long) {
-            return (Long) value;
-        }
-        if (value instanceof BigInteger) {
-            return ((BigInteger) value).longValue();
-        }
-        return null;
-    }
-
-    private Date getMapDateValue(String fieldName, Map<String, Object> map) {
-        Object value = map.get(fieldName);
-        if (value == null) {
-            return null;
-        }
-        if (value instanceof Date) {
-            return (Date) value;
-        }
-        if (value instanceof String) {
-            return DateUtil.parseTime(((String) value));
-        }
-        return null;
-
-    }
-
-}
diff --git a/rocketmq-streams-lease/src/test/java/org/apache/rocketmq/streams/lease/LeaseComponentTest.java b/rocketmq-streams-lease/src/test/java/org/apache/rocketmq/streams/lease/LeaseComponentTest.java
deleted file mode 100644
index bff40c1..0000000
--- a/rocketmq-streams-lease/src/test/java/org/apache/rocketmq/streams/lease/LeaseComponentTest.java
+++ /dev/null
@@ -1,117 +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.lease;
-
-import java.util.Date;
-import org.apache.rocketmq.streams.common.component.ComponentCreator;
-import org.apache.rocketmq.streams.common.configure.ConfigureFileKey;
-import org.apache.rocketmq.streams.db.driver.DriverBuilder;
-import org.apache.rocketmq.streams.db.driver.JDBCDriver;
-import org.apache.rocketmq.streams.lease.model.LeaseInfo;
-import org.apache.rocketmq.streams.lease.service.ILeaseGetCallback;
-import org.apache.rocketmq.streams.lease.service.ILeaseService;
-import org.junit.Test;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-public class LeaseComponentTest {
-
-    private String URL = "";
-    protected String USER_NAME = "";
-    protected String PASSWORD = "";
-
-    public LeaseComponentTest() {
-
-        //正式使用时,在配置文件配置
-        ComponentCreator.getProperties().put(ConfigureFileKey.CONNECT_TYPE, "DB");
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_URL, URL);//数据库连接url
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_USERNAME, USER_NAME);//用户名
-        ComponentCreator.getProperties().put(ConfigureFileKey.JDBC_PASSWORD, PASSWORD);//password
-
-        JDBCDriver driver = DriverBuilder.createDriver();
-        driver.execute(LeaseInfo.createTableSQL());
-    }
-
-    @Test
-    public void testLease() throws InterruptedException {
-        String leaseName = "lease.test";//自己取个名字,和其他租约不同就行,可以用source 的namespace+configurableName
-        int leaseTime = 5;//租期,时间是秒。如果机器挂了,
-        LeaseComponent.getInstance().getService().startLeaseTask(leaseName, leaseTime, new ILeaseGetCallback() {
-            @Override
-            public void callback(Date nextLeaseDate) {
-                System.out.println("I get lease");
-            }
-        });
-        assertTrue(LeaseComponent.getInstance().getService().hasLease(leaseName));
-        Thread.sleep(5000);
-        assertTrue(LeaseComponent.getInstance().getService().hasLease(leaseName));//会一直续约
-        Thread.sleep(5000);
-        assertTrue(LeaseComponent.getInstance().getService().hasLease(leaseName));//会一直续约
-    }
-
-    @Test
-    public void testLock() throws InterruptedException {
-        String name = "dipper";
-        String lockName = "lease.test";
-        int leaseTime = 5;
-        boolean success = LeaseComponent.getInstance().getService().lock(name, lockName, leaseTime);//锁定5秒钟
-        assertTrue(success);//获取锁
-        Thread.sleep(6000);
-        assertFalse(LeaseComponent.getInstance().getService().hasHoldLock(name, lockName));//超期释放
-    }
-
-    /**
-     * holdlock是一直持有锁,和租约的区别是,当释放锁后,无其他实例抢占
-     *
-     * @throws InterruptedException
-     */
-    @Test
-    public void testHoldLock() throws InterruptedException {
-        String name = "dipper";
-        String lockName = "lease.test";
-        int leaseTime = 6;
-        boolean success = LeaseComponent.getInstance().getService().holdLock(name, lockName, leaseTime);//锁定5秒钟
-        assertTrue(success);//获取锁
-        Thread.sleep(8000);
-        assertTrue(LeaseComponent.getInstance().getService().hasHoldLock(name, lockName));//会自动续约,不会释放,可以手动释放
-        LeaseComponent.getInstance().getService().unlock(name, lockName);
-        assertFalse(LeaseComponent.getInstance().getService().hasHoldLock(name, lockName));
-    }
-
-    @Test
-    public void testHoldLockContinue() throws InterruptedException {
-        String name = "dipper";
-        String lockName = "lease.test";
-        int leaseTime = 6;
-        boolean success = holdLock(name, lockName, leaseTime);//锁定5秒钟
-        while (true) {
-            Thread.sleep(1000);
-            System.out.println(holdLock(name, lockName, leaseTime));
-        }
-    }
-
-    protected boolean holdLock(String name, String lockName, int leaseTime) {
-        ILeaseService leaseService = LeaseComponent.getInstance().getService();
-        if (leaseService.hasHoldLock(name, lockName)) {
-            return true;
-        }
-
-        return leaseService.holdLock(name, lockName, leaseTime);
-    }
-
-}
diff --git a/rocketmq-streams-lease/src/test/resources/log4j.xml b/rocketmq-streams-lease/src/test/resources/log4j.xml
deleted file mode 100755
index fb0edb1..0000000
--- a/rocketmq-streams-lease/src/test/resources/log4j.xml
+++ /dev/null
@@ -1,36 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" ?>
-<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
-<!--
-  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.
-  -->
-<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
-
-    <appender name="Console" class="org.apache.log4j.ConsoleAppender">
-        <layout class="org.apache.log4j.PatternLayout">
-            <param name="ConversionPattern" value="%d{ISO8601} %l [%t] %-5p - %m%n%n"/>
-        </layout>
-        <filter class="org.apache.log4j.varia.LevelRangeFilter">
-            <param name="LevelMin" value="INFO"/>
-            <param name="LevelMax" value="ERROR"/>
-        </filter>
-    </appender>
-
-    <root>
-        <priority value="INFO"/>
-        <appender-ref ref="Console"/>
-    </root>
-
-</log4j:configuration>
\ No newline at end of file
diff --git a/rocketmq-streams-runner/assembly/distribution.xml b/rocketmq-streams-runner/assembly/distribution.xml
deleted file mode 100644
index f44e34f..0000000
--- a/rocketmq-streams-runner/assembly/distribution.xml
+++ /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.
--->
-<assembly>
-    <id>distribution</id>
-    <formats>
-        <format>tar.gz</format>
-    </formats>
-    <fileSets>
-        <fileSet>
-            <fileMode>0775</fileMode>
-            <directory>target/rocketmq-streams-${project.version}-standalone/bin</directory>
-            <outputDirectory>bin</outputDirectory>
-        </fileSet>
-        <fileSet>
-            <directory>target/rocketmq-streams-${project.version}-standalone/conf</directory>
-            <outputDirectory>conf</outputDirectory>
-        </fileSet>
-        <fileSet>
-            <directory>target/rocketmq-streams-${project.version}-standalone/jobs</directory>
-            <outputDirectory>jobs</outputDirectory>
-        </fileSet>
-        <fileSet>
-            <fileMode>0775</fileMode>
-            <directory>target/rocketmq-streams-${project.version}-standalone/lib</directory>
-            <outputDirectory>lib</outputDirectory>
-        </fileSet>
-        <fileSet>
-            <fileMode>0775</fileMode>
-            <directory>target/rocketmq-streams-${project.version}-standalone/log</directory>
-            <outputDirectory>log</outputDirectory>
-        </fileSet>
-    </fileSets>
-    <files>
-        <file>
-            <source>target/rocketmq-streams-${project.version}-standalone/LICENSE</source>
-            <outputDirectory/>
-        </file>
-        <file>
-            <source>target/rocketmq-streams-${project.version}-standalone/NOTICE</source>
-            <outputDirectory/>
-        </file>
-        <file>
-            <source>target/rocketmq-streams-${project.version}-standalone/quick_start.md</source>
-            <outputDirectory/>
-        </file>
-        <file>
-            <source>target/rocketmq-streams-${project.version}-standalone/README.md</source>
-            <outputDirectory/>
-        </file>
-        <file>
-            <source>target/rocketmq-streams-${project.version}-standalone/README-chinese.md</source>
-            <outputDirectory/>
-        </file>
-    </files>
-</assembly>
diff --git a/rocketmq-streams-runner/assembly/standalone.xml b/rocketmq-streams-runner/assembly/standalone.xml
deleted file mode 100644
index 69075a3..0000000
--- a/rocketmq-streams-runner/assembly/standalone.xml
+++ /dev/null
@@ -1,72 +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.
--->
-<assembly>
-    <id>standalone</id>
-    <formats>
-        <format>dir</format>
-    </formats>
-    <includeBaseDirectory>false</includeBaseDirectory>
-
-    <fileSets>
-        <fileSet>
-            <directory>bin</directory>
-            <outputDirectory>/bin</outputDirectory>
-            <fileMode>0755</fileMode>
-        </fileSet>
-        <fileSet>
-            <directory>src/main/resources</directory>
-            <outputDirectory>/conf</outputDirectory>
-            <fileMode>0755</fileMode>
-        </fileSet>
-        <fileSet>
-            <directory>conf</directory>
-            <outputDirectory>/conf</outputDirectory>
-            <fileMode>0755</fileMode>
-        </fileSet>
-        <fileSet>
-            <directory>jobs</directory>
-            <outputDirectory>/jobs</outputDirectory>
-            <fileMode>0755</fileMode>
-        </fileSet>
-        <fileSet>
-            <directory>log</directory>
-            <outputDirectory>/log</outputDirectory>
-            <fileMode>0755</fileMode>
-        </fileSet>
-        <fileSet>
-            <directory>../</directory>
-            <outputDirectory>/</outputDirectory>
-            <includes>
-                <include>LICENSE</include>
-                <include>NOTICE</include>
-                <include>quick_start.md</include>
-                <include>README.md</include>
-                <include>README-chinese.md</include>
-            </includes>
-        </fileSet>
-    </fileSets>
-
-    <dependencySets>
-        <dependencySet>
-            <useProjectArtifact>true</useProjectArtifact>
-            <outputDirectory>lib</outputDirectory>
-            <!-- 只包含runtime作用域的依赖 -->
-            <scope>runtime</scope>
-        </dependencySet>
-    </dependencySets>
-
-</assembly>
diff --git a/rocketmq-streams-runner/bin/start.sh b/rocketmq-streams-runner/bin/start.sh
deleted file mode 100755
index 44797c4..0000000
--- a/rocketmq-streams-runner/bin/start.sh
+++ /dev/null
@@ -1,58 +0,0 @@
-#!/bin/sh
-set -e
-
-PROG_NAME=$0
-MAIN_CLASS=$1
-
-if [ -z "${MAIN_CLASS}" ]; then
-  usage
-fi
-
-usage() {
-    echo "Usage: $PROG_NAME {mainClass or mainClasses splited with comma}"
-    exit 2 # bad usage
-}
-
-
-JVM_CONFIG=$2
-if [ -z "${JVM_CONFIG}" ]; then
-  JVM_CONFIG="-Xms2048m -Xmx2048m -Xss512k"
-fi
-
-ROCKETMQ_STREAMS_HOME=$(cd $(dirname ${BASH_SOURCE[0]})/..; pwd)
-ROCKETMQ_STREAMS_JOBS_DIR=$ROCKETMQ_STREAMS_HOME/jobs
-ROCKETMQ_STREAMS_DEPENDENCIES=$ROCKETMQ_STREAMS_HOME/lib
-ROCKETMQ_STREAMS_LOGS=$ROCKETMQ_STREAMS_HOME/log/catalina.out
-
-if [ -z "${JAVA_HOME:-}" ]; then
-  JAVA="java -server"
-else
-  JAVA="$JAVA_HOME/bin/java -server"
-fi
-
-JAVA_OPTIONS=${JAVA_OPTIONS:-}
-
-JVM_OPTS=()
-if [ ! -z "${JAVA_OPTIONS}" ]; then
-  JVM_OPTS+=("${JAVA_OPTIONS}")
-fi
-if [ ! -z "${JVM_CONFIG}" ]; then
-  JVM_OPTS+=("${JVM_CONFIG}")
-fi
-
-JVM_OPTS+=("-Dlogback.configurationFile=conf/logback.xml")
-
-
-
-# shellcheck disable=SC2039
-# shellcheck disable=SC2206
-array=(${MAIN_CLASS//,/ })
-
-# shellcheck disable=SC2068
-# shellcheck disable=SC2039
-for var in ${array[@]}
-do
-   # shellcheck disable=SC2068
-   # shellcheck disable=SC2039
-   eval exec $JAVA ${JVM_OPTS[@]} -classpath "$ROCKETMQ_STREAMS_JOBS_DIR/*:$ROCKETMQ_STREAMS_DEPENDENCIES/*" $var "&" >>"$ROCKETMQ_STREAMS_LOGS" 2>&1
-done
diff --git a/rocketmq-streams-runner/bin/stop.sh b/rocketmq-streams-runner/bin/stop.sh
deleted file mode 100755
index 5734c92..0000000
--- a/rocketmq-streams-runner/bin/stop.sh
+++ /dev/null
@@ -1,33 +0,0 @@
-#!/bin/sh
-set -e
-PROG_NAME=$0
-MAIN_CLASS=$1
-
-if [ -z "${MAIN_CLASS}" ]; then
-  usage
-fi
-
-# shellcheck disable=SC2039
-# shellcheck disable=SC2206
-array=(${MAIN_CLASS//,/ })
-
-# shellcheck disable=SC2068
-# shellcheck disable=SC2039
-for var in ${array[@]}
-do
-  STREAM_JOB_PIC="$(ps -ef | grep "$var" | grep -v grep | grep -v "$PROG_NAME" | awk '{print $2}' | sed 's/addr://g')"
-  if [ ! -z "$STREAM_JOB_PIC" ]; then
-    echo $STREAM_JOB_PIC
-    echo "Stop rocketmq-streams job"
-    echo "kill -9 $STREAM_JOB_PIC"
-    kill -9 $STREAM_JOB_PIC
-    echo "Job($MAIN_CLASS) shutdown completed."
-  else
-    echo "Job($MAIN_CLASS) not started."
-  fi
-done
-
-
-
-
-
diff --git a/rocketmq-streams-runner/pom.xml b/rocketmq-streams-runner/pom.xml
deleted file mode 100644
index 6dcd83f..0000000
--- a/rocketmq-streams-runner/pom.xml
+++ /dev/null
@@ -1,80 +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-runner</artifactId>
-    <name>ROCKETMQ STREAMS :: runner</name>
-
-    <properties>
-        <maven.compiler.source>8</maven.compiler.source>
-        <maven.compiler.target>8</maven.compiler.target>
-    </properties>
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-clients</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-examples</artifactId>
-        </dependency>
-    </dependencies>
-
-
-    <build>
-        <plugins>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-surefire-plugin</artifactId>
-            </plugin>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-failsafe-plugin</artifactId>
-            </plugin>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-assembly-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <phase>package</phase>
-                        <goals>
-                            <goal>single</goal>
-                        </goals>
-                    </execution>
-                </executions>
-                <configuration>
-                    <attach>true</attach>
-                    <descriptors>
-                        <descriptor>assembly/standalone.xml</descriptor>
-                        <descriptor>assembly/distribution.xml</descriptor>
-                    </descriptors>
-                    <finalName>rocketmq-streams-${project.version}</finalName>
-                    <outputDirectory>target</outputDirectory>
-                    <workDirectory>target/assembly/work</workDirectory>
-                    <tarLongFileMode>warn</tarLongFileMode>
-                </configuration>
-            </plugin>
-        </plugins>
-    </build>
-</project>
\ No newline at end of file
diff --git a/rocketmq-streams-runner/src/main/resources/log4j.xml b/rocketmq-streams-runner/src/main/resources/log4j.xml
deleted file mode 100644
index de54778..0000000
--- a/rocketmq-streams-runner/src/main/resources/log4j.xml
+++ /dev/null
@@ -1,51 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!DOCTYPE log4j:configuration PUBLIC "-//log4j/log4j Configuration//EN" "log4j.dtd">
-
-<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
-
-
-    <appender name="console" class="org.apache.log4j.ConsoleAppender">
-        <layout class="org.apache.log4j.PatternLayout">
-            <param name="ConversionPattern" value="%d{yyyy-MM-dd HH:mm:ss SSS} %-5p %c{1} %m %n"/>
-        </layout>
-        <filter class="org.apache.log4j.varia.LevelRangeFilter">
-            <param name="levelMin" value="INFO"/>
-            <param name="levelMax" value="ERROR"/>
-            <param name="AcceptOnMatch" value="true"/>
-        </filter>
-    </appender>
-
-    <appender name="fileAppender" class="org.apache.log4j.DailyRollingFileAppender">
-        <param name="File" value="${log4j.home}/rocketmq-streams.log"/>
-        <param name="Append" value="true"/>
-        <param name="DatePattern" value="'.'yyyy-MM-dd'.log'"/>
-        <layout class="org.apache.log4j.PatternLayout">
-            <param name="ConversionPattern" value="%d{yyyy-MM-dd HH:mm:ss SSS} %-5p %c{1} %m %n"/>
-        </layout>
-        <filter class="org.apache.log4j.varia.LevelRangeFilter">
-            <param name="levelMin" value="${log4j.level}"/>
-            <param name="levelMax" value="ERROR"/>
-            <param name="AcceptOnMatch" value="true"/>
-        </filter>
-    </appender>
-
-    <appender name="traceAppender" class="org.apache.log4j.DailyRollingFileAppender">
-        <param name="File" value="${log4j.home}/trace.log"/>
-        <param name="Append" value="true"/>
-        <param name="DatePattern" value="'.'yyyy-MM-dd'.log'"/>
-        <layout class="org.apache.log4j.PatternLayout">
-            <param name="ConversionPattern" value="%d{yyyy-MM-dd HH:mm:ss SSS} %-5p %c{1} %m %n"/>
-        </layout>
-    </appender>
-
-    <!--<category name="org.apache.rocketmq.streams.common.utils.TraceUtil" additivity="false">-->
-    <!--<level value="INFO"/>-->
-    <!--<appender-ref ref="traceAppender"/>-->
-    <!--</category>-->
-
-    <root>
-        <appender-ref ref="console"/>
-        <!--<appender-ref ref="fileAppender"/>-->
-    </root>
-
-</log4j:configuration>
\ No newline at end of file
diff --git a/rocketmq-streams-schedule/pom.xml b/rocketmq-streams-schedule/pom.xml
deleted file mode 100755
index 74cdb89..0000000
--- a/rocketmq-streams-schedule/pom.xml
+++ /dev/null
@@ -1,56 +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">
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.rocketmq</groupId>
-        <artifactId>rocketmq-streams</artifactId>
-        <version>1.0.2-preview-SNAPSHOT</version>
-    </parent>
-    <artifactId>rocketmq-streams-schedule</artifactId>
-    <name>ROCKETMQ STREAMS :: schedule</name>
-    <packaging>jar</packaging>
-    <dependencies>
-
-
-        <dependency>
-            <groupId>org.quartz-scheduler</groupId>
-            <artifactId>quartz</artifactId>
-            <exclusions>
-            <exclusion>
-                <groupId>ch.qos.logback</groupId>
-                <artifactId>logback-classic</artifactId>
-            </exclusion>
-            <exclusion>
-                <groupId>ch.qos.logback</groupId>
-                <artifactId>logback-core</artifactId>
-            </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
-            <groupId>org.quartz-scheduler</groupId>
-            <artifactId>quartz-jobs</artifactId>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-serviceloader</artifactId>
-        </dependency>
-    </dependencies>
-</project>
diff --git a/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/ScheduleComponent.java b/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/ScheduleComponent.java
deleted file mode 100644
index c06a7d4..0000000
--- a/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/ScheduleComponent.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.schedule;
-
-import java.util.Properties;
-import org.apache.rocketmq.streams.common.component.AbstractComponent;
-import org.apache.rocketmq.streams.common.component.ComponentCreator;
-import org.apache.rocketmq.streams.schedule.service.IScheduleService;
-import org.apache.rocketmq.streams.schedule.service.impl.ScheduleServiceImpl;
-
-public class ScheduleComponent extends AbstractComponent<IScheduleService> {
-    private static ScheduleComponent scheduleComponent;
-    protected ScheduleServiceImpl scheduleService = new ScheduleServiceImpl();
-
-    @Override
-    public boolean stop() {
-        scheduleService.stop();
-        return true;
-    }
-
-    public static ScheduleComponent getInstance() {
-        if (scheduleComponent != null) {
-            return scheduleComponent;
-        }
-        synchronized (ScheduleComponent.class) {
-            if (scheduleComponent != null) {
-                return scheduleComponent;
-            }
-            ScheduleComponent tmp = ComponentCreator.getComponent(null, ScheduleComponent.class);
-            scheduleComponent = tmp;
-        }
-        return scheduleComponent;
-    }
-
-    @Override
-    public IScheduleService getService() {
-        return scheduleService;
-    }
-
-    @Override
-    protected boolean startComponent(String name) {
-        scheduleService.start();
-        return true;
-    }
-
-    @Override
-    protected boolean initProperties(Properties properties) {
-        return true;
-    }
-}
diff --git a/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/job/ConfigurableExecutorJob.java b/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/job/ConfigurableExecutorJob.java
deleted file mode 100644
index 4ffc2d5..0000000
--- a/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/job/ConfigurableExecutorJob.java
+++ /dev/null
@@ -1,55 +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.schedule.job;
-
-import com.alibaba.fastjson.JSON;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.interfaces.IScheduleExecutor;
-import org.apache.rocketmq.streams.common.monitor.IMonitor;
-import org.apache.rocketmq.streams.common.monitor.MonitorFactory;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.quartz.DisallowConcurrentExecution;
-import org.quartz.JobDetail;
-import org.quartz.JobExecutionContext;
-import org.quartz.JobExecutionException;
-import org.quartz.StatefulJob;
-
-@DisallowConcurrentExecution
-public class ConfigurableExecutorJob implements StatefulJob {
-    private Log LOG = LogFactory.getLog(ConfigurableExecutorJob.class);
-
-    @Override
-    public void execute(JobExecutionContext context) throws JobExecutionException {
-        IScheduleExecutor channelExecutor = null;
-
-        try {
-            JobDetail jobDetail = context.getJobDetail();
-            channelExecutor = (IScheduleExecutor)jobDetail.getJobDataMap().get(IScheduleExecutor.class.getName());
-            channelExecutor.doExecute();
-        } catch (Exception e) {
-            //降低日志量
-            //            LOG.error("schedule error "+channelExecutor.toString(),e);
-            IMonitor startupMonitor = MonitorFactory.getOrCreateMonitor(MapKeyUtil.createKey(MonitorFactory.PIPLINE_START_UP, channelExecutor.getNameSpace()));
-            IMonitor monitor = startupMonitor.createChildren(channelExecutor.getConfigureName());
-            monitor.addContextMessage(JSON.parse(channelExecutor.toString()));
-            String name = MapKeyUtil.createKeyBySign(".", channelExecutor.getNameSpace(), channelExecutor.getConfigureName());
-            monitor.occureError(e, name + " schedule error", e.getMessage());
-        }
-
-    }
-}
diff --git a/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/service/IScheduleService.java b/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/service/IScheduleService.java
deleted file mode 100644
index 7865210..0000000
--- a/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/service/IScheduleService.java
+++ /dev/null
@@ -1,89 +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.schedule.service;
-
-import java.util.Date;
-import java.util.concurrent.TimeUnit;
-import org.apache.rocketmq.streams.common.interfaces.IScheduleExecutor;
-
-/**
- * 基于quartz 封装定时调度组件
- */
-public interface IScheduleService {
-
-    /**
-     * 启动一个调度,
-     *
-     * @param executor  到时间后,需要执行的业务逻辑
-     * @param crotabStr 基于cron语法的定时描述
-     * @param startNow  是否立刻执行一次
-     */
-    void startSchedule(IScheduleExecutor executor, String crotabStr, boolean startNow);
-
-    /**
-     * 在某个具体时间点执行
-     *
-     * @param executor 到时间后,需要执行的业务逻辑
-     * @param date     具体的执行时间
-     */
-    void startSchedule(IScheduleExecutor executor, Date date);
-
-    /**
-     * n秒轮询一次,允许超过60,需要转化成对应的小时,分,天
-     *
-     * @param executor          到时间后,需要执行的业务逻辑
-     * @param secondPollingTime 轮询时间,单位是秒
-     * @param startNow          是否立刻执行一次
-     */
-    void startSchedule(IScheduleExecutor executor, int secondPollingTime, boolean startNow);
-
-    /**
-     * n秒轮询一次,允许超过60,需要转化成对应的小时,分,天
-     *
-     * @param executor    到时间后,需要执行的业务逻辑
-     * @param pollingTime 轮询时间
-     * @param timeUnit    轮询时间的单位
-     * @param startNow    是否立刻执行一次
-     */
-    void startScheduleUsingPollingTime(IScheduleExecutor executor, int pollingTime, TimeUnit timeUnit, boolean startNow);
-
-    /**
-     * 每天具体时间跑一次
-     *
-     * @param executor 到时间后,需要执行的业务逻辑
-     * @param dateTime 15:00:00
-     * @param startNow 是否立刻执行一次
-     */
-    void startScheduleDailyTime(IScheduleExecutor executor, String dateTime, boolean startNow);
-
-    /**
-     * 每小时跑一次,指定开始的分钟
-     *
-     * @param executor 到时间后,需要执行的业务逻辑
-     * @param dateTime 15:00
-     * @param startNow 是否立刻执行一次
-     */
-    void startScheduleHourTime(IScheduleExecutor executor, String dateTime, boolean startNow);
-
-    /**
-     * 把轮询时间转化成cron
-     *
-     * @param secondPollingTime 轮询时间,单位是秒
-     * @return cron时间描述
-     */
-    String convertCron(int secondPollingTime);
-}
diff --git a/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/service/impl/ScheduleServiceImpl.java b/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/service/impl/ScheduleServiceImpl.java
deleted file mode 100644
index a481ce2..0000000
--- a/rocketmq-streams-schedule/src/main/java/org/apache/rocketmq/streams/schedule/service/impl/ScheduleServiceImpl.java
+++ /dev/null
@@ -1,263 +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.schedule.service.impl;
-
-import java.util.Date;
-import java.util.concurrent.TimeUnit;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.rocketmq.streams.common.interfaces.IScheduleExecutor;
-import org.apache.rocketmq.streams.common.utils.DateUtil;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
-import org.apache.rocketmq.streams.schedule.job.ConfigurableExecutorJob;
-import org.apache.rocketmq.streams.schedule.service.IScheduleService;
-import org.quartz.JobBuilder;
-import org.quartz.JobDataMap;
-import org.quartz.JobDetail;
-import org.quartz.Scheduler;
-import org.quartz.SchedulerException;
-import org.quartz.Trigger;
-import org.quartz.impl.StdSchedulerFactory;
-
-import static org.quartz.CronScheduleBuilder.cronSchedule;
-import static org.quartz.TriggerBuilder.newTrigger;
-
-public class ScheduleServiceImpl implements IScheduleService {
-    private static final Log LOG = LogFactory.getLog(ScheduleServiceImpl.class);
-    protected Scheduler scheduler;
-
-    public ScheduleServiceImpl() {
-        try {
-            scheduler = StdSchedulerFactory.getDefaultScheduler();
-        } catch (SchedulerException e) {
-            throw new RuntimeException("create scheduler container error ", e);
-        }
-    }
-
-    @Override
-    public void startSchedule(IScheduleExecutor channelExecutor, String crotabStr, boolean startNow) {
-        String name = MapKeyUtil.createKey(channelExecutor.getType(), channelExecutor.getConfigureName());
-        Trigger trigger = newTrigger()
-            .withIdentity(name, channelExecutor.getNameSpace())
-            .withSchedule(cronSchedule(crotabStr))
-            .forJob(name, channelExecutor.getNameSpace())
-            .build();
-        try {
-            JobDetail jobDetail = createJobDetail(channelExecutor);
-            scheduler.scheduleJob(jobDetail, trigger);
-            if (startNow) {
-                Trigger startNowTrigger = newTrigger()
-                    .withIdentity(name + "_now", channelExecutor.getNameSpace())
-                    .forJob(jobDetail)
-                    .startNow()
-                    .build();
-
-                scheduler.scheduleJob(startNowTrigger);
-            }
-
-        } catch (SchedulerException e) {
-            throw new RuntimeException("create schedule erro " + channelExecutor.toString(), e);
-        }
-    }
-
-    @Override
-    public void startSchedule(IScheduleExecutor receiver, Date date) {
-        int second = DateUtil.getSecond(date);
-        int mintue = DateUtil.getMinute(date);
-        int hour = DateUtil.getHour(date);
-        int day = DateUtil.getDay(date);
-        int month = DateUtil.getMonth(date);
-        String year = DateUtil.getYear(date);
-        String cron = second + " " + mintue + " " + hour + " " + day + " " + month + " ? " + year;
-        startSchedule(receiver, cron, false);
-    }
-
-    @Override
-    public void startSchedule(IScheduleExecutor receiver, int secondPollingTime, boolean startNow) {
-        String cron = convertCron(secondPollingTime);
-        startSchedule(receiver, cron, startNow);
-    }
-
-    @Override
-    public String convertCron(int secondPollingTime) {
-        String cron = null;
-        if (secondPollingTime < 60) {
-            cron = "0/" + secondPollingTime + " * * * * ?";
-        } else if (secondPollingTime >= 60 && secondPollingTime < 3600) {
-            int minute = secondPollingTime / 60;
-            int second = secondPollingTime % 60;
-            cron = second + " 0/" + minute + " * * * ?";
-        } else if (secondPollingTime >= 3600 && secondPollingTime < 86400) {
-            int hour = secondPollingTime / 3600;
-            int other = secondPollingTime % 3600;
-            String minuteStr = "0";
-            String secondStr = "0";
-            if (other > 60) {
-                int minute = other / 60;
-                int second = other % 60;
-                minuteStr = minute + "";
-                secondStr = second + "";
-            } else {
-                secondStr = other + "";
-            }
-            cron = secondStr + " " + minuteStr + " " + "0/" + hour + " * * ?";
-        } else if (secondPollingTime >= 86400 && secondPollingTime < 86400 * 2) {
-            secondPollingTime = secondPollingTime % 86400;
-            int hour = secondPollingTime / 3600;
-            int other = secondPollingTime % 3600;
-            String minuteStr = "0";
-            String secondStr = "0";
-            if (other > 60) {
-                int minute = other / 60;
-                int second = other % 60;
-                minuteStr = minute + "";
-                secondStr = second + "";
-            } else {
-                secondStr = other + "";
-            }
-            cron = secondStr + " " + minuteStr + " " + hour + " * * ?";
-        } else {
-            throw new RuntimeException("can not support this value ,please use startSchedule cron method");
-        }
-        return cron;
-    }
-
-    @Override
-    public void startScheduleUsingPollingTime(IScheduleExecutor channelExecutor, int pollingTime, TimeUnit timeUnit, boolean startNow) {
-        String crotabStr = createCrotablStr(pollingTime, timeUnit);
-        startSchedule(channelExecutor, crotabStr, startNow);
-    }
-
-    @Override
-    public void startScheduleDailyTime(IScheduleExecutor channelExecutor, String dateTime, boolean startNow) {
-        String crotabStr = createCrotablStr(dateTime, TimeUnit.DAYS);
-        startSchedule(channelExecutor, crotabStr, startNow);
-    }
-
-    @Override
-    public void startScheduleHourTime(IScheduleExecutor channelExecutor, String dateTime, boolean startNow) {
-        String crotabStr = createCrotablStr(dateTime, TimeUnit.HOURS);
-        startSchedule(channelExecutor, crotabStr, startNow);
-    }
-
-    public void start() {
-        try {
-            this.scheduler.start();
-        } catch (SchedulerException e) {
-            throw new RuntimeException("start schedule error ", e);
-        }
-    }
-
-    public void stop() {
-        try {
-            this.scheduler.shutdown();
-        } catch (SchedulerException e) {
-            throw new RuntimeException("start schedule error ", e);
-        }
-    }
-
-    /**
-     * 创建jobdetail
-     *
-     * @param executor
-     * @return
-     */
-    protected JobDetail createJobDetail(IScheduleExecutor executor) {
-        JobDataMap jobDataMap = new JobDataMap();
-        jobDataMap.put(IScheduleExecutor.class.getName(), executor);
-        String name = MapKeyUtil.createKey(executor.getType(), executor.getConfigureName());
-        JobDetail jobDetail = JobBuilder.newJob(ConfigurableExecutorJob.class)
-            .withIdentity(name, executor.getNameSpace()) // name "myJob", group "group1"
-            .usingJobData(jobDataMap)
-            .build();
-        return jobDetail;
-    }
-
-    /**
-     * 轮询时间,多长时间轮询一次
-     *
-     * @param pollingTime
-     * @param timeUnit
-     * @return
-     */
-    protected String createCrotablStr(int pollingTime, TimeUnit timeUnit) {
-        String crotab = null;
-        if (pollingTime > 60) {
-            throw new RuntimeException("pollingTime can not exceed 60, must in the unit " + timeUnit + ". the value is " + pollingTime);
-        }
-        if (TimeUnit.DAYS == timeUnit && pollingTime > 31) {
-            throw new RuntimeException("pollingTime can not exceed 31, must in the day unit . the value is " + pollingTime);
-        }
-        if (TimeUnit.SECONDS == timeUnit) {
-            crotab = "0/" + pollingTime + " * * * * ?";
-        } else if (TimeUnit.MINUTES == timeUnit) {
-            crotab = "0 0/" + pollingTime + " * * * ?";
-        } else if (TimeUnit.HOURS == timeUnit) {
-            crotab = "0 0 0/" + pollingTime + " * * ?";
-        } else if (TimeUnit.DAYS == timeUnit) {
-            crotab = "0 0 0 0/" + pollingTime + " * ?";
-        } else {
-            throw new RuntimeException("can not support the timeunit");
-        }
-        return crotab;
-    }
-
-    /**
-     * 每天/每小时,几点执行一次
-     *
-     * @param dateTime
-     * @param timeUnit
-     * @return
-     */
-    protected String createCrotablStr(String dateTime, TimeUnit timeUnit) {
-        String crotab = null;
-        if (TimeUnit.DAYS == timeUnit) {
-            String[] values = dateTime.split(":");
-            String hour = getTimeValue(values[0]);
-            String minute = getTimeValue(values[1]);
-            String second = getTimeValue(values[2]);
-            crotab = second + " " + minute + " " + hour + " 0/1 * ?";
-        } else if (TimeUnit.HOURS == timeUnit) {
-            String[] values = dateTime.split(":");
-            String minute = getTimeValue(values[0]);
-            String second = getTimeValue(values[1]);
-            crotab = second + " " + minute + " 0/1 * * ?";
-        } else {
-            throw new RuntimeException("can not support the timeunit");
-        }
-        return crotab;
-    }
-
-    private String getTimeValue(String value) {
-        if (value.startsWith("0")) {
-            return value.substring(1);
-        }
-        return value;
-    }
-
-    public static void main(String[] args) throws SchedulerException {
-        Date date = new Date();
-        int second = DateUtil.getSecond(date);
-        int mintue = DateUtil.getMinute(date);
-        int hour = DateUtil.getHour(date);
-        int day = DateUtil.getDay(date);
-        int month = DateUtil.getMonth(date);
-        String year = DateUtil.getYear(date);
-        String cron = second + " " + mintue + " " + hour + " " + day + " " + month + " ? " + year;
-        System.out.println(cron);
-    }
-}
diff --git a/rocketmq-streams-schedule/src/test/java/org/apache/rocketmq/stream/schedule/ScheduleComponentTest.java b/rocketmq-streams-schedule/src/test/java/org/apache/rocketmq/stream/schedule/ScheduleComponentTest.java
deleted file mode 100644
index 69e34e6..0000000
--- a/rocketmq-streams-schedule/src/test/java/org/apache/rocketmq/stream/schedule/ScheduleComponentTest.java
+++ /dev/null
@@ -1,62 +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.stream.schedule;
-
-import org.apache.rocketmq.streams.common.interfaces.IScheduleExecutor;
-import org.apache.rocketmq.streams.common.utils.DateUtil;
-import org.apache.rocketmq.streams.schedule.ScheduleComponent;
-import org.junit.Test;
-
-public class ScheduleComponentTest {
-    private ScheduleComponent scheduleComponent = ScheduleComponent.getInstance();
-
-    @Test
-    public void testSchedule() throws InterruptedException {
-
-        scheduleComponent.getService().startSchedule(create(), "2/2 * * * * ?", true);
-
-        while (true) {
-            Thread.sleep(1000);
-        }
-    }
-
-    protected IScheduleExecutor create() {
-        IScheduleExecutor channelExecutor = new IScheduleExecutor() {
-            @Override
-            public void doExecute() throws InterruptedException {
-                System.out.println(DateUtil.getCurrentTimeString());
-            }
-
-            @Override
-            public String getConfigureName() {
-                return "name";
-            }
-
-            @Override
-            public String getNameSpace() {
-                return "namespace";
-            }
-
-            @Override
-            public String getType() {
-                return "type";
-            }
-
-        };
-        return channelExecutor;
-    }
-}
diff --git a/rocketmq-streams-schedule/src/test/resources/log4j.xml b/rocketmq-streams-schedule/src/test/resources/log4j.xml
deleted file mode 100755
index fb0edb1..0000000
--- a/rocketmq-streams-schedule/src/test/resources/log4j.xml
+++ /dev/null
@@ -1,36 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" ?>
-<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
-<!--
-  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.
-  -->
-<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
-
-    <appender name="Console" class="org.apache.log4j.ConsoleAppender">
-        <layout class="org.apache.log4j.PatternLayout">
-            <param name="ConversionPattern" value="%d{ISO8601} %l [%t] %-5p - %m%n%n"/>
-        </layout>
-        <filter class="org.apache.log4j.varia.LevelRangeFilter">
-            <param name="LevelMin" value="INFO"/>
-            <param name="LevelMax" value="ERROR"/>
-        </filter>
-    </appender>
-
-    <root>
-        <priority value="INFO"/>
-        <appender-ref ref="Console"/>
-    </root>
-
-</log4j:configuration>
\ No newline at end of file
diff --git a/rocketmq-streams-window/pom.xml b/rocketmq-streams-window/pom.xml
index 8483a4e..86d276b 100755
--- a/rocketmq-streams-window/pom.xml
+++ b/rocketmq-streams-window/pom.xml
@@ -29,24 +29,20 @@
 
         <dependency>
             <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-db-operator</artifactId>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
             <artifactId>rocketmq-streams-state</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-lease</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-streams-dim</artifactId>
+            <artifactId>rocketmq-streams-filter</artifactId>
         </dependency>
         <dependency>
             <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..eb513c4 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,8 +706,7 @@
     }
 
     public void removeInstanceFromMap(WindowInstance windowInstance) {
-        this.windowInstanceMap.remove(windowInstance.createWindowInstanceId());
-
+        this.windowInstanceMap.remove(windowInstance.getWindowInstanceId());
     }
 
     @Override
@@ -757,7 +719,7 @@
         return windowCache;
     }
 
-    public WindowStorage getStorage() {
+    public IStorage getStorage() {
         return storage;
     }
 
@@ -765,9 +727,6 @@
         return windowFireSource;
     }
 
-    public IWindowMaxValueManager getWindowMaxValueManager() {
-        return windowMaxValueManager;
-    }
 
     public Long getMsgMaxGapSecond() {
         return msgMaxGapSecond;
@@ -781,16 +740,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 +772,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..65ac261 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,9 @@
                         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 +442,86 @@
         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/ExpressionMatcher.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/ExpressionMatcher.java
new file mode 100644
index 0000000..5295362
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/operator/join/ExpressionMatcher.java
@@ -0,0 +1,141 @@
+/*
+ * 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;
+/*
+ * 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 com.alibaba.fastjson.JSONObject;
+import org.apache.rocketmq.streams.common.utils.DataTypeUtil;
+import org.apache.rocketmq.streams.common.utils.StringUtil;
+import org.apache.rocketmq.streams.filter.builder.ExpressionBuilder;
+import org.apache.rocketmq.streams.filter.operator.Rule;
+import org.apache.rocketmq.streams.filter.operator.expression.Expression;
+import org.apache.rocketmq.streams.filter.operator.expression.RelationExpression;
+import org.apache.rocketmq.streams.script.ScriptComponent;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class ExpressionMatcher {
+    public static List<Map<String, Object>> matchExpressionByLoop(Iterator<Map<String, Object>> it,
+                                                                  String expressionStr, JSONObject msg, boolean needAll) {
+        return matchExpressionByLoop(it, expressionStr, msg, needAll, null, new HashSet<>());
+    }
+
+
+    public static List<Map<String, Object>> matchExpressionByLoop(Iterator<Map<String, Object>> it,
+                                                                  String expressionStr, JSONObject msg, boolean needAll, String script, Set<String> colunmNames) {
+        List<Map<String, Object>> rows = new ArrayList<>();
+        Rule ruleTemplete = ExpressionBuilder.createRule("tmp", "tmpRule", expressionStr);
+        while (it.hasNext()) {
+            Map<String, Object> oldRow = it.next();
+            Map<String, Object> newRow = isMatch(ruleTemplete, oldRow, msg, script, colunmNames);
+            if (newRow != null) {
+                rows.add(newRow);
+                if (!needAll) {
+                    return rows;
+                }
+            }
+        }
+        return rows;
+    }
+
+    public static Map<String, Object> isMatch(Rule ruleTemplete, Map<String, Object> dimRow, JSONObject msgRow,
+                                              String script, Set<String> colunmNames) {
+        Map<String, Object> oldRow = dimRow;
+        Map<String, Object> newRow = executeScript(oldRow, script);
+        if (ruleTemplete == null) {
+            return newRow;
+        }
+        Rule rule = ruleTemplete.copy();
+        Map<String, Expression> expressionMap = new HashMap<>();
+        String dimAsName = null;
+        ;
+        for (Expression expression : rule.getExpressionMap().values()) {
+            expressionMap.put(expression.getConfigureName(), expression);
+            if (expression instanceof RelationExpression) {
+                continue;
+            }
+            Object object = expression.getValue();
+            if (object != null && DataTypeUtil.isString(object.getClass())) {
+                String fieldName = (String) object;
+                Object value = newRow.get(fieldName);
+                if (value != null) {
+                    Expression e = expression.copy();
+                    e.setValue(value.toString());
+                    expressionMap.put(e.getConfigureName(), e);
+                }
+            }
+            if (expression.getVarName().contains(".")) {
+                String[] values = expression.getVarName().split("\\.");
+                if (values.length == 2) {
+                    String asName = values[0];
+                    String varName = values[1];
+                    if (colunmNames.contains(varName)) {
+                        dimAsName = asName;
+                    }
+                }
+
+            }
+        }
+        rule.setExpressionMap(expressionMap);
+        rule.initElements();
+        JSONObject copyMsg = msgRow;
+        if (StringUtil.isNotEmpty(dimAsName)) {
+            copyMsg = new JSONObject(msgRow);
+            for (String key : newRow.keySet()) {
+                copyMsg.put(dimAsName + "." + key, newRow.get(key));
+            }
+        }
+        boolean matched = rule.execute(copyMsg);
+        if (matched) {
+            return newRow;
+        }
+        return null;
+    }
+
+    protected static Map<String, Object> executeScript(Map<String, Object> oldRow, String script) {
+        if (script == null) {
+            return oldRow;
+        }
+        ScriptComponent scriptComponent = ScriptComponent.getInstance();
+        JSONObject msg = new JSONObject();
+        msg.putAll(oldRow);
+        scriptComponent.getService().executeScript(msg, script);
+        return msg;
+    }
+}
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..5b5356b 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;
@@ -34,7 +36,6 @@
 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.dim.model.AbstractDim;
 import org.apache.rocketmq.streams.window.model.WindowCache;
 import org.apache.rocketmq.streams.window.model.WindowInstance;
 import org.apache.rocketmq.streams.window.operator.AbstractShuffleWindow;
@@ -42,18 +43,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 +59,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 +124,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 +135,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 +144,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 +154,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 +183,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 +191,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 +220,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 +251,6 @@
         }
         for (Map.Entry<String, Object> tmp : raw.entrySet()) {
             object.put(rightAsName + "." + tmp.getKey(), tmp.getValue());
-            //            raw.remove(tmp.getKey());
         }
         return object;
     }
@@ -315,15 +267,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 +302,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 +320,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 +336,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 +354,7 @@
             }
         }
 
-        return StringUtil.createMD5Str(buffer.toString());
+        return buffer.toString();
     }
 
     @Override
@@ -418,16 +364,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,21 +381,47 @@
         }
 
         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) {
-
-        return AbstractDim.matchExpressionByLoop(rows.iterator(), expression, msg, true);
+        return ExpressionMatcher.matchExpressionByLoop(rows.iterator(), expression, msg, true);
     }
 
     private List<Map<String, Object>> converToMapFromList(List<WindowBaseValue> rows) {
@@ -540,24 +503,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..5139f70 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,106 @@
 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);
+
+            for (String loadQueueId : loadResult.keySet()) {
+                hasLoad.put(loadQueueId, 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 +130,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 +156,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 +168,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..f9bd341 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,33 @@
         }
         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);
-            }
+        Future<?> future = this.window.getStorage().load(splitIds);
+        for (String splitId : splitIds) {
+             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 +251,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 +268,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 +283,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 +326,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 +403,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 +467,6 @@
         return splitNum > 0 ? splitNum : 32;
     }
 
-    public Set<String> getCurrentQueueIds() {
-        return currentQueueIds;
-    }
 
     public List<ISplit> getQueueList() {
         return queueList;
@@ -530,13 +480,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..c853454 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
@@ -38,11 +38,9 @@
 import org.apache.rocketmq.streams.common.utils.Base64Utils;
 import org.apache.rocketmq.streams.common.utils.DataTypeUtil;
 import org.apache.rocketmq.streams.common.utils.DateUtil;
-import org.apache.rocketmq.streams.common.utils.MapKeyUtil;
 import org.apache.rocketmq.streams.common.utils.ReflectUtil;
 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.script.context.FunctionContext;
 import org.apache.rocketmq.streams.script.operator.impl.AggregationScript;
 import org.apache.rocketmq.streams.script.operator.impl.FunctionScript;
@@ -242,7 +240,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 +318,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 +423,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..217443e
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/AbstractStorage.java
@@ -0,0 +1,112 @@
+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.Set;
+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(Set<String> shuffleIds) {
+        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%
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/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..8b32e8b 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,62 @@
  * 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.Set;
+import java.util.concurrent.Future;
 
-public interface IStorage<T> {
+public interface IStorage {
+    String SEPARATOR = "@";
 
+    Future<?> load(Set<String> shuffleIds);
+
+
+    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%
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/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-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/IBalanceTask.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowJoinType.java
similarity index 85%
rename from rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/IBalanceTask.java
rename to rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowJoinType.java
index 5275fe4..01278c3 100644
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/IBalanceTask.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
@@ -14,11 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.streams.connectors.balance;
 
-/**
- * @description
- */
-public interface IBalanceTask extends Runnable {
-
+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-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/IBalanceTask.java b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowType.java
similarity index 85%
copy from rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/IBalanceTask.java
copy to rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/WindowType.java
index 5275fe4..74029cf 100644
--- a/rocketmq-streams-connectors/src/main/java/org/apache/rocketmq/streams/connectors/balance/IBalanceTask.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
@@ -14,11 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.streams.connectors.balance;
 
-/**
- * @description
- */
-public interface IBalanceTask extends Runnable {
-
+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..5c4fc9b
--- /dev/null
+++ b/rocketmq-streams-window/src/main/java/org/apache/rocketmq/streams/window/storage/rocketmq/DefaultStorage.java
@@ -0,0 +1,379 @@
+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.Set;
+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 static final long pollTimeoutMillis = 50L;
+    private Map<Integer, MessageQueue> queueId2MQ = new HashMap<>();
+    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.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();
+            } catch (Throwable t) {
+                throw new RuntimeException("start rocketmq client error.", t);
+            }
+        }
+    }
+
+    @Override
+    public Future<?> load(Set<String> shuffleIds) {
+        if (isLocalStorageOnly) {
+            return super.load(shuffleIds);
+        }
+
+        HashSet<MessageQueue> queues = new HashSet<>();
+
+        for (String shuffleId : shuffleIds) {
+            MessageQueue messageQueue = getMessageQueue(shuffleId);
+            if (messageQueue == null) {
+                throw new RuntimeException("can not find MQ with shuffleId = [" + shuffleId + "]");
+            }
+            queues.add(messageQueue);
+        }
+
+        this.checkpointConsumer.assign(queues);
+        //从上一offset提交位置,poll到最新数据位置
+        return this.checkpointExecutor.submit(() -> this.pollToLast(queues));
+    }
+
+    private void pollToLast(Set<MessageQueue> messageQueue) {
+        try {
+
+            synchronized (this.checkpointConsumer) {
+                this.checkpointConsumer.assign(messageQueue);
+
+                List<MessageExt> msgs = this.checkpointConsumer.poll(pollTimeoutMillis);
+                while (msgs.size() != 0) {
+                    replayState(msgs);
+                    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);
+            }
+
+            //todo 指定messageQueue提交offset
+            HashSet<MessageQueue> set = new HashSet<>();
+            //提交上次checkpoint/load时,poll消息的offset
+            for (String queueId : queueIdList) {
+                final MessageQueue queue = getMessageQueue(queueId);
+                set.add(queue);
+            }
+
+            this.checkpointConsumer.commit(set, true);
+
+            //poll到最新的checkpoint,为下一次提交offset做准备;
+            this.checkpointExecutor.execute(() -> this.pollToLast(set));
+
+        } 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数量需要相同,broker;
+    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);
+                        for (MessageQueue messageQueue : messageQueues) {
+                            if (shuffleId.contains(messageQueue.getBrokerName())) {
+                                this.queueId2MQ.put(queueId, messageQueue);
+                                break;
+                            }
+                        }
+
+
+                    }
+                }
+            } 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..a07aaa3 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
@@ -24,7 +24,6 @@
 import org.apache.rocketmq.streams.common.context.Message;
 import org.apache.rocketmq.streams.common.utils.CollectionUtil;
 import org.apache.rocketmq.streams.common.utils.DateUtil;
-import org.apache.rocketmq.streams.db.driver.orm.ORMUtil;
 import org.apache.rocketmq.streams.window.model.FireMode;
 import org.apache.rocketmq.streams.window.model.WindowInstance;
 import org.apache.rocketmq.streams.window.operator.AbstractWindow;
@@ -36,19 +35,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);
-    }
-}