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);
- }
-}