diff --git a/broker/pom.xml b/broker/pom.xml
index 8cdafea34947e2526d004c2c97bfe1629fa001a1..0f8ad0a135946ed0642e850f68c6974539d00291 100644
--- a/broker/pom.xml
+++ b/broker/pom.xml
@@ -48,6 +48,10 @@
${project.groupId}rocketmq-srvutil
+
+ ${project.groupId}
+ rocketmq-filter
+ ch.qos.logbacklogback-classic
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
index 6acd40ca7760a0e334d6957e6639cbc8b92f98f1..bacd25ccf5e1131f25a53a1485dba5de846f2815 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
@@ -37,6 +37,8 @@ import org.apache.rocketmq.broker.client.DefaultConsumerIdsChangeListener;
import org.apache.rocketmq.broker.client.ProducerManager;
import org.apache.rocketmq.broker.client.net.Broker2Client;
import org.apache.rocketmq.broker.client.rebalance.RebalanceLockManager;
+import org.apache.rocketmq.broker.filter.CommitLogDispatcherCalcBitMap;
+import org.apache.rocketmq.broker.filter.ConsumerFilterManager;
import org.apache.rocketmq.broker.filtersrv.FilterServerManager;
import org.apache.rocketmq.broker.latency.BrokerFastFailure;
import org.apache.rocketmq.broker.latency.BrokerFixedThreadPoolExecutor;
@@ -96,6 +98,7 @@ public class BrokerController {
private final MessageStoreConfig messageStoreConfig;
private final ConsumerOffsetManager consumerOffsetManager;
private final ConsumerManager consumerManager;
+ private final ConsumerFilterManager consumerFilterManager;
private final ProducerManager producerManager;
private final ClientHousekeepingService clientHousekeepingService;
private final PullMessageProcessor pullMessageProcessor;
@@ -149,6 +152,7 @@ public class BrokerController {
this.messageArrivingListener = new NotifyMessageArrivingListener(this.pullRequestHoldService);
this.consumerIdsChangeListener = new DefaultConsumerIdsChangeListener(this);
this.consumerManager = new ConsumerManager(this.consumerIdsChangeListener);
+ this.consumerFilterManager = new ConsumerFilterManager(this);
this.producerManager = new ProducerManager();
this.clientHousekeepingService = new ClientHousekeepingService(this);
this.broker2Client = new Broker2Client(this);
@@ -192,6 +196,7 @@ public class BrokerController {
result = result && this.consumerOffsetManager.load();
result = result && this.subscriptionGroupManager.load();
+ result = result && this.consumerFilterManager.load();
if (result) {
try {
@@ -202,6 +207,7 @@ public class BrokerController {
//load plugin
MessageStorePluginContext context = new MessageStorePluginContext(messageStoreConfig, brokerStatsManager, messageArrivingListener, brokerConfig);
this.messageStore = MessageStoreFactory.build(context, this.messageStore);
+ this.messageStore.getDispatcherList().addFirst(new CommitLogDispatcherCalcBitMap(this.brokerConfig, this.consumerFilterManager));
} catch (IOException e) {
result = false;
e.printStackTrace();
@@ -274,6 +280,17 @@ public class BrokerController {
}
}, 1000 * 10, this.brokerConfig.getFlushConsumerOffsetInterval(), TimeUnit.MILLISECONDS);
+ this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ BrokerController.this.consumerFilterManager.persist();
+ } catch (Throwable e) {
+ log.error("schedule persist consumer filter error.", e);
+ }
+ }
+ }, 1000 * 10, 1000 * 10, TimeUnit.MILLISECONDS);
+
this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
@Override
public void run() {
@@ -400,9 +417,11 @@ public class BrokerController {
ClientManageProcessor clientProcessor = new ClientManageProcessor(this);
this.remotingServer.registerProcessor(RequestCode.HEART_BEAT, clientProcessor, this.clientManageExecutor);
this.remotingServer.registerProcessor(RequestCode.UNREGISTER_CLIENT, clientProcessor, this.clientManageExecutor);
+ this.remotingServer.registerProcessor(RequestCode.CHECK_CLIENT_CONFIG, clientProcessor, this.clientManageExecutor);
this.fastRemotingServer.registerProcessor(RequestCode.HEART_BEAT, clientProcessor, this.clientManageExecutor);
this.fastRemotingServer.registerProcessor(RequestCode.UNREGISTER_CLIENT, clientProcessor, this.clientManageExecutor);
+ this.remotingServer.registerProcessor(RequestCode.CHECK_CLIENT_CONFIG, clientProcessor, this.clientManageExecutor);
/**
* ConsumerManageProcessor
@@ -504,6 +523,10 @@ public class BrokerController {
return consumerManager;
}
+ public ConsumerFilterManager getConsumerFilterManager() {
+ return consumerFilterManager;
+ }
+
public ConsumerOffsetManager getConsumerOffsetManager() {
return consumerOffsetManager;
}
@@ -590,6 +613,10 @@ public class BrokerController {
if (this.brokerFastFailure != null) {
this.brokerFastFailure.shutdown();
}
+
+ if (this.consumerFilterManager != null) {
+ this.consumerFilterManager.persist();
+ }
}
private void unregisterBrokerAll() {
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerPathConfigHelper.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerPathConfigHelper.java
index 24876df48ea17226d0d5420d086372b7c7f75a3a..0a323ee80c84b4b88f3b8276f1275044330db15b 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerPathConfigHelper.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerPathConfigHelper.java
@@ -44,4 +44,7 @@ public class BrokerPathConfigHelper {
return rootDir + File.separator + "config" + File.separator + "subscriptionGroup.json";
}
+ public static String getConsumerFilterPath(final String rootDir) {
+ return rootDir + File.separator + "config" + File.separator + "consumerFilter.json";
+ }
}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupEvent.java b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupEvent.java
new file mode 100644
index 0000000000000000000000000000000000000000..717fb7085e51fd9bb1356c4f39546901f9d92690
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupEvent.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.broker.client;
+
+public enum ConsumerGroupEvent {
+
+ /**
+ * Some consumers in the group are changed.
+ */
+ CHANGE,
+ /**
+ * The group of consumer is unregistered.
+ */
+ UNREGISTER,
+ /**
+ * The group of consumer is registered.
+ */
+ REGISTER
+}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java
index 07d28dca4bb8f5b6e58221e56290e9f474b5e551..831e2932733fa6a2e9763529aecf479bd7e44c93 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java
@@ -16,9 +16,7 @@
*/
package org.apache.rocketmq.broker.client;
-import io.netty.channel.Channel;
-import java.util.List;
-
public interface ConsumerIdsChangeListener {
- void consumerIdsChanged(final String group, final List channels);
+
+ void handle(ConsumerGroupEvent event, String group, Object... args);
}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerManager.java b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerManager.java
index a2d88d5d4aa17d05412c482b3a81b92811fd24fe..a5ddec8cc2ac88fcbd7f5cc44f5b8af128253358 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerManager.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerManager.java
@@ -85,10 +85,11 @@ public class ConsumerManager {
if (remove != null) {
log.info("unregister consumer ok, no any connection, and remove consumer group, {}",
next.getKey());
+ this.consumerIdsChangeListener.handle(ConsumerGroupEvent.UNREGISTER, next.getKey());
}
}
- this.consumerIdsChangeListener.consumerIdsChanged(next.getKey(), info.getAllChannel());
+ this.consumerIdsChangeListener.handle(ConsumerGroupEvent.CHANGE, next.getKey(), info.getAllChannel());
}
}
}
@@ -111,10 +112,12 @@ public class ConsumerManager {
if (r1 || r2) {
if (isNotifyConsumerIdsChangedEnable) {
- this.consumerIdsChangeListener.consumerIdsChanged(group, consumerGroupInfo.getAllChannel());
+ this.consumerIdsChangeListener.handle(ConsumerGroupEvent.CHANGE, group, consumerGroupInfo.getAllChannel());
}
}
+ this.consumerIdsChangeListener.handle(ConsumerGroupEvent.REGISTER, group, subList);
+
return r1 || r2;
}
@@ -126,10 +129,12 @@ public class ConsumerManager {
ConsumerGroupInfo remove = this.consumerTable.remove(group);
if (remove != null) {
log.info("unregister consumer ok, no any connection, and remove consumer group, {}", group);
+
+ this.consumerIdsChangeListener.handle(ConsumerGroupEvent.UNREGISTER, group);
}
}
if (isNotifyConsumerIdsChangedEnable) {
- this.consumerIdsChangeListener.consumerIdsChanged(group, consumerGroupInfo.getAllChannel());
+ this.consumerIdsChangeListener.handle(ConsumerGroupEvent.CHANGE, group, consumerGroupInfo.getAllChannel());
}
}
}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java b/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java
index a1b2d8ac69580d73c3dfee93552f83b5766df631..d716a339fa037ce68b4d404335677d23f7f859c5 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java
@@ -17,8 +17,12 @@
package org.apache.rocketmq.broker.client;
import io.netty.channel.Channel;
+
+import java.util.Collection;
import java.util.List;
+
import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
public class DefaultConsumerIdsChangeListener implements ConsumerIdsChangeListener {
private final BrokerController brokerController;
@@ -28,11 +32,34 @@ public class DefaultConsumerIdsChangeListener implements ConsumerIdsChangeListen
}
@Override
- public void consumerIdsChanged(String group, List channels) {
- if (channels != null && brokerController.getBrokerConfig().isNotifyConsumerIdsChangedEnable()) {
- for (Channel chl : channels) {
- this.brokerController.getBroker2Client().notifyConsumerIdsChanged(chl, group);
- }
+ public void handle(ConsumerGroupEvent event, String group, Object... args) {
+ if (event == null) {
+ return;
+ }
+ switch (event) {
+ case CHANGE:
+ if (args == null || args.length < 1) {
+ return;
+ }
+ List channels = (List) args[0];
+ if (channels != null && brokerController.getBrokerConfig().isNotifyConsumerIdsChangedEnable()) {
+ for (Channel chl : channels) {
+ this.brokerController.getBroker2Client().notifyConsumerIdsChanged(chl, group);
+ }
+ }
+ break;
+ case UNREGISTER:
+ this.brokerController.getConsumerFilterManager().unRegister(group);
+ break;
+ case REGISTER:
+ if (args == null || args.length < 1) {
+ return;
+ }
+ Collection subscriptionDataList = (Collection) args[0];
+ this.brokerController.getConsumerFilterManager().register(group, subscriptionDataList);
+ break;
+ default:
+ throw new RuntimeException("Unknown event " + event);
}
}
}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filter/CommitLogDispatcherCalcBitMap.java b/broker/src/main/java/org/apache/rocketmq/broker/filter/CommitLogDispatcherCalcBitMap.java
new file mode 100644
index 0000000000000000000000000000000000000000..85415d6297cf2fdae46ef8555e0b1f12f002b55c
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/filter/CommitLogDispatcherCalcBitMap.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.broker.filter;
+
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.filter.util.BitsArray;
+import org.apache.rocketmq.store.CommitLogDispatcher;
+import org.apache.rocketmq.store.DispatchRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Iterator;
+
+/**
+ * Calculate bit map of filter.
+ */
+public class CommitLogDispatcherCalcBitMap implements CommitLogDispatcher {
+
+ private static final Logger log = LoggerFactory.getLogger(LoggerName.FILTER_LOGGER_NAME);
+
+ protected final BrokerConfig brokerConfig;
+ protected final ConsumerFilterManager consumerFilterManager;
+
+ public CommitLogDispatcherCalcBitMap(BrokerConfig brokerConfig, ConsumerFilterManager consumerFilterManager) {
+ this.brokerConfig = brokerConfig;
+ this.consumerFilterManager = consumerFilterManager;
+ }
+
+ @Override
+ public void dispatch(DispatchRequest request) {
+ if (!this.brokerConfig.isEnableCalcFilterBitMap()) {
+ return;
+ }
+
+ try {
+
+ Collection filterDatas = consumerFilterManager.get(request.getTopic());
+
+ if (filterDatas == null || filterDatas.isEmpty()) {
+ return;
+ }
+
+ Iterator iterator = filterDatas.iterator();
+ BitsArray filterBitMap = BitsArray.create(
+ this.consumerFilterManager.getBloomFilter().getM()
+ );
+
+ long startTime = System.currentTimeMillis();
+ while (iterator.hasNext()) {
+ ConsumerFilterData filterData = iterator.next();
+
+ if (filterData.getCompiledExpression() == null) {
+ log.error("[BUG] Consumer in filter manager has no compiled expression! {}", filterData);
+ continue;
+ }
+
+ if (filterData.getBloomFilterData() == null) {
+ log.error("[BUG] Consumer in filter manager has no bloom data! {}", filterData);
+ continue;
+ }
+
+ Object ret = null;
+ try {
+ MessageEvaluationContext context = new MessageEvaluationContext(request.getPropertiesMap());
+
+ ret = filterData.getCompiledExpression().evaluate(context);
+ } catch (Throwable e) {
+ log.error("Calc filter bit map error!commitLogOffset={}, consumer={}, {}", request.getCommitLogOffset(), filterData, e);
+ }
+
+ log.debug("Result of Calc bit map:ret={}, data={}, props={}, offset={}", ret, filterData, request.getPropertiesMap(), request.getCommitLogOffset());
+
+ // eval true
+ if (ret != null && ret instanceof Boolean && (Boolean) ret) {
+ consumerFilterManager.getBloomFilter().hashTo(
+ filterData.getBloomFilterData(),
+ filterBitMap
+ );
+ }
+ }
+
+ request.setBitMap(filterBitMap.bytes());
+
+ long eclipseTime = System.currentTimeMillis() - startTime;
+ // 1ms
+ if (eclipseTime >= 1) {
+ log.warn("Spend {} ms to calc bit map, consumerNum={}, topic={}", eclipseTime, filterDatas.size(), request.getTopic());
+ }
+ } catch (Throwable e) {
+ log.error("Calc bit map error! topic={}, offset={}, queueId={}, {}", request.getTopic(), request.getCommitLogOffset(), request.getQueueId(), e);
+ }
+ }
+}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterData.java b/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterData.java
new file mode 100644
index 0000000000000000000000000000000000000000..4db02e26f8cf9720c5e015a95d97b447477d7c91
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterData.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.broker.filter;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ReflectionToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.rocketmq.filter.expression.Expression;
+import org.apache.rocketmq.filter.util.BloomFilterData;
+
+import java.util.Collections;
+
+/**
+ * Filter data of consumer.
+ */
+public class ConsumerFilterData {
+
+ private String consumerGroup;
+ private String topic;
+ private String expression;
+ private String expressionType;
+ private transient Expression compiledExpression;
+ private long bornTime;
+ private long deadTime = 0;
+ private BloomFilterData bloomFilterData;
+ private long clientVersion;
+
+ public boolean isDead() {
+ return this.deadTime >= this.bornTime;
+ }
+
+ public long howLongAfterDeath() {
+ if (isDead()) {
+ return System.currentTimeMillis() - getDeadTime();
+ }
+ return -1;
+ }
+
+ /**
+ * Check this filter data has been used to calculate bit map when msg was stored in server.
+ *
+ * @param msgStoreTime
+ * @return
+ */
+ public boolean isMsgInLive(long msgStoreTime) {
+ return msgStoreTime > getBornTime();
+ }
+
+ public String getConsumerGroup() {
+ return consumerGroup;
+ }
+
+ public void setConsumerGroup(final String consumerGroup) {
+ this.consumerGroup = consumerGroup;
+ }
+
+ public String getTopic() {
+ return topic;
+ }
+
+ public void setTopic(final String topic) {
+ this.topic = topic;
+ }
+
+ public String getExpression() {
+ return expression;
+ }
+
+ public void setExpression(final String expression) {
+ this.expression = expression;
+ }
+
+ public String getExpressionType() {
+ return expressionType;
+ }
+
+ public void setExpressionType(final String expressionType) {
+ this.expressionType = expressionType;
+ }
+
+ public Expression getCompiledExpression() {
+ return compiledExpression;
+ }
+
+ public void setCompiledExpression(final Expression compiledExpression) {
+ this.compiledExpression = compiledExpression;
+ }
+
+ public long getBornTime() {
+ return bornTime;
+ }
+
+ public void setBornTime(final long bornTime) {
+ this.bornTime = bornTime;
+ }
+
+ public long getDeadTime() {
+ return deadTime;
+ }
+
+ public void setDeadTime(final long deadTime) {
+ this.deadTime = deadTime;
+ }
+
+ public BloomFilterData getBloomFilterData() {
+ return bloomFilterData;
+ }
+
+ public void setBloomFilterData(final BloomFilterData bloomFilterData) {
+ this.bloomFilterData = bloomFilterData;
+ }
+
+ public long getClientVersion() {
+ return clientVersion;
+ }
+
+ public void setClientVersion(long clientVersion) {
+ this.clientVersion = clientVersion;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ return EqualsBuilder.reflectionEquals(this, o, Collections.emptyList());
+ }
+
+ @Override
+ public int hashCode() {
+ return HashCodeBuilder.reflectionHashCode(this, Collections.emptyList());
+ }
+
+ @Override
+ public String toString() {
+ return ReflectionToStringBuilder.toString(this, ToStringStyle.SHORT_PREFIX_STYLE);
+ }
+}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterManager.java b/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterManager.java
new file mode 100644
index 0000000000000000000000000000000000000000..7f790af624f14c44946a9a7cf4f752e663d4ae7d
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterManager.java
@@ -0,0 +1,471 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.broker.filter;
+
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.broker.BrokerPathConfigHelper;
+import org.apache.rocketmq.common.ConfigManager;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
+import org.apache.rocketmq.filter.FilterFactory;
+import org.apache.rocketmq.common.filter.ExpressionType;
+import org.apache.rocketmq.filter.util.BloomFilter;
+import org.apache.rocketmq.filter.util.BloomFilterData;
+import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Consumer filter data manager.Just manage the consumers use expression filter.
+ */
+public class ConsumerFilterManager extends ConfigManager {
+
+ private static final Logger log = LoggerFactory.getLogger(LoggerName.FILTER_LOGGER_NAME);
+
+ private static final long MS_24_HOUR = 24 * 3600 * 1000;
+
+ private ConcurrentHashMap
+ filterDataByTopic = new ConcurrentHashMap(256);
+
+ private transient BrokerController brokerController;
+ private transient BloomFilter bloomFilter;
+
+ public ConsumerFilterManager() {
+ // just for test
+ this.bloomFilter = BloomFilter.createByFn(20, 64);
+ }
+
+ public ConsumerFilterManager(BrokerController brokerController) {
+ this.brokerController = brokerController;
+ this.bloomFilter = BloomFilter.createByFn(
+ brokerController.getBrokerConfig().getMaxErrorRateOfBloomFilter(),
+ brokerController.getBrokerConfig().getExpectConsumerNumUseFilter()
+ );
+ // then set bit map length of store config.
+ brokerController.getMessageStoreConfig().setBitMapLengthConsumeQueueExt(
+ this.bloomFilter.getM()
+ );
+ }
+
+ /**
+ * Build consumer filter data.Be care, bloom filter data is not included.
+ *
+ * @param topic
+ * @param consumerGroup
+ * @param expression
+ * @param type
+ * @param clientVersion
+ * @return maybe null
+ */
+ public static ConsumerFilterData build(final String topic, final String consumerGroup,
+ final String expression, final String type,
+ final long clientVersion) {
+ if (ExpressionType.isTagType(type)) {
+ return null;
+ }
+
+ ConsumerFilterData consumerFilterData = new ConsumerFilterData();
+ consumerFilterData.setTopic(topic);
+ consumerFilterData.setConsumerGroup(consumerGroup);
+ consumerFilterData.setBornTime(System.currentTimeMillis());
+ consumerFilterData.setDeadTime(0);
+ consumerFilterData.setExpression(expression);
+ consumerFilterData.setExpressionType(type);
+ consumerFilterData.setClientVersion(clientVersion);
+ try {
+ consumerFilterData.setCompiledExpression(
+ FilterFactory.INSTANCE.get(type).compile(expression)
+ );
+ } catch (Throwable e) {
+ log.error("parse error: expr={}, topic={}, group={}, error={}", expression, topic, consumerGroup, e.getMessage());
+ return null;
+ }
+
+ return consumerFilterData;
+ }
+
+ public void register(final String consumerGroup, final Collection subList) {
+ for (SubscriptionData subscriptionData : subList) {
+ register(
+ subscriptionData.getTopic(),
+ consumerGroup,
+ subscriptionData.getSubString(),
+ subscriptionData.getExpressionType(),
+ subscriptionData.getSubVersion()
+ );
+ }
+
+ // make illegal topic dead.
+ Collection groupFilterData = getByGroup(consumerGroup);
+
+ Iterator iterator = groupFilterData.iterator();
+ while (iterator.hasNext()) {
+ ConsumerFilterData filterData = iterator.next();
+
+ boolean exist = false;
+ for (SubscriptionData subscriptionData : subList) {
+ if (subscriptionData.getTopic().equals(filterData.getTopic())) {
+ exist = true;
+ break;
+ }
+ }
+
+ if (!exist && !filterData.isDead()) {
+ filterData.setDeadTime(System.currentTimeMillis());
+ log.info("Consumer filter changed: {}, make illegal topic dead:{}", consumerGroup, filterData);
+ }
+ }
+ }
+
+ public boolean register(final String topic, final String consumerGroup, final String expression,
+ final String type, final long clientVersion) {
+ if (ExpressionType.isTagType(type)) {
+ return false;
+ }
+
+ if (expression == null || expression.length() == 0) {
+ return false;
+ }
+
+ FilterDataMapByTopic filterDataMapByTopic = this.filterDataByTopic.get(topic);
+
+ if (filterDataMapByTopic == null) {
+ FilterDataMapByTopic temp = new FilterDataMapByTopic(topic);
+ FilterDataMapByTopic prev = this.filterDataByTopic.putIfAbsent(topic, temp);
+ filterDataMapByTopic = prev != null ? prev : temp;
+ }
+
+ BloomFilterData bloomFilterData = bloomFilter.generate(consumerGroup + "#" + topic);
+
+ return filterDataMapByTopic.register(consumerGroup, expression, type, bloomFilterData, clientVersion);
+ }
+
+ public void unRegister(final String consumerGroup) {
+ for (String topic : filterDataByTopic.keySet()) {
+ this.filterDataByTopic.get(topic).unRegister(consumerGroup);
+ }
+ }
+
+ public ConsumerFilterData get(final String topic, final String consumerGroup) {
+ if (!this.filterDataByTopic.containsKey(topic)) {
+ return null;
+ }
+ if (this.filterDataByTopic.get(topic).getGroupFilterData().isEmpty()) {
+ return null;
+ }
+
+ return this.filterDataByTopic.get(topic).getGroupFilterData().get(consumerGroup);
+ }
+
+ public Collection getByGroup(final String consumerGroup) {
+ Collection ret = new HashSet();
+
+ Iterator topicIterator = this.filterDataByTopic.values().iterator();
+ while (topicIterator.hasNext()) {
+ FilterDataMapByTopic filterDataMapByTopic = topicIterator.next();
+
+ Iterator filterDataIterator = filterDataMapByTopic.getGroupFilterData().values().iterator();
+
+ while (filterDataIterator.hasNext()) {
+ ConsumerFilterData filterData = filterDataIterator.next();
+
+ if (filterData.getConsumerGroup().equals(consumerGroup)) {
+ ret.add(filterData);
+ }
+ }
+ }
+
+ return ret;
+ }
+
+ public final Collection get(final String topic) {
+ if (!this.filterDataByTopic.containsKey(topic)) {
+ return null;
+ }
+ if (this.filterDataByTopic.get(topic).getGroupFilterData().isEmpty()) {
+ return null;
+ }
+
+ return this.filterDataByTopic.get(topic).getGroupFilterData().values();
+ }
+
+ public BloomFilter getBloomFilter() {
+ return bloomFilter;
+ }
+
+ @Override
+ public String encode() {
+ return encode(false);
+ }
+
+ @Override
+ public String configFilePath() {
+ if (this.brokerController != null) {
+ return BrokerPathConfigHelper.getConsumerFilterPath(
+ this.brokerController.getMessageStoreConfig().getStorePathRootDir()
+ );
+ }
+ return BrokerPathConfigHelper.getConsumerFilterPath("./unit_test");
+ }
+
+ @Override
+ public void decode(final String jsonString) {
+ ConsumerFilterManager load = RemotingSerializable.fromJson(jsonString, ConsumerFilterManager.class);
+ if (load != null && load.filterDataByTopic != null) {
+ boolean bloomChanged = false;
+ for (String topic : load.filterDataByTopic.keySet()) {
+ FilterDataMapByTopic dataMapByTopic = load.filterDataByTopic.get(topic);
+ if (dataMapByTopic == null) {
+ continue;
+ }
+
+ for (String group : dataMapByTopic.getGroupFilterData().keySet()) {
+
+ ConsumerFilterData filterData = dataMapByTopic.getGroupFilterData().get(group);
+
+ if (filterData == null) {
+ continue;
+ }
+
+ try {
+ filterData.setCompiledExpression(
+ FilterFactory.INSTANCE.get(filterData.getExpressionType()).compile(filterData.getExpression())
+ );
+ } catch (Exception e) {
+ log.error("load filter data error, " + filterData, e);
+ }
+
+ // check whether bloom filter is changed
+ // if changed, ignore the bit map calculated before.
+ if (!this.bloomFilter.isValid(filterData.getBloomFilterData())) {
+ bloomChanged = true;
+ log.info("Bloom filter is changed!So ignore all filter data persisted! {}, {}", this.bloomFilter, filterData.getBloomFilterData());
+ break;
+ }
+
+ log.info("load exist consumer filter data: {}", filterData);
+
+ if (filterData.getDeadTime() == 0) {
+ // we think all consumers are dead when load
+ long deadTime = System.currentTimeMillis() - 30 * 1000;
+ filterData.setDeadTime(
+ deadTime <= filterData.getBornTime() ? filterData.getBornTime() : deadTime
+ );
+ }
+ }
+ }
+
+ if (!bloomChanged) {
+ this.filterDataByTopic = load.filterDataByTopic;
+ }
+ }
+ }
+
+ @Override
+ public String encode(final boolean prettyFormat) {
+ // clean
+ {
+ clean();
+ }
+ return RemotingSerializable.toJson(this, prettyFormat);
+ }
+
+ public void clean() {
+ Iterator> topicIterator = this.filterDataByTopic.entrySet().iterator();
+ while (topicIterator.hasNext()) {
+ Map.Entry filterDataMapByTopic = topicIterator.next();
+
+ Iterator> filterDataIterator
+ = filterDataMapByTopic.getValue().getGroupFilterData().entrySet().iterator();
+
+ while (filterDataIterator.hasNext()) {
+ Map.Entry filterDataByGroup = filterDataIterator.next();
+
+ ConsumerFilterData filterData = filterDataByGroup.getValue();
+ if (filterData.howLongAfterDeath() >= (this.brokerController == null ? MS_24_HOUR : this.brokerController.getBrokerConfig().getFilterDataCleanTimeSpan())) {
+ log.info("Remove filter consumer {}, died too long!", filterDataByGroup.getValue());
+ filterDataIterator.remove();
+ }
+ }
+
+ if (filterDataMapByTopic.getValue().getGroupFilterData().isEmpty()) {
+ log.info("Topic has no consumer, remove it! {}", filterDataMapByTopic.getKey());
+ topicIterator.remove();
+ }
+ }
+ }
+
+ public ConcurrentHashMap getFilterDataByTopic() {
+ return filterDataByTopic;
+ }
+
+ public void setFilterDataByTopic(final ConcurrentHashMap filterDataByTopic) {
+ this.filterDataByTopic = filterDataByTopic;
+ }
+
+ public static class FilterDataMapByTopic {
+
+ private ConcurrentHashMap
+ groupFilterData = new ConcurrentHashMap();
+
+ private String topic;
+
+ public FilterDataMapByTopic() {
+ }
+
+ public FilterDataMapByTopic(String topic) {
+ this.topic = topic;
+ }
+
+ public void unRegister(String consumerGroup) {
+ if (!this.groupFilterData.containsKey(consumerGroup)) {
+ return;
+ }
+
+ ConsumerFilterData data = this.groupFilterData.get(consumerGroup);
+
+ if (data == null || data.isDead()) {
+ return;
+ }
+
+ long now = System.currentTimeMillis();
+
+ log.info("Unregister consumer filter: {}, deadTime: {}", data, now);
+
+ data.setDeadTime(now);
+ }
+
+ public boolean register(String consumerGroup, String expression, String type, BloomFilterData bloomFilterData, long clientVersion) {
+ ConsumerFilterData old = this.groupFilterData.get(consumerGroup);
+
+ if (old == null) {
+ ConsumerFilterData consumerFilterData = build(topic, consumerGroup, expression, type, clientVersion);
+ if (consumerFilterData == null) {
+ return false;
+ }
+ consumerFilterData.setBloomFilterData(bloomFilterData);
+
+ old = this.groupFilterData.putIfAbsent(consumerGroup, consumerFilterData);
+ if (old == null) {
+ log.info("New consumer filter registered: {}", consumerFilterData);
+ return true;
+ } else {
+ if (clientVersion <= old.getClientVersion()) {
+ if (!type.equals(old.getExpressionType()) || !expression.equals(old.getExpression())) {
+ log.warn("Ignore consumer({} : {}) filter(concurrent), because of version {} <= {}, but maybe info changed!old={}:{}, ignored={}:{}",
+ consumerGroup, topic,
+ clientVersion, old.getClientVersion(),
+ old.getExpressionType(), old.getExpression(),
+ type, expression);
+ }
+ if (clientVersion == old.getClientVersion() && old.isDead()) {
+ reAlive(old);
+ return true;
+ }
+
+ return false;
+ } else {
+ this.groupFilterData.put(consumerGroup, consumerFilterData);
+ log.info("New consumer filter registered(concurrent): {}, old: {}", consumerFilterData, old);
+ return true;
+ }
+ }
+ } else {
+ if (clientVersion <= old.getClientVersion()) {
+ if (!type.equals(old.getExpressionType()) || !expression.equals(old.getExpression())) {
+ log.info("Ignore consumer({}:{}) filter, because of version {} <= {}, but maybe info changed!old={}:{}, ignored={}:{}",
+ consumerGroup, topic,
+ clientVersion, old.getClientVersion(),
+ old.getExpressionType(), old.getExpression(),
+ type, expression);
+ }
+ if (clientVersion == old.getClientVersion() && old.isDead()) {
+ reAlive(old);
+ return true;
+ }
+
+ return false;
+ }
+
+ boolean change = !old.getExpression().equals(expression) || !old.getExpressionType().equals(type);
+ if (old.getBloomFilterData() == null && bloomFilterData != null) {
+ change = true;
+ }
+ if (old.getBloomFilterData() != null && !old.getBloomFilterData().equals(bloomFilterData)) {
+ change = true;
+ }
+
+ // if subscribe data is changed, or consumer is died too long.
+ if (change) {
+ ConsumerFilterData consumerFilterData = build(topic, consumerGroup, expression, type, clientVersion);
+ if (consumerFilterData == null) {
+ // new expression compile error, remove old, let client report error.
+ this.groupFilterData.remove(consumerGroup);
+ return false;
+ }
+ consumerFilterData.setBloomFilterData(bloomFilterData);
+
+ this.groupFilterData.put(consumerGroup, consumerFilterData);
+
+ log.info("Consumer filter info change, old: {}, new: {}, change: {}",
+ old, consumerFilterData, change);
+
+ return true;
+ } else {
+ old.setClientVersion(clientVersion);
+ if (old.isDead()) {
+ reAlive(old);
+ }
+ return true;
+ }
+ }
+ }
+
+ protected void reAlive(ConsumerFilterData filterData) {
+ long oldDeadTime = filterData.getDeadTime();
+ filterData.setDeadTime(0);
+ log.info("Re alive consumer filter: {}, oldDeadTime: {}", filterData, oldDeadTime);
+ }
+
+ public final ConsumerFilterData get(String consumerGroup) {
+ return this.groupFilterData.get(consumerGroup);
+ }
+
+ public final ConcurrentHashMap getGroupFilterData() {
+ return this.groupFilterData;
+ }
+
+ public void setGroupFilterData(final ConcurrentHashMap groupFilterData) {
+ this.groupFilterData = groupFilterData;
+ }
+
+ public String getTopic() {
+ return topic;
+ }
+
+ public void setTopic(final String topic) {
+ this.topic = topic;
+ }
+ }
+}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filter/ExpressionForRetryMessageFilter.java b/broker/src/main/java/org/apache/rocketmq/broker/filter/ExpressionForRetryMessageFilter.java
new file mode 100644
index 0000000000000000000000000000000000000000..95181789517c2565aa72a30eeea28d0cc7c4f680
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/filter/ExpressionForRetryMessageFilter.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.broker.filter;
+
+
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.common.filter.ExpressionType;
+import org.apache.rocketmq.common.message.MessageConst;
+import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+/**
+ * Support filter to retry topic.
+ * It will decode properties first in order to get real topic.
+ */
+public class ExpressionForRetryMessageFilter extends ExpressionMessageFilter {
+ public ExpressionForRetryMessageFilter(SubscriptionData subscriptionData, ConsumerFilterData consumerFilterData, ConsumerFilterManager consumerFilterManager) {
+ super(subscriptionData, consumerFilterData, consumerFilterManager);
+ }
+
+ @Override
+ public boolean isMatchedByCommitLog(ByteBuffer msgBuffer, Map properties) {
+ if (subscriptionData == null) {
+ return true;
+ }
+
+ if (subscriptionData.isClassFilterMode()) {
+ return true;
+ }
+
+ boolean isRetryTopic = subscriptionData.getTopic().startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX);
+
+ if (!isRetryTopic && ExpressionType.isTagType(subscriptionData.getExpressionType())) {
+ return true;
+ }
+
+ ConsumerFilterData realFilterData = this.consumerFilterData;
+ Map tempProperties = properties;
+ boolean decoded = false;
+ if (isRetryTopic) {
+ // retry topic, use original filter data.
+ // poor performance to support retry filter.
+ if (tempProperties == null && msgBuffer != null) {
+ decoded = true;
+ tempProperties = MessageDecoder.decodeProperties(msgBuffer);
+ }
+ String realTopic = tempProperties.get(MessageConst.PROPERTY_RETRY_TOPIC);
+ String group = subscriptionData.getTopic().substring(MixAll.RETRY_GROUP_TOPIC_PREFIX.length());
+ realFilterData = this.consumerFilterManager.get(realTopic, group);
+ }
+
+ // no expression
+ if (realFilterData == null || realFilterData.getExpression() == null
+ || realFilterData.getCompiledExpression() == null) {
+ return true;
+ }
+
+ if (!decoded && tempProperties == null && msgBuffer != null) {
+ tempProperties = MessageDecoder.decodeProperties(msgBuffer);
+ }
+
+ Object ret = null;
+ try {
+ MessageEvaluationContext context = new MessageEvaluationContext(tempProperties);
+
+ ret = realFilterData.getCompiledExpression().evaluate(context);
+ } catch (Throwable e) {
+ log.error("Message Filter error, " + realFilterData + ", " + tempProperties, e);
+ }
+
+ log.debug("Pull eval result: {}, {}, {}", ret, realFilterData, tempProperties);
+
+ if (ret == null || !(ret instanceof Boolean)) {
+ return false;
+ }
+
+ return (Boolean) ret;
+ }
+}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filter/ExpressionMessageFilter.java b/broker/src/main/java/org/apache/rocketmq/broker/filter/ExpressionMessageFilter.java
new file mode 100644
index 0000000000000000000000000000000000000000..893df0d7c6b94d0892b6d358facfcbef887d28b7
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/filter/ExpressionMessageFilter.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.broker.filter;
+
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.filter.ExpressionType;
+import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
+import org.apache.rocketmq.filter.util.BitsArray;
+import org.apache.rocketmq.filter.util.BloomFilter;
+import org.apache.rocketmq.store.ConsumeQueueExt;
+import org.apache.rocketmq.store.MessageFilter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+public class ExpressionMessageFilter implements MessageFilter {
+
+ protected static final Logger log = LoggerFactory.getLogger(LoggerName.FILTER_LOGGER_NAME);
+
+ protected final SubscriptionData subscriptionData;
+ protected final ConsumerFilterData consumerFilterData;
+ protected final ConsumerFilterManager consumerFilterManager;
+ protected final boolean bloomDataValid;
+
+ public ExpressionMessageFilter(SubscriptionData subscriptionData, ConsumerFilterData consumerFilterData,
+ ConsumerFilterManager consumerFilterManager) {
+ this.subscriptionData = subscriptionData;
+ this.consumerFilterData = consumerFilterData;
+ this.consumerFilterManager = consumerFilterManager;
+ if (consumerFilterData == null) {
+ bloomDataValid = false;
+ return;
+ }
+ BloomFilter bloomFilter = this.consumerFilterManager.getBloomFilter();
+ if (bloomFilter != null && bloomFilter.isValid(consumerFilterData.getBloomFilterData())) {
+ bloomDataValid = true;
+ } else {
+ bloomDataValid = false;
+ }
+ }
+
+ @Override
+ public boolean isMatchedByConsumeQueue(Long tagsCode, ConsumeQueueExt.CqExtUnit cqExtUnit) {
+ if (null == subscriptionData) {
+ return true;
+ }
+
+ if (subscriptionData.isClassFilterMode()) {
+ return true;
+ }
+
+ // by tags code.
+ if (ExpressionType.isTagType(subscriptionData.getExpressionType())) {
+
+ if (tagsCode == null || tagsCode < 0L) {
+ return true;
+ }
+
+ if (subscriptionData.getSubString().equals(SubscriptionData.SUB_ALL)) {
+ return true;
+ }
+
+ return subscriptionData.getCodeSet().contains(tagsCode.intValue());
+ } else {
+ // no expression or no bloom
+ if (consumerFilterData == null || consumerFilterData.getExpression() == null
+ || consumerFilterData.getCompiledExpression() == null || consumerFilterData.getBloomFilterData() == null) {
+ return true;
+ }
+
+ // message is before consumer
+ if (cqExtUnit == null || !consumerFilterData.isMsgInLive(cqExtUnit.getMsgStoreTime())) {
+ log.debug("Pull matched because not in live: {}, {}", consumerFilterData, cqExtUnit);
+ return true;
+ }
+
+ byte[] filterBitMap = cqExtUnit.getFilterBitMap();
+ BloomFilter bloomFilter = this.consumerFilterManager.getBloomFilter();
+ if (filterBitMap == null || !this.bloomDataValid
+ || filterBitMap.length * Byte.SIZE != consumerFilterData.getBloomFilterData().getBitNum()) {
+ return true;
+ }
+
+ BitsArray bitsArray = null;
+ try {
+ bitsArray = BitsArray.create(filterBitMap);
+ boolean ret = bloomFilter.isHit(consumerFilterData.getBloomFilterData(), bitsArray);
+ log.debug("Pull {} by bit map:{}, {}, {}", ret, consumerFilterData, bitsArray, cqExtUnit);
+ return ret;
+ } catch (Throwable e) {
+ log.error("bloom filter error, sub=" + subscriptionData
+ + ", filter=" + consumerFilterData + ", bitMap=" + bitsArray, e);
+ }
+ }
+
+ return true;
+ }
+
+ @Override
+ public boolean isMatchedByCommitLog(ByteBuffer msgBuffer, Map properties) {
+ if (subscriptionData == null) {
+ return true;
+ }
+
+ if (subscriptionData.isClassFilterMode()) {
+ return true;
+ }
+
+ if (ExpressionType.isTagType(subscriptionData.getExpressionType())) {
+ return true;
+ }
+
+ ConsumerFilterData realFilterData = this.consumerFilterData;
+ Map tempProperties = properties;
+
+ // no expression
+ if (realFilterData == null || realFilterData.getExpression() == null
+ || realFilterData.getCompiledExpression() == null) {
+ return true;
+ }
+
+ if (tempProperties == null && msgBuffer != null) {
+ tempProperties = MessageDecoder.decodeProperties(msgBuffer);
+ }
+
+ Object ret = null;
+ try {
+ MessageEvaluationContext context = new MessageEvaluationContext(tempProperties);
+
+ ret = realFilterData.getCompiledExpression().evaluate(context);
+ } catch (Throwable e) {
+ log.error("Message Filter error, " + realFilterData + ", " + tempProperties, e);
+ }
+
+ log.debug("Pull eval result: {}, {}, {}", ret, realFilterData, tempProperties);
+
+ if (ret == null || !(ret instanceof Boolean)) {
+ return false;
+ }
+
+ return (Boolean) ret;
+ }
+
+}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filter/MessageEvaluationContext.java b/broker/src/main/java/org/apache/rocketmq/broker/filter/MessageEvaluationContext.java
new file mode 100644
index 0000000000000000000000000000000000000000..879d17908f6c23c2c27aafcfcae4a878a2f4c9c7
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/filter/MessageEvaluationContext.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.broker.filter;
+
+import org.apache.rocketmq.filter.expression.EvaluationContext;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Evaluation context from message.
+ */
+public class MessageEvaluationContext implements EvaluationContext {
+
+ private Map properties;
+
+ public MessageEvaluationContext(Map properties) {
+ this.properties = properties;
+ }
+
+ @Override
+ public Object get(final String name) {
+ if (this.properties == null) {
+ return null;
+ }
+ return this.properties.get(name);
+ }
+
+ @Override
+ public Map keyValues() {
+ if (properties == null) {
+ return null;
+ }
+
+ Map copy = new HashMap(properties.size(), 1);
+
+ for (String key : properties.keySet()) {
+ copy.put(key, properties.get(key));
+ }
+
+ return copy;
+ }
+}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java
index 2dec9f746ba75643604b0df7c819eb86d378ab21..fd38c4fb7660d358cc7ffe87544f267e32a3aced 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java
@@ -19,6 +19,8 @@ package org.apache.rocketmq.broker.longpolling;
import org.apache.rocketmq.store.MessageArrivingListener;
+import java.util.Map;
+
public class NotifyMessageArrivingListener implements MessageArrivingListener {
private final PullRequestHoldService pullRequestHoldService;
@@ -27,7 +29,9 @@ public class NotifyMessageArrivingListener implements MessageArrivingListener {
}
@Override
- public void arriving(String topic, int queueId, long logicOffset, long tagsCode) {
- this.pullRequestHoldService.notifyMessageArriving(topic, queueId, logicOffset, tagsCode);
+ public void arriving(String topic, int queueId, long logicOffset, long tagsCode,
+ long msgStoreTime, byte[] filterBitMap, Map properties) {
+ this.pullRequestHoldService.notifyMessageArriving(topic, queueId, logicOffset, tagsCode,
+ msgStoreTime, filterBitMap, properties);
}
}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequest.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequest.java
index b66344f3bef2f2e938312be225e5f5ec9996bbcf..045ab9b16a2985aad530e2b8d7873770b1185e3d 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequest.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequest.java
@@ -19,6 +19,7 @@ package org.apache.rocketmq.broker.longpolling;
import io.netty.channel.Channel;
import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+import org.apache.rocketmq.store.MessageFilter;
public class PullRequest {
private final RemotingCommand requestCommand;
@@ -27,15 +28,18 @@ public class PullRequest {
private final long suspendTimestamp;
private final long pullFromThisOffset;
private final SubscriptionData subscriptionData;
+ private final MessageFilter messageFilter;
public PullRequest(RemotingCommand requestCommand, Channel clientChannel, long timeoutMillis, long suspendTimestamp,
- long pullFromThisOffset, SubscriptionData subscriptionData) {
+ long pullFromThisOffset, SubscriptionData subscriptionData,
+ MessageFilter messageFilter) {
this.requestCommand = requestCommand;
this.clientChannel = clientChannel;
this.timeoutMillis = timeoutMillis;
this.suspendTimestamp = suspendTimestamp;
this.pullFromThisOffset = pullFromThisOffset;
this.subscriptionData = subscriptionData;
+ this.messageFilter = messageFilter;
}
public RemotingCommand getRequestCommand() {
@@ -61,4 +65,8 @@ public class PullRequest {
public SubscriptionData getSubscriptionData() {
return subscriptionData;
}
+
+ public MessageFilter getMessageFilter() {
+ return messageFilter;
+ }
}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java
index fdba50daaf6d8a320efbfb29d8ec4aa58229dce0..1a53db198bb08ae7cfffb3ceb59f4af89a2d63c5 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java
@@ -18,13 +18,13 @@ package org.apache.rocketmq.broker.longpolling;
import java.util.ArrayList;
import java.util.List;
+import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.rocketmq.broker.BrokerController;
import org.apache.rocketmq.common.ServiceThread;
import org.apache.rocketmq.common.SystemClock;
import org.apache.rocketmq.common.constant.LoggerName;
-import org.apache.rocketmq.store.DefaultMessageFilter;
-import org.apache.rocketmq.store.MessageFilter;
+import org.apache.rocketmq.store.ConsumeQueueExt;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -33,7 +33,6 @@ public class PullRequestHoldService extends ServiceThread {
private static final String TOPIC_QUEUEID_SEPARATOR = "@";
private final BrokerController brokerController;
private final SystemClock systemClock = new SystemClock();
- private final MessageFilter messageFilter = new DefaultMessageFilter();
private ConcurrentHashMap pullRequestTable =
new ConcurrentHashMap(1024);
@@ -110,10 +109,11 @@ public class PullRequestHoldService extends ServiceThread {
}
public void notifyMessageArriving(final String topic, final int queueId, final long maxOffset) {
- notifyMessageArriving(topic, queueId, maxOffset, null);
+ notifyMessageArriving(topic, queueId, maxOffset, null, 0, null, null);
}
- public void notifyMessageArriving(final String topic, final int queueId, final long maxOffset, final Long tagsCode) {
+ public void notifyMessageArriving(final String topic, final int queueId, final long maxOffset, final Long tagsCode,
+ long msgStoreTime, byte[] filterBitMap, Map properties) {
String key = this.buildKey(topic, queueId);
ManyPullRequest mpr = this.pullRequestTable.get(key);
if (mpr != null) {
@@ -128,7 +128,14 @@ public class PullRequestHoldService extends ServiceThread {
}
if (newestOffset > request.getPullFromThisOffset()) {
- if (this.messageFilter.isMessageMatched(request.getSubscriptionData(), tagsCode)) {
+ boolean match = request.getMessageFilter().isMatchedByConsumeQueue(tagsCode,
+ new ConsumeQueueExt.CqExtUnit(tagsCode, msgStoreTime, filterBitMap));
+ // match by bit map, need eval again when properties is not null.
+ if (match && properties != null) {
+ match = request.getMessageFilter().isMatchedByCommitLog(null, properties);
+ }
+
+ if (match) {
try {
this.brokerController.getPullMessageProcessor().executeRequestWhenWakeup(request.getClientChannel(),
request.getRequestCommand());
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java
index 039c94298488a06ababcb84d7ba31eec72cc7a76..6c2a987d29eea590f7d485499262592c44b69feb 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java
@@ -50,7 +50,7 @@ import org.slf4j.LoggerFactory;
public class BrokerOuterAPI {
private static final Logger log = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
private final RemotingClient remotingClient;
- private final TopAddressing topAddressing = new TopAddressing(MixAll.WS_ADDR);
+ private final TopAddressing topAddressing = new TopAddressing(MixAll.getWSAddr());
private String nameSrvAddr = null;
public BrokerOuterAPI(final NettyClientConfig nettyClientConfig) {
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java b/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java
index 00257fd680957d70d1bb22272eb8b1e9fbabc079..8ded973e0eb14ced3971524f34086c3d2d834a00 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java
@@ -18,11 +18,14 @@
package org.apache.rocketmq.broker.plugin;
import java.util.HashMap;
+import java.util.LinkedList;
import java.util.Set;
import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
+import org.apache.rocketmq.store.CommitLogDispatcher;
+import org.apache.rocketmq.store.ConsumeQueue;
import org.apache.rocketmq.store.GetMessageResult;
import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.store.MessageFilter;
import org.apache.rocketmq.store.MessageStore;
import org.apache.rocketmq.store.PutMessageResult;
import org.apache.rocketmq.store.QueryMessageResult;
@@ -84,8 +87,8 @@ public abstract class AbstractPluginMessageStore implements MessageStore {
@Override
public GetMessageResult getMessage(String group, String topic, int queueId, long offset,
- int maxMsgNums, SubscriptionData subscriptionData) {
- return next.getMessage(group, topic, queueId, offset, maxMsgNums, subscriptionData);
+ int maxMsgNums, final MessageFilter messageFilter) {
+ return next.getMessage(group, topic, queueId, offset, maxMsgNums, messageFilter);
}
@Override
@@ -234,4 +237,13 @@ public abstract class AbstractPluginMessageStore implements MessageStore {
next.setConfirmOffset(phyOffset);
}
+ @Override
+ public LinkedList getDispatcherList() {
+ return next.getDispatcherList();
+ }
+
+ @Override
+ public ConsumeQueue getConsumeQueue(String topic, int queueId) {
+ return next.getConsumeQueue(topic, queueId);
+ }
}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
index e35316d603dfccd9b0902e8b564cd7912bed8235..daea53c7de89fb05dc2d0ec4e96f841b655bc3da 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
@@ -16,6 +16,7 @@
*/
package org.apache.rocketmq.broker.processor;
+import com.alibaba.fastjson.JSON;
import io.netty.channel.Channel;
import io.netty.channel.ChannelHandlerContext;
import java.io.UnsupportedEncodingException;
@@ -32,6 +33,8 @@ import java.util.concurrent.ConcurrentHashMap;
import org.apache.rocketmq.broker.BrokerController;
import org.apache.rocketmq.broker.client.ClientChannelInfo;
import org.apache.rocketmq.broker.client.ConsumerGroupInfo;
+import org.apache.rocketmq.broker.filter.ConsumerFilterData;
+import org.apache.rocketmq.broker.filter.ExpressionMessageFilter;
import org.apache.rocketmq.common.MQVersion;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.TopicConfig;
@@ -49,6 +52,7 @@ import org.apache.rocketmq.common.protocol.ResponseCode;
import org.apache.rocketmq.common.protocol.body.BrokerStatsData;
import org.apache.rocketmq.common.protocol.body.BrokerStatsItem;
import org.apache.rocketmq.common.protocol.body.Connection;
+import org.apache.rocketmq.common.protocol.body.ConsumeQueueData;
import org.apache.rocketmq.common.protocol.body.ConsumeStatsList;
import org.apache.rocketmq.common.protocol.body.ConsumerConnection;
import org.apache.rocketmq.common.protocol.body.GroupList;
@@ -56,6 +60,7 @@ import org.apache.rocketmq.common.protocol.body.KVTable;
import org.apache.rocketmq.common.protocol.body.LockBatchRequestBody;
import org.apache.rocketmq.common.protocol.body.LockBatchResponseBody;
import org.apache.rocketmq.common.protocol.body.ProducerConnection;
+import org.apache.rocketmq.common.protocol.body.QueryConsumeQueueResponseBody;
import org.apache.rocketmq.common.protocol.body.QueryConsumeTimeSpanBody;
import org.apache.rocketmq.common.protocol.body.QueryCorrectionOffsetBody;
import org.apache.rocketmq.common.protocol.body.QueueTimeSpan;
@@ -81,6 +86,7 @@ import org.apache.rocketmq.common.protocol.header.GetMinOffsetRequestHeader;
import org.apache.rocketmq.common.protocol.header.GetMinOffsetResponseHeader;
import org.apache.rocketmq.common.protocol.header.GetProducerConnectionListRequestHeader;
import org.apache.rocketmq.common.protocol.header.GetTopicStatsInfoRequestHeader;
+import org.apache.rocketmq.common.protocol.header.QueryConsumeQueueRequestHeader;
import org.apache.rocketmq.common.protocol.header.QueryConsumeTimeSpanRequestHeader;
import org.apache.rocketmq.common.protocol.header.QueryCorrectionOffsetHeader;
import org.apache.rocketmq.common.protocol.header.QueryTopicConsumeByWhoRequestHeader;
@@ -94,6 +100,7 @@ import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
import org.apache.rocketmq.common.stats.StatsItem;
import org.apache.rocketmq.common.stats.StatsSnapshot;
import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig;
+import org.apache.rocketmq.filter.util.BitsArray;
import org.apache.rocketmq.remoting.common.RemotingHelper;
import org.apache.rocketmq.remoting.exception.RemotingCommandException;
import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
@@ -101,7 +108,10 @@ import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
import org.apache.rocketmq.remoting.protocol.LanguageCode;
import org.apache.rocketmq.remoting.protocol.RemotingCommand;
import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
+import org.apache.rocketmq.store.ConsumeQueue;
+import org.apache.rocketmq.store.ConsumeQueueExt;
import org.apache.rocketmq.store.DefaultMessageStore;
+import org.apache.rocketmq.store.MessageFilter;
import org.apache.rocketmq.store.SelectMappedBufferResult;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -187,6 +197,8 @@ public class AdminBrokerProcessor implements NettyRequestProcessor {
return ViewBrokerStatsData(ctx, request);
case RequestCode.GET_BROKER_CONSUME_STATS:
return fetchAllConsumeStatsInBroker(ctx, request);
+ case RequestCode.QUERY_CONSUME_QUEUE:
+ return queryConsumeQueue(ctx, request);
default:
break;
}
@@ -1244,4 +1256,83 @@ public class AdminBrokerProcessor implements NettyRequestProcessor {
}
}
+ private RemotingCommand queryConsumeQueue(ChannelHandlerContext ctx, RemotingCommand request) throws RemotingCommandException {
+ QueryConsumeQueueRequestHeader requestHeader =
+ (QueryConsumeQueueRequestHeader) request.decodeCommandCustomHeader(QueryConsumeQueueRequestHeader.class);
+
+ RemotingCommand response = RemotingCommand.createResponseCommand(null);
+
+ ConsumeQueue consumeQueue = this.brokerController.getMessageStore().getConsumeQueue(requestHeader.getTopic(),
+ requestHeader.getQueueId());
+ if (consumeQueue == null) {
+ response.setCode(ResponseCode.SYSTEM_ERROR);
+ response.setRemark(String.format("%d@%s is not exist!", requestHeader.getQueueId(), requestHeader.getTopic()));
+ return response;
+ }
+
+ QueryConsumeQueueResponseBody body = new QueryConsumeQueueResponseBody();
+ response.setCode(ResponseCode.SUCCESS);
+ response.setBody(body.encode());
+
+ body.setMaxQueueIndex(consumeQueue.getMaxOffsetInQueue());
+ body.setMinQueueIndex(consumeQueue.getMinOffsetInQueue());
+
+ MessageFilter messageFilter = null;
+ if (requestHeader.getConsumerGroup() != null) {
+ SubscriptionData subscriptionData = this.brokerController.getConsumerManager().findSubscriptionData(
+ requestHeader.getConsumerGroup(), requestHeader.getTopic()
+ );
+ body.setSubscriptionData(subscriptionData);
+ if (subscriptionData == null) {
+ body.setFilterData(String.format("%s@%s is not online!", requestHeader.getConsumerGroup(), requestHeader.getTopic()));
+ } else {
+ ConsumerFilterData filterData = this.brokerController.getConsumerFilterManager()
+ .get(requestHeader.getTopic(), requestHeader.getConsumerGroup());
+ body.setFilterData(JSON.toJSONString(filterData, true));
+
+ messageFilter = new ExpressionMessageFilter(subscriptionData, filterData,
+ this.brokerController.getConsumerFilterManager());
+ }
+ }
+
+ SelectMappedBufferResult result = consumeQueue.getIndexBuffer(requestHeader.getIndex());
+ if (result == null) {
+ response.setRemark(String.format("Index %d of %d@%s is not exist!", requestHeader.getIndex(), requestHeader.getQueueId(), requestHeader.getTopic()));
+ return response;
+ }
+ try {
+ List queues = new ArrayList<>();
+ for (int i = 0; i < result.getSize() && i < requestHeader.getCount() * ConsumeQueue.CQ_STORE_UNIT_SIZE; i += ConsumeQueue.CQ_STORE_UNIT_SIZE) {
+ ConsumeQueueData one = new ConsumeQueueData();
+ one.setPhysicOffset(result.getByteBuffer().getLong());
+ one.setPhysicSize(result.getByteBuffer().getInt());
+ one.setTagsCode(result.getByteBuffer().getLong());
+
+ if (!consumeQueue.isExtAddr(one.getTagsCode())) {
+ queues.add(one);
+ continue;
+ }
+
+ ConsumeQueueExt.CqExtUnit cqExtUnit = consumeQueue.getExt(one.getTagsCode());
+ if (cqExtUnit != null) {
+ one.setExtendDataJson(JSON.toJSONString(cqExtUnit));
+ if (cqExtUnit.getFilterBitMap() != null) {
+ one.setBitMap(BitsArray.create(cqExtUnit.getFilterBitMap()).toString());
+ }
+ if (messageFilter != null) {
+ one.setEval(messageFilter.isMatchedByConsumeQueue(cqExtUnit.getTagsCode(), cqExtUnit));
+ }
+ } else {
+ one.setMsg("Cq extend not exist!addr: " + one.getTagsCode());
+ }
+
+ queues.add(one);
+ }
+ body.setQueueData(queues);
+ } finally {
+ result.release();
+ }
+
+ return response;
+ }
}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java
index 6349ffc9eb5db78fbbf9245057ccac55d3bc4b25..67807a863dadb02432dab07abb1393623c454057 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java
@@ -22,15 +22,19 @@ import org.apache.rocketmq.broker.client.ClientChannelInfo;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.constant.LoggerName;
import org.apache.rocketmq.common.constant.PermName;
+import org.apache.rocketmq.common.filter.ExpressionType;
import org.apache.rocketmq.common.protocol.RequestCode;
import org.apache.rocketmq.common.protocol.ResponseCode;
+import org.apache.rocketmq.common.protocol.body.CheckClientRequestBody;
import org.apache.rocketmq.common.protocol.header.UnregisterClientRequestHeader;
import org.apache.rocketmq.common.protocol.header.UnregisterClientResponseHeader;
import org.apache.rocketmq.common.protocol.heartbeat.ConsumerData;
import org.apache.rocketmq.common.protocol.heartbeat.HeartbeatData;
import org.apache.rocketmq.common.protocol.heartbeat.ProducerData;
+import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig;
import org.apache.rocketmq.common.sysflag.TopicSysFlag;
+import org.apache.rocketmq.filter.FilterFactory;
import org.apache.rocketmq.remoting.common.RemotingHelper;
import org.apache.rocketmq.remoting.exception.RemotingCommandException;
import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
@@ -54,6 +58,8 @@ public class ClientManageProcessor implements NettyRequestProcessor {
return this.heartBeat(ctx, request);
case RequestCode.UNREGISTER_CLIENT:
return this.unregisterClient(ctx, request);
+ case RequestCode.CHECK_CLIENT_CONFIG:
+ return this.checkClientConfig(ctx, request);
default:
break;
}
@@ -157,4 +163,42 @@ public class ClientManageProcessor implements NettyRequestProcessor {
response.setRemark(null);
return response;
}
+
+ public RemotingCommand checkClientConfig(ChannelHandlerContext ctx, RemotingCommand request)
+ throws RemotingCommandException {
+ final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+
+ CheckClientRequestBody requestBody = CheckClientRequestBody.decode(request.getBody(),
+ CheckClientRequestBody.class);
+
+ if (requestBody != null && requestBody.getSubscriptionData() != null) {
+ SubscriptionData subscriptionData = requestBody.getSubscriptionData();
+
+ if (ExpressionType.isTagType(subscriptionData.getExpressionType())) {
+ response.setCode(ResponseCode.SUCCESS);
+ response.setRemark(null);
+ return response;
+ }
+
+ if (!this.brokerController.getBrokerConfig().isEnablePropertyFilter()) {
+ response.setCode(ResponseCode.SYSTEM_ERROR);
+ response.setRemark("The broker does not support consumer to filter message by " + subscriptionData.getExpressionType());
+ return response;
+ }
+
+ try {
+ FilterFactory.INSTANCE.get(subscriptionData.getExpressionType()).compile(subscriptionData.getSubString());
+ } catch (Exception e) {
+ log.warn("Client {}@{} filter message, but failed to compile expression! sub={}, error={}",
+ requestBody.getClientId(), requestBody.getGroup(), requestBody.getSubscriptionData(), e.getMessage());
+ response.setCode(ResponseCode.SUBSCRIPTION_PARSE_FAILED);
+ response.setRemark(e.getMessage());
+ return response;
+ }
+ }
+
+ response.setCode(ResponseCode.SUCCESS);
+ response.setRemark(null);
+ return response;
+ }
}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java
index 89967d8f6ae2a98e05b19f05e304fc41e3b6f4d4..10945da97c986c01e61507e501198b3a0583720d 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java
@@ -25,6 +25,10 @@ import java.nio.ByteBuffer;
import java.util.List;
import org.apache.rocketmq.broker.BrokerController;
import org.apache.rocketmq.broker.client.ConsumerGroupInfo;
+import org.apache.rocketmq.broker.filter.ConsumerFilterData;
+import org.apache.rocketmq.broker.filter.ConsumerFilterManager;
+import org.apache.rocketmq.broker.filter.ExpressionForRetryMessageFilter;
+import org.apache.rocketmq.broker.filter.ExpressionMessageFilter;
import org.apache.rocketmq.broker.longpolling.PullRequest;
import org.apache.rocketmq.broker.mqtrace.ConsumeMessageContext;
import org.apache.rocketmq.broker.mqtrace.ConsumeMessageHook;
@@ -34,6 +38,7 @@ import org.apache.rocketmq.common.TopicConfig;
import org.apache.rocketmq.common.TopicFilterType;
import org.apache.rocketmq.common.constant.LoggerName;
import org.apache.rocketmq.common.constant.PermName;
+import org.apache.rocketmq.common.filter.ExpressionType;
import org.apache.rocketmq.common.filter.FilterAPI;
import org.apache.rocketmq.common.help.FAQUrl;
import org.apache.rocketmq.common.message.MessageDecoder;
@@ -54,6 +59,7 @@ import org.apache.rocketmq.remoting.netty.RequestTask;
import org.apache.rocketmq.remoting.protocol.RemotingCommand;
import org.apache.rocketmq.store.GetMessageResult;
import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.store.MessageFilter;
import org.apache.rocketmq.store.PutMessageResult;
import org.apache.rocketmq.store.config.BrokerRole;
import org.apache.rocketmq.store.stats.BrokerStatsManager;
@@ -142,13 +148,22 @@ public class PullMessageProcessor implements NettyRequestProcessor {
}
SubscriptionData subscriptionData = null;
+ ConsumerFilterData consumerFilterData = null;
if (hasSubscriptionFlag) {
try {
- subscriptionData = FilterAPI.buildSubscriptionData(requestHeader.getConsumerGroup(), requestHeader.getTopic(),
- requestHeader.getSubscription());
+ subscriptionData = FilterAPI.build(
+ requestHeader.getTopic(), requestHeader.getSubscription(), requestHeader.getExpressionType()
+ );
+ if (!ExpressionType.isTagType(subscriptionData.getExpressionType())) {
+ consumerFilterData = ConsumerFilterManager.build(
+ requestHeader.getTopic(), requestHeader.getConsumerGroup(), requestHeader.getSubscription(),
+ requestHeader.getExpressionType(), requestHeader.getSubVersion()
+ );
+ assert consumerFilterData != null;
+ }
} catch (Exception e) {
LOG.warn("Parse the consumer's subscription[{}] failed, group: {}", requestHeader.getSubscription(), //
- requestHeader.getConsumerGroup());
+ requestHeader.getConsumerGroup());
response.setCode(ResponseCode.SUBSCRIPTION_PARSE_FAILED);
response.setRemark("parse the consumer's subscription failed");
return response;
@@ -180,16 +195,48 @@ public class PullMessageProcessor implements NettyRequestProcessor {
if (subscriptionData.getSubVersion() < requestHeader.getSubVersion()) {
LOG.warn("The broker's subscription is not latest, group: {} {}", requestHeader.getConsumerGroup(),
- subscriptionData.getSubString());
+ subscriptionData.getSubString());
response.setCode(ResponseCode.SUBSCRIPTION_NOT_LATEST);
response.setRemark("the consumer's subscription not latest");
return response;
}
+ if (!ExpressionType.isTagType(subscriptionData.getExpressionType())) {
+ consumerFilterData = this.brokerController.getConsumerFilterManager().get(requestHeader.getTopic(),
+ requestHeader.getConsumerGroup());
+ if (consumerFilterData == null) {
+ response.setCode(ResponseCode.FILTER_DATA_NOT_EXIST);
+ response.setRemark("The broker's consumer filter data is not exist!Your expression may be wrong!");
+ return response;
+ }
+ if (consumerFilterData.getClientVersion() < requestHeader.getSubVersion()) {
+ LOG.warn("The broker's consumer filter data is not latest, group: {}, topic: {}, serverV: {}, clientV: {}",
+ requestHeader.getConsumerGroup(), requestHeader.getTopic(), consumerFilterData.getClientVersion(), requestHeader.getSubVersion());
+ response.setCode(ResponseCode.FILTER_DATA_NOT_LATEST);
+ response.setRemark("the consumer's consumer filter data not latest");
+ return response;
+ }
+ }
+ }
+
+ if (!ExpressionType.isTagType(subscriptionData.getExpressionType())
+ && !this.brokerController.getBrokerConfig().isEnablePropertyFilter()) {
+ response.setCode(ResponseCode.SYSTEM_ERROR);
+ response.setRemark("The broker does not support consumer to filter message by " + subscriptionData.getExpressionType());
+ return response;
+ }
+
+ MessageFilter messageFilter;
+ if (this.brokerController.getBrokerConfig().isFilterSupportRetry()) {
+ messageFilter = new ExpressionForRetryMessageFilter(subscriptionData, consumerFilterData,
+ this.brokerController.getConsumerFilterManager());
+ } else {
+ messageFilter = new ExpressionMessageFilter(subscriptionData, consumerFilterData,
+ this.brokerController.getConsumerFilterManager());
}
final GetMessageResult getMessageResult =
this.brokerController.getMessageStore().getMessage(requestHeader.getConsumerGroup(), requestHeader.getTopic(),
- requestHeader.getQueueId(), requestHeader.getQueueOffset(), requestHeader.getMaxMsgNums(), subscriptionData);
+ requestHeader.getQueueId(), requestHeader.getQueueOffset(), requestHeader.getMaxMsgNums(), messageFilter);
if (getMessageResult != null) {
response.setRemark(getMessageResult.getStatus().name());
responseHeader.setNextBeginOffset(getMessageResult.getNextBeginOffset());
@@ -368,7 +415,7 @@ public class PullMessageProcessor implements NettyRequestProcessor {
long offset = requestHeader.getQueueOffset();
int queueId = requestHeader.getQueueId();
PullRequest pullRequest = new PullRequest(request, channel, pollingTimeMills,
- this.brokerController.getMessageStore().now(), offset, subscriptionData);
+ this.brokerController.getMessageStore().now(), offset, subscriptionData, messageFilter);
this.brokerController.getPullRequestHoldService().suspendPullRequest(topic, queueId, pullRequest);
response = null;
break;
diff --git a/broker/src/test/java/org/apache/rocketmq/broker/filter/CommitLogDispatcherCalcBitMapTest.java b/broker/src/test/java/org/apache/rocketmq/broker/filter/CommitLogDispatcherCalcBitMapTest.java
new file mode 100644
index 0000000000000000000000000000000000000000..87f625616cf52987e93931e6ff82f8cec67ee4ea
--- /dev/null
+++ b/broker/src/test/java/org/apache/rocketmq/broker/filter/CommitLogDispatcherCalcBitMapTest.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.broker.filter;
+
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.filter.ExpressionType;
+import org.apache.rocketmq.filter.util.BitsArray;
+import org.apache.rocketmq.store.DispatchRequest;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class CommitLogDispatcherCalcBitMapTest {
+
+ @Test
+ public void testDispatch_filterDataIllegal() {
+ BrokerConfig brokerConfig = new BrokerConfig();
+ brokerConfig.setEnableCalcFilterBitMap(true);
+
+ ConsumerFilterManager filterManager = new ConsumerFilterManager();
+
+ filterManager.register("topic0", "CID_0", "a is not null and a >= 5",
+ ExpressionType.SQL92, System.currentTimeMillis());
+
+ filterManager.register("topic0", "CID_1", "a is not null and a >= 15",
+ ExpressionType.SQL92, System.currentTimeMillis());
+
+ ConsumerFilterData nullExpression = filterManager.get("topic0", "CID_0");
+ nullExpression.setExpression(null);
+ nullExpression.setCompiledExpression(null);
+ ConsumerFilterData nullBloomData = filterManager.get("topic0", "CID_1");
+ nullBloomData.setBloomFilterData(null);
+
+
+ CommitLogDispatcherCalcBitMap calcBitMap = new CommitLogDispatcherCalcBitMap(brokerConfig,
+ filterManager);
+
+ for (int i = 0; i < 1; i++) {
+ Map properties = new HashMap(4);
+ properties.put("a", String.valueOf(i * 10 + 5));
+
+ String topic = "topic" + i;
+
+ DispatchRequest dispatchRequest = new DispatchRequest(
+ topic,
+ 0,
+ i * 100 + 123,
+ 100,
+ (long) ("tags" + i).hashCode(),
+ System.currentTimeMillis(),
+ i,
+ null,
+ UUID.randomUUID().toString(),
+ 0,
+ 0,
+ properties
+ );
+
+ calcBitMap.dispatch(dispatchRequest);
+
+ assertThat(dispatchRequest.getBitMap()).isNotNull();
+
+ BitsArray bitsArray = BitsArray.create(dispatchRequest.getBitMap(),
+ filterManager.getBloomFilter().getM());
+
+ for (int j = 0; j < bitsArray.bitLength(); j++) {
+ assertThat(bitsArray.getBit(j)).isFalse();
+ }
+ }
+ }
+
+ @Test
+ public void testDispatch_blankFilterData() {
+ BrokerConfig brokerConfig = new BrokerConfig();
+ brokerConfig.setEnableCalcFilterBitMap(true);
+
+ ConsumerFilterManager filterManager = new ConsumerFilterManager();
+
+ CommitLogDispatcherCalcBitMap calcBitMap = new CommitLogDispatcherCalcBitMap(brokerConfig,
+ filterManager);
+
+ for (int i = 0; i < 10; i++) {
+ Map properties = new HashMap(4);
+ properties.put("a", String.valueOf(i * 10 + 5));
+
+ String topic = "topic" + i;
+
+ DispatchRequest dispatchRequest = new DispatchRequest(
+ topic,
+ 0,
+ i * 100 + 123,
+ 100,
+ (long) ("tags" + i).hashCode(),
+ System.currentTimeMillis(),
+ i,
+ null,
+ UUID.randomUUID().toString(),
+ 0,
+ 0,
+ properties
+ );
+
+ calcBitMap.dispatch(dispatchRequest);
+
+ assertThat(dispatchRequest.getBitMap()).isNull();
+ }
+ }
+
+ @Test
+ public void testDispatch() {
+ BrokerConfig brokerConfig = new BrokerConfig();
+ brokerConfig.setEnableCalcFilterBitMap(true);
+
+ ConsumerFilterManager filterManager = ConsumerFilterManagerTest.gen(10, 10);
+
+ CommitLogDispatcherCalcBitMap calcBitMap = new CommitLogDispatcherCalcBitMap(brokerConfig,
+ filterManager);
+
+ for (int i = 0; i < 10; i++) {
+ Map properties = new HashMap(4);
+ properties.put("a", String.valueOf(i * 10 + 5));
+
+ String topic = "topic" + i;
+
+ DispatchRequest dispatchRequest = new DispatchRequest(
+ topic,
+ 0,
+ i * 100 + 123,
+ 100,
+ (long) ("tags" + i).hashCode(),
+ System.currentTimeMillis(),
+ i,
+ null,
+ UUID.randomUUID().toString(),
+ 0,
+ 0,
+ properties
+ );
+
+ calcBitMap.dispatch(dispatchRequest);
+
+ assertThat(dispatchRequest.getBitMap()).isNotNull();
+
+ BitsArray bits = BitsArray.create(dispatchRequest.getBitMap());
+
+ Collection filterDatas = filterManager.get(topic);
+
+ for (ConsumerFilterData filterData : filterDatas) {
+
+ if (filterManager.getBloomFilter().isHit(filterData.getBloomFilterData(), bits)) {
+ try {
+ assertThat((Boolean) filterData.getCompiledExpression().evaluate(
+ new MessageEvaluationContext(properties)
+ )).isTrue();
+ } catch (Exception e) {
+ e.printStackTrace();
+ assertThat(true).isFalse();
+ }
+ } else {
+ try {
+ assertThat((Boolean) filterData.getCompiledExpression().evaluate(
+ new MessageEvaluationContext(properties)
+ )).isFalse();
+ } catch (Exception e) {
+ e.printStackTrace();
+ assertThat(true).isFalse();
+ }
+ }
+ }
+ }
+ }
+}
diff --git a/broker/src/test/java/org/apache/rocketmq/broker/filter/ConsumerFilterManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/filter/ConsumerFilterManagerTest.java
new file mode 100644
index 0000000000000000000000000000000000000000..c8412a8bff8911f9a0ff6b4940dbebfdb248e65b
--- /dev/null
+++ b/broker/src/test/java/org/apache/rocketmq/broker/filter/ConsumerFilterManagerTest.java
@@ -0,0 +1,291 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.broker.filter;
+
+import org.apache.rocketmq.common.filter.ExpressionType;
+import org.apache.rocketmq.common.filter.FilterAPI;
+import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class ConsumerFilterManagerTest {
+
+ public static ConsumerFilterManager gen(int topicCount, int consumerCount) {
+ ConsumerFilterManager filterManager = new ConsumerFilterManager();
+
+ for (int i = 0; i < topicCount; i++) {
+ String topic = "topic" + i;
+
+ for (int j = 0; j < consumerCount; j++) {
+
+ String consumer = "CID_" + j;
+
+ filterManager.register(topic, consumer, expr(j), ExpressionType.SQL92, System.currentTimeMillis());
+ }
+ }
+
+ return filterManager;
+ }
+
+ public static String expr(int i) {
+ return "a is not null and a > " + ((i - 1) * 10) + " and a < " + ((i + 1) * 10);
+ }
+
+ @Test
+ public void testRegister_newExpressionCompileErrorAndRemoveOld() {
+ ConsumerFilterManager filterManager = gen(10, 10);
+
+ assertThat(filterManager.get("topic9", "CID_9")).isNotNull();
+
+ String newExpr = "a between 10,20";
+
+ assertThat(filterManager.register("topic9", "CID_9", newExpr, ExpressionType.SQL92, System.currentTimeMillis() + 1))
+ .isFalse();
+ assertThat(filterManager.get("topic9", "CID_9")).isNull();
+
+ newExpr = "a between 10 AND 20";
+
+ assertThat(filterManager.register("topic9", "CID_9", newExpr, ExpressionType.SQL92, System.currentTimeMillis() + 1))
+ .isTrue();
+
+ ConsumerFilterData filterData = filterManager.get("topic9", "CID_9");
+
+ assertThat(filterData).isNotNull();
+ assertThat(newExpr).isEqualTo(filterData.getExpression());
+ }
+
+ @Test
+ public void testRegister_change() {
+ ConsumerFilterManager filterManager = gen(10, 10);
+
+ ConsumerFilterData filterData = filterManager.get("topic9", "CID_9");
+
+ System.out.println(filterData.getCompiledExpression());
+
+ String newExpr = "a > 0 and a < 10";
+
+ filterManager.register("topic9", "CID_9", newExpr, ExpressionType.SQL92, System.currentTimeMillis() + 1);
+
+ filterData = filterManager.get("topic9", "CID_9");
+
+ assertThat(newExpr).isEqualTo(filterData.getExpression());
+
+ System.out.println(filterData.toString());
+ }
+
+ @Test
+ public void testRegister() {
+ ConsumerFilterManager filterManager = gen(10, 10);
+
+ ConsumerFilterData filterData = filterManager.get("topic9", "CID_9");
+
+ assertThat(filterData).isNotNull();
+ assertThat(filterData.isDead()).isFalse();
+
+ // new version
+ assertThat(filterManager.register(
+ "topic9", "CID_9", "a is not null", ExpressionType.SQL92, System.currentTimeMillis() + 1000
+ )).isTrue();
+
+ ConsumerFilterData newFilter = filterManager.get("topic9", "CID_9");
+
+ assertThat(newFilter).isNotEqualTo(filterData);
+
+ // same version
+ assertThat(filterManager.register(
+ "topic9", "CID_9", "a is null", ExpressionType.SQL92, newFilter.getClientVersion()
+ )).isFalse();
+
+ ConsumerFilterData filterData1 = filterManager.get("topic9", "CID_9");
+
+ assertThat(newFilter).isEqualTo(filterData1);
+ }
+
+ @Test
+ public void testRegister_reAlive() {
+ ConsumerFilterManager filterManager = gen(10, 10);
+
+ ConsumerFilterData filterData = filterManager.get("topic9", "CID_9");
+
+ assertThat(filterData).isNotNull();
+ assertThat(filterData.isDead()).isFalse();
+
+ //make dead
+ filterManager.unRegister("CID_9");
+
+ //reAlive
+ filterManager.register(
+ filterData.getTopic(),
+ filterData.getConsumerGroup(),
+ filterData.getExpression(),
+ filterData.getExpressionType(),
+ System.currentTimeMillis()
+ );
+
+ ConsumerFilterData newFilterData = filterManager.get("topic9", "CID_9");
+
+ assertThat(newFilterData).isNotNull();
+ assertThat(newFilterData.isDead()).isFalse();
+ }
+
+ @Test
+ public void testRegister_bySubscriptionData() {
+ ConsumerFilterManager filterManager = new ConsumerFilterManager();
+ List subscriptionDatas = new ArrayList<>();
+ for (int i = 0; i < 10; i++) {
+ try {
+ subscriptionDatas.add(
+ FilterAPI.build(
+ "topic" + i,
+ "a is not null and a > " + i,
+ ExpressionType.SQL92
+ )
+ );
+ } catch (Exception e) {
+ e.printStackTrace();
+ assertThat(true).isFalse();
+ }
+ }
+
+ filterManager.register("CID_0", subscriptionDatas);
+
+ Collection filterDatas = filterManager.getByGroup("CID_0");
+
+ assertThat(filterDatas).isNotNull();
+ assertThat(filterDatas.size()).isEqualTo(10);
+
+ Iterator iterator = filterDatas.iterator();
+ while (iterator.hasNext()) {
+ ConsumerFilterData filterData = iterator.next();
+
+ assertThat(filterData).isNotNull();
+ assertThat(filterManager.getBloomFilter().isValid(filterData.getBloomFilterData())).isTrue();
+ }
+ }
+
+ @Test
+ public void testRegister_tag() {
+ ConsumerFilterManager filterManager = new ConsumerFilterManager();
+
+ assertThat(filterManager.register("topic0", "CID_0", "*", null, System.currentTimeMillis())).isFalse();
+
+ Collection filterDatas = filterManager.getByGroup("CID_0");
+
+ assertThat(filterDatas).isNullOrEmpty();
+ }
+
+ @Test
+ public void testUnregister() {
+ ConsumerFilterManager filterManager = gen(10, 10);
+
+ ConsumerFilterData filterData = filterManager.get("topic9", "CID_9");
+
+ assertThat(filterData).isNotNull();
+ assertThat(filterData.isDead()).isFalse();
+
+ filterManager.unRegister("CID_9");
+
+ assertThat(filterData.isDead()).isTrue();
+ }
+
+ @Test
+ public void testPersist() {
+ ConsumerFilterManager filterManager = gen(10, 10);
+
+ try {
+ filterManager.persist();
+
+ ConsumerFilterData filterData = filterManager.get("topic9", "CID_9");
+
+ assertThat(filterData).isNotNull();
+ assertThat(filterData.isDead()).isFalse();
+
+ ConsumerFilterManager loadFilter = new ConsumerFilterManager();
+
+ assertThat(loadFilter.load()).isTrue();
+
+ filterData = loadFilter.get("topic9", "CID_9");
+
+ assertThat(filterData).isNotNull();
+ assertThat(filterData.isDead()).isTrue();
+ assertThat(filterData.getCompiledExpression()).isNotNull();
+ } finally {
+ deleteDirectory("./unit_test");
+ }
+ }
+
+ @Test
+ public void testPersist_clean() {
+ ConsumerFilterManager filterManager = gen(10, 10);
+
+ String topic = "topic9";
+ for (int i = 0; i < 10; i++) {
+ String cid = "CID_" + i;
+
+ ConsumerFilterData filterData = filterManager.get(topic, cid);
+
+ assertThat(filterData).isNotNull();
+ assertThat(filterData.isDead()).isFalse();
+
+ //make dead more than 24h
+ filterData.setBornTime(System.currentTimeMillis() - 26 * 60 * 60 * 1000);
+ filterData.setDeadTime(System.currentTimeMillis() - 25 * 60 * 60 * 1000);
+ }
+
+ try {
+ filterManager.persist();
+
+ ConsumerFilterManager loadFilter = new ConsumerFilterManager();
+
+ assertThat(loadFilter.load()).isTrue();
+
+ ConsumerFilterData filterData = loadFilter.get(topic, "CID_9");
+
+ assertThat(filterData).isNull();
+
+ Collection topicData = loadFilter.get(topic);
+
+ assertThat(topicData).isNullOrEmpty();
+ } finally {
+ deleteDirectory("./unit_test");
+ }
+ }
+
+ protected void deleteDirectory(String rootPath) {
+ File file = new File(rootPath);
+ deleteFile(file);
+ }
+
+ protected void deleteFile(File file) {
+ File[] subFiles = file.listFiles();
+ if (subFiles != null) {
+ for (File sub : subFiles) {
+ deleteFile(sub);
+ }
+ }
+
+ file.delete();
+ }
+}
diff --git a/broker/src/test/java/org/apache/rocketmq/broker/filter/MessageStoreWithFilterTest.java b/broker/src/test/java/org/apache/rocketmq/broker/filter/MessageStoreWithFilterTest.java
new file mode 100644
index 0000000000000000000000000000000000000000..53e563edc2fcc40a2160be1d8b1d7739ca91dab5
--- /dev/null
+++ b/broker/src/test/java/org/apache/rocketmq/broker/filter/MessageStoreWithFilterTest.java
@@ -0,0 +1,392 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.broker.filter;
+
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.filter.ExpressionType;
+import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
+import org.apache.rocketmq.store.CommitLogDispatcher;
+import org.apache.rocketmq.store.DefaultMessageStore;
+import org.apache.rocketmq.store.DispatchRequest;
+import org.apache.rocketmq.store.GetMessageResult;
+import org.apache.rocketmq.store.GetMessageStatus;
+import org.apache.rocketmq.store.MessageArrivingListener;
+import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.store.PutMessageResult;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+import org.apache.rocketmq.store.stats.BrokerStatsManager;
+import org.junit.Test;
+
+import java.io.File;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class MessageStoreWithFilterTest {
+
+ private static final String msg = "Once, there was a chance for me!";
+ private static final byte[] msgBody = msg.getBytes();
+
+ private static final String topic = "topic";
+ private static final int queueId = 0;
+ private static final String storePath = "." + File.separator + "unit_test_store";
+ private static final int commitLogFileSize = 1024 * 1024 * 256;
+ private static final int cqFileSize = 300000 * 20;
+ private static final int cqExtFileSize = 300000 * 128;
+
+ private static SocketAddress BornHost;
+
+ private static SocketAddress StoreHost;
+
+ static {
+ try {
+ StoreHost = new InetSocketAddress(InetAddress.getLocalHost(), 8123);
+ } catch (UnknownHostException e) {
+ e.printStackTrace();
+ }
+ try {
+ BornHost = new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 0);
+ } catch (UnknownHostException e) {
+ e.printStackTrace();
+ }
+ }
+
+ public MessageExtBrokerInner buildMessage() {
+ MessageExtBrokerInner msg = new MessageExtBrokerInner();
+ msg.setTopic(topic);
+ msg.setTags("TAG1");
+ msg.setKeys("Hello");
+ msg.setBody(msgBody);
+ msg.setKeys(String.valueOf(System.currentTimeMillis()));
+ msg.setQueueId(queueId);
+ msg.setSysFlag(0);
+ msg.setBornTimestamp(System.currentTimeMillis());
+ msg.setStoreHost(StoreHost);
+ msg.setBornHost(BornHost);
+ for (int i = 1; i < 3; i++) {
+ msg.putUserProperty(String.valueOf(i), "imagoodperson" + i);
+ }
+ msg.setPropertiesString(MessageDecoder.messageProperties2String(msg.getProperties()));
+
+ return msg;
+ }
+
+ public MessageStoreConfig buildStoreConfig(int commitLogFileSize, int cqFileSize,
+ boolean enableCqExt, int cqExtFileSize) {
+ MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
+ messageStoreConfig.setMapedFileSizeCommitLog(commitLogFileSize);
+ messageStoreConfig.setMapedFileSizeConsumeQueue(cqFileSize);
+ messageStoreConfig.setMappedFileSizeConsumeQueueExt(cqExtFileSize);
+ messageStoreConfig.setMessageIndexEnable(false);
+ messageStoreConfig.setEnableConsumeQueueExt(enableCqExt);
+
+ messageStoreConfig.setStorePathRootDir(storePath);
+ messageStoreConfig.setStorePathCommitLog(storePath + File.separator + "commitlog");
+
+ return messageStoreConfig;
+ }
+
+ protected DefaultMessageStore gen(ConsumerFilterManager filterManager) throws Exception {
+ MessageStoreConfig messageStoreConfig = buildStoreConfig(
+ commitLogFileSize, cqFileSize, true, cqExtFileSize
+ );
+
+ BrokerConfig brokerConfig = new BrokerConfig();
+ brokerConfig.setEnableCalcFilterBitMap(true);
+ brokerConfig.setMaxErrorRateOfBloomFilter(20);
+ brokerConfig.setExpectConsumerNumUseFilter(64);
+
+ DefaultMessageStore master = new DefaultMessageStore(
+ messageStoreConfig,
+ new BrokerStatsManager(brokerConfig.getBrokerClusterName()),
+ new MessageArrivingListener() {
+ @Override
+ public void arriving(String topic, int queueId, long logicOffset, long tagsCode,
+ long msgStoreTime, byte[] filterBitMap, Map properties) {
+// System.out.println(String.format("Msg coming: %s, %d, %d, %d",
+// topic, queueId, logicOffset, tagsCode));
+ }
+ }
+ , brokerConfig);
+
+ master.getDispatcherList().addFirst(new CommitLogDispatcher() {
+ @Override
+ public void dispatch(DispatchRequest request) {
+ try {
+// System.out.println(String.format("offset:%d, bitMap:%s", request.getCommitLogOffset(),
+// BitsArray.create(request.getBitMap()).toString()));
+ } catch (Throwable e) {
+ e.printStackTrace();
+ }
+ }
+ });
+ master.getDispatcherList().addFirst(new CommitLogDispatcherCalcBitMap(brokerConfig, filterManager));
+
+ assertThat(master.load()).isTrue();
+
+ master.start();
+
+ return master;
+ }
+
+ protected List putMsg(DefaultMessageStore master, int topicCount, int msgCountPerTopic) throws Exception {
+ List msgs = new ArrayList();
+ for (int i = 0; i < topicCount; i++) {
+ String realTopic = topic + i;
+ for (int j = 0; j < msgCountPerTopic; j++) {
+ MessageExtBrokerInner msg = buildMessage();
+ msg.setTopic(realTopic);
+ msg.putUserProperty("a", String.valueOf(j * 10 + 5));
+ msg.setPropertiesString(MessageDecoder.messageProperties2String(msg.getProperties()));
+
+ PutMessageResult result = master.putMessage(msg);
+
+ msg.setMsgId(result.getAppendMessageResult().getMsgId());
+
+ msgs.add(msg);
+ }
+ }
+
+ return msgs;
+ }
+
+ protected void deleteDirectory(String rootPath) {
+ File file = new File(rootPath);
+ deleteFile(file);
+ }
+
+ protected void deleteFile(File file) {
+ File[] subFiles = file.listFiles();
+ if (subFiles != null) {
+ for (File sub : subFiles) {
+ deleteFile(sub);
+ }
+ }
+
+ file.delete();
+ }
+
+ protected List filtered(List msgs, ConsumerFilterData filterData) {
+ List filteredMsgs = new ArrayList();
+
+ for (MessageExtBrokerInner messageExtBrokerInner : msgs) {
+
+ if (!messageExtBrokerInner.getTopic().equals(filterData.getTopic())) {
+ continue;
+ }
+
+ try {
+ Object evlRet = filterData.getCompiledExpression().evaluate(new MessageEvaluationContext(messageExtBrokerInner.getProperties()));
+
+ if (evlRet == null || !(evlRet instanceof Boolean) || (Boolean) evlRet) {
+ filteredMsgs.add(messageExtBrokerInner);
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ assertThat(true).isFalse();
+ }
+ }
+
+ return filteredMsgs;
+ }
+
+ @Test
+ public void testGetMessage_withFilterBitMapAndConsumerChanged() {
+ int topicCount = 10, msgPerTopic = 10;
+ ConsumerFilterManager filterManager = ConsumerFilterManagerTest.gen(topicCount, msgPerTopic);
+
+ DefaultMessageStore master = null;
+ try {
+ master = gen(filterManager);
+ } catch (Exception e) {
+ e.printStackTrace();
+ assertThat(true).isFalse();
+ }
+
+ try {
+ List msgs = null;
+ try {
+ msgs = putMsg(master, topicCount, msgPerTopic);
+ } catch (Exception e) {
+ e.printStackTrace();
+ assertThat(true).isFalse();
+ }
+
+ // sleep to wait for consume queue has been constructed.
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ assertThat(true).isFalse();
+ }
+
+ // reset consumer;
+ String topic = "topic" + 0;
+ String resetGroup = "CID_" + 2;
+ String normalGroup = "CID_" + 3;
+
+ {
+ // reset CID_2@topic0 to get all messages.
+ SubscriptionData resetSubData = new SubscriptionData();
+ resetSubData.setExpressionType(ExpressionType.SQL92);
+ resetSubData.setTopic(topic);
+ resetSubData.setClassFilterMode(false);
+ resetSubData.setSubString("a is not null OR a is null");
+
+ ConsumerFilterData resetFilterData = ConsumerFilterManager.build(topic,
+ resetGroup, resetSubData.getSubString(), resetSubData.getExpressionType(),
+ System.currentTimeMillis());
+
+ GetMessageResult resetGetResult = master.getMessage(resetGroup, topic, queueId, 0, 1000,
+ new ExpressionMessageFilter(resetSubData, resetFilterData, filterManager));
+
+ try {
+ assertThat(resetGetResult).isNotNull();
+
+ List filteredMsgs = filtered(msgs, resetFilterData);
+
+ assertThat(resetGetResult.getMessageBufferList().size()).isEqualTo(filteredMsgs.size());
+ } finally {
+ resetGetResult.release();
+ }
+ }
+
+ {
+ ConsumerFilterData normalFilterData = filterManager.get(topic, normalGroup);
+ assertThat(normalFilterData).isNotNull();
+ assertThat(normalFilterData.getBornTime()).isLessThan(System.currentTimeMillis());
+
+ SubscriptionData normalSubData = new SubscriptionData();
+ normalSubData.setExpressionType(normalFilterData.getExpressionType());
+ normalSubData.setTopic(topic);
+ normalSubData.setClassFilterMode(false);
+ normalSubData.setSubString(normalFilterData.getExpression());
+
+ List filteredMsgs = filtered(msgs, normalFilterData);
+
+ GetMessageResult normalGetResult = master.getMessage(normalGroup, topic, queueId, 0, 1000,
+ new ExpressionMessageFilter(normalSubData, normalFilterData, filterManager));
+
+ try {
+ assertThat(normalGetResult).isNotNull();
+ assertThat(normalGetResult.getMessageBufferList().size()).isEqualTo(filteredMsgs.size());
+ } finally {
+ normalGetResult.release();
+ }
+ }
+ } finally {
+ master.shutdown();
+ master.destroy();
+ deleteDirectory(storePath);
+ }
+ }
+
+ @Test
+ public void testGetMessage_withFilterBitMap() {
+ int topicCount = 10, msgPerTopic = 500;
+ ConsumerFilterManager filterManager = ConsumerFilterManagerTest.gen(topicCount, msgPerTopic);
+
+ DefaultMessageStore master = null;
+ try {
+ master = gen(filterManager);
+ } catch (Exception e) {
+ e.printStackTrace();
+ assertThat(true).isFalse();
+ }
+
+ try {
+ List msgs = null;
+ try {
+ msgs = putMsg(master, topicCount, msgPerTopic);
+ // sleep to wait for consume queue has been constructed.
+ Thread.sleep(1000);
+ } catch (Exception e) {
+ e.printStackTrace();
+ assertThat(true).isFalse();
+ }
+
+ for (int i = 0; i < topicCount; i++) {
+ String realTopic = topic + i;
+
+ for (int j = 0; j < msgPerTopic; j++) {
+ String group = "CID_" + j;
+
+ ConsumerFilterData filterData = filterManager.get(realTopic, group);
+ assertThat(filterData).isNotNull();
+
+ List filteredMsgs = filtered(msgs, filterData);
+
+ SubscriptionData subscriptionData = new SubscriptionData();
+ subscriptionData.setExpressionType(filterData.getExpressionType());
+ subscriptionData.setTopic(filterData.getTopic());
+ subscriptionData.setClassFilterMode(false);
+ subscriptionData.setSubString(filterData.getExpression());
+
+ GetMessageResult getMessageResult = master.getMessage(group, realTopic, queueId, 0, 10000,
+ new ExpressionMessageFilter(subscriptionData, filterData, filterManager));
+ String assertMsg = group + "-" + realTopic;
+ try {
+ assertThat(getMessageResult).isNotNull();
+ assertThat(GetMessageStatus.FOUND).isEqualTo(getMessageResult.getStatus());
+ assertThat(getMessageResult.getMessageBufferList()).isNotNull().isNotEmpty();
+ assertThat(getMessageResult.getMessageBufferList().size()).isEqualTo(filteredMsgs.size());
+
+ for (ByteBuffer buffer : getMessageResult.getMessageBufferList()) {
+ MessageExt messageExt = MessageDecoder.decode(buffer.slice(), false);
+ assertThat(messageExt).isNotNull();
+
+ Object evlRet = null;
+ try {
+ evlRet = filterData.getCompiledExpression().evaluate(new MessageEvaluationContext(messageExt.getProperties()));
+ } catch (Exception e) {
+ e.printStackTrace();
+ assertThat(true).isFalse();
+ }
+
+ assertThat(evlRet).isNotNull().isEqualTo(Boolean.TRUE);
+
+ // check
+ boolean find = false;
+ for (MessageExtBrokerInner messageExtBrokerInner : filteredMsgs) {
+ if (messageExtBrokerInner.getMsgId().equals(messageExt.getMsgId())) {
+ find = true;
+ }
+ }
+ assertThat(find).isTrue();
+ }
+ } finally {
+ getMessageResult.release();
+ }
+ }
+ }
+ } finally {
+ master.shutdown();
+ master.destroy();
+ deleteDirectory(storePath);
+ }
+ }
+}
diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/PullMessageProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/PullMessageProcessorTest.java
index d3d98120547ffc7c67e2b236dee5b2a2022bae02..941d4a77feaaba6687840db3a4e59612f7409905 100644
--- a/broker/src/test/java/org/apache/rocketmq/broker/processor/PullMessageProcessorTest.java
+++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/PullMessageProcessorTest.java
@@ -25,6 +25,7 @@ import java.util.List;
import java.util.Set;
import org.apache.rocketmq.broker.BrokerController;
import org.apache.rocketmq.broker.client.ClientChannelInfo;
+import org.apache.rocketmq.broker.filter.ExpressionMessageFilter;
import org.apache.rocketmq.broker.mqtrace.ConsumeMessageContext;
import org.apache.rocketmq.broker.mqtrace.ConsumeMessageHook;
import org.apache.rocketmq.common.BrokerConfig;
@@ -126,7 +127,7 @@ public class PullMessageProcessorTest {
@Test
public void testProcessRequest_Found() throws RemotingCommandException {
GetMessageResult getMessageResult = createGetMessageResult();
- when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(SubscriptionData.class))).thenReturn(getMessageResult);
+ when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(ExpressionMessageFilter.class))).thenReturn(getMessageResult);
final RemotingCommand request = createPullMsgCommand(RequestCode.PULL_MESSAGE);
RemotingCommand response = pullMessageProcessor.processRequest(handlerContext, request);
@@ -137,7 +138,7 @@ public class PullMessageProcessorTest {
@Test
public void testProcessRequest_FoundWithHook() throws RemotingCommandException {
GetMessageResult getMessageResult = createGetMessageResult();
- when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(SubscriptionData.class))).thenReturn(getMessageResult);
+ when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(ExpressionMessageFilter.class))).thenReturn(getMessageResult);
List consumeMessageHookList = new ArrayList<>();
final ConsumeMessageContext[] messageContext = new ConsumeMessageContext[1];
ConsumeMessageHook consumeMessageHook = new ConsumeMessageHook() {
@@ -168,7 +169,7 @@ public class PullMessageProcessorTest {
public void testProcessRequest_MsgWasRemoving() throws RemotingCommandException {
GetMessageResult getMessageResult = createGetMessageResult();
getMessageResult.setStatus(GetMessageStatus.MESSAGE_WAS_REMOVING);
- when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(SubscriptionData.class))).thenReturn(getMessageResult);
+ when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(ExpressionMessageFilter.class))).thenReturn(getMessageResult);
final RemotingCommand request = createPullMsgCommand(RequestCode.PULL_MESSAGE);
RemotingCommand response = pullMessageProcessor.processRequest(handlerContext, request);
@@ -180,7 +181,7 @@ public class PullMessageProcessorTest {
public void testProcessRequest_NoMsgInQueue() throws RemotingCommandException {
GetMessageResult getMessageResult = createGetMessageResult();
getMessageResult.setStatus(GetMessageStatus.NO_MESSAGE_IN_QUEUE);
- when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(SubscriptionData.class))).thenReturn(getMessageResult);
+ when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(ExpressionMessageFilter.class))).thenReturn(getMessageResult);
final RemotingCommand request = createPullMsgCommand(RequestCode.PULL_MESSAGE);
RemotingCommand response = pullMessageProcessor.processRequest(handlerContext, request);
diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java
index 3903fe2b4f83c99ce553d14070497324584386db..9c9b59ef0762b554c90e5cb7e719aabce2937e48 100644
--- a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java
+++ b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java
@@ -518,6 +518,21 @@ public class DefaultMQPushConsumer extends ClientConfig implements MQPushConsume
this.defaultMQPushConsumerImpl.subscribe(topic, fullClassName, filterClassSource);
}
+ /**
+ * Subscribe a topic by message selector.
+ *
+ * @see org.apache.rocketmq.client.consumer.MessageSelector#bySql
+ * @see org.apache.rocketmq.client.consumer.MessageSelector#byTag
+ *
+ * @param topic topic to consume.
+ * @param messageSelector {@link org.apache.rocketmq.client.consumer.MessageSelector}
+ * @throws MQClientException
+ */
+ @Override
+ public void subscribe(final String topic, final MessageSelector messageSelector) throws MQClientException {
+ this.defaultMQPushConsumerImpl.subscribe(topic, messageSelector);
+ }
+
/**
* Un-subscribe the specified topic from subscription.
* @param topic message topic
diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/MQPushConsumer.java b/client/src/main/java/org/apache/rocketmq/client/consumer/MQPushConsumer.java
index 92552815237f08292ba8091f6f9b73a2eeae802d..9c6c1f17805ec9b2c1490304d9dffdbdfa3175b0 100644
--- a/client/src/main/java/org/apache/rocketmq/client/consumer/MQPushConsumer.java
+++ b/client/src/main/java/org/apache/rocketmq/client/consumer/MQPushConsumer.java
@@ -69,6 +69,27 @@ public interface MQPushConsumer extends MQConsumer {
*/
void subscribe(final String topic, final String fullClassName, final String filterClassSource) throws MQClientException;
+ /**
+ * Subscribe some topic with selector.
+ *
+ * This interface also has the ability of {@link #subscribe(String, String)},
+ * and, support other message selection, such as {@link org.apache.rocketmq.common.filter.ExpressionType#SQL92}.
+ *
+ *
+ * @param topic
+ * @param selector message selector({@link MessageSelector}), can be null.
+ * @throws MQClientException
+ */
+ void subscribe(final String topic, final MessageSelector selector) throws MQClientException;
+
/**
* Unsubscribe consumption some topic
*
diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/MessageSelector.java b/client/src/main/java/org/apache/rocketmq/client/consumer/MessageSelector.java
new file mode 100644
index 0000000000000000000000000000000000000000..35a5181fd96de3228ccbe4e82ea8cdfda9b52f06
--- /dev/null
+++ b/client/src/main/java/org/apache/rocketmq/client/consumer/MessageSelector.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.client.consumer;
+
+import org.apache.rocketmq.common.filter.ExpressionType;
+
+/**
+ *
+ * Message selector: select message at server.
+ *
+ * Example:
+ * (a > 10 AND a < 100) OR (b IS NOT NULL AND b=TRUE)
+ *
+ */
+ public static final String SQL92 = "SQL92";
+
+ /**
+ * Only support or operation such as
+ * "tag1 || tag2 || tag3",
+ * If null or * expression,meaning subscribe all.
+ */
+ public static final String TAG = "TAG";
+
+ public static boolean isTagType(String type) {
+ if (type == null || TAG.equals(type)) {
+ return true;
+ }
+ return false;
+ }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/filter/FilterAPI.java b/common/src/main/java/org/apache/rocketmq/common/filter/FilterAPI.java
index e9bf3fa2902384636f80c8d43f2a3fe505ad2e55..fc8525c18ea23b5162ec5c4dc112b1ba3252019b 100644
--- a/common/src/main/java/org/apache/rocketmq/common/filter/FilterAPI.java
+++ b/common/src/main/java/org/apache/rocketmq/common/filter/FilterAPI.java
@@ -63,4 +63,22 @@ public class FilterAPI {
return subscriptionData;
}
+
+ public static SubscriptionData build(final String topic, final String subString,
+ final String type) throws Exception {
+ if (ExpressionType.TAG.equals(type) || type == null) {
+ return buildSubscriptionData(null, topic, subString);
+ }
+
+ if (subString == null || subString.length() < 1) {
+ throw new IllegalArgumentException("Expression can't be null! " + type);
+ }
+
+ SubscriptionData subscriptionData = new SubscriptionData();
+ subscriptionData.setTopic(topic);
+ subscriptionData.setSubString(subString);
+ subscriptionData.setExpressionType(type);
+
+ return subscriptionData;
+ }
}
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/Message.java b/common/src/main/java/org/apache/rocketmq/common/message/Message.java
index f3bff83292d7363a4a2a605b520ed21add4fa88c..2c81f5cd5e41e95d1d9bff6eee1b6635e5df65bf 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/Message.java
+++ b/common/src/main/java/org/apache/rocketmq/common/message/Message.java
@@ -81,6 +81,12 @@ public class Message implements Serializable {
throw new RuntimeException(String.format(
"The Property<%s> is used by system, input another please", name));
}
+ if (value == null || value == "" || value.trim() == ""
+ || name == null || name == "" || name.trim() == "") {
+ throw new IllegalArgumentException(
+ "The name or value of property can not be null or blank string!"
+ );
+ }
this.putProperty(name, value);
}
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java
index 90b837a335a03d1296fc4060753ce9c636bd3479..e41ec9d206a00f373321d566b75b5bfcfaad2ba0 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java
+++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java
@@ -41,6 +41,20 @@ public class MessageDecoder {
public final static int MESSAGE_MAGIC_CODE = 0xAABBCCDD ^ 1880681586 + 8;
public static final char NAME_VALUE_SEPARATOR = 1;
public static final char PROPERTY_SEPARATOR = 2;
+ public static final int BODY_SIZE_POSITION = 4 // 1 TOTALSIZE
+ + 4 // 2 MAGICCODE
+ + 4 // 3 BODYCRC
+ + 4 // 4 QUEUEID
+ + 4 // 5 FLAG
+ + 8 // 6 QUEUEOFFSET
+ + 8 // 7 PHYSICALOFFSET
+ + 4 // 8 SYSFLAG
+ + 8 // 9 BORNTIMESTAMP
+ + 8 // 10 BORNHOST
+ + 8 // 11 STORETIMESTAMP
+ + 8 // 12 STOREHOSTADDRESS
+ + 4 // 13 RECONSUMETIMES
+ + 8; // 14 Prepared Transaction Offset
public static String createMessageId(final ByteBuffer input, final ByteBuffer addr, final long offset) {
input.flip();
@@ -80,6 +94,31 @@ public class MessageDecoder {
return new MessageId(address, offset);
}
+ /**
+ * Just decode properties from msg buffer.
+ *
+ * @param byteBuffer msg commit log buffer.
+ * @return
+ */
+ public static Map decodeProperties(java.nio.ByteBuffer byteBuffer) {
+ int topicLengthPosition = BODY_SIZE_POSITION + 4 + byteBuffer.getInt(BODY_SIZE_POSITION);
+
+ byte topicLength = byteBuffer.get(topicLengthPosition);
+
+ short propertiesLength = byteBuffer.getShort(topicLengthPosition + 1 + topicLength);
+
+ byteBuffer.position(topicLengthPosition + 1 + topicLength + 2);
+
+ if (propertiesLength > 0) {
+ byte[] properties = new byte[propertiesLength];
+ byteBuffer.get(properties);
+ String propertiesString = new String(properties, CHARSET_UTF8);
+ Map map = string2messageProperties(propertiesString);
+ return map;
+ }
+ return null;
+ }
+
public static MessageExt decode(java.nio.ByteBuffer byteBuffer) {
return decode(byteBuffer, true, true, false);
}
diff --git a/common/src/main/java/org/apache/rocketmq/common/namesrv/TopAddressing.java b/common/src/main/java/org/apache/rocketmq/common/namesrv/TopAddressing.java
index 74fd9654c9bf50cf70023fd64cdf553caaa2a4e8..990e7486d7731d4e3394b590547ad3ad78c58540 100644
--- a/common/src/main/java/org/apache/rocketmq/common/namesrv/TopAddressing.java
+++ b/common/src/main/java/org/apache/rocketmq/common/namesrv/TopAddressing.java
@@ -88,7 +88,7 @@ public class TopAddressing {
if (verbose) {
String errorMsg =
- "connect to " + url + " failed, maybe the domain name " + MixAll.WS_DOMAIN_NAME + " not bind in /etc/hosts";
+ "connect to " + url + " failed, maybe the domain name " + MixAll.getWSAddr() + " not bind in /etc/hosts";
errorMsg += FAQUrl.suggestTodo(FAQUrl.NAME_SERVER_ADDR_NOT_EXIST_URL);
log.warn(errorMsg);
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/RequestCode.java b/common/src/main/java/org/apache/rocketmq/common/protocol/RequestCode.java
index c6b0925535fceca0db7c4db3c6c10274e1070f3b..6f132f7cdf47254980422935580dd39f38fcde45 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/RequestCode.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/RequestCode.java
@@ -68,6 +68,8 @@ public class RequestCode {
public static final int GET_ALL_DELAY_OFFSET = 45;
+ public static final int CHECK_CLIENT_CONFIG = 46;
+
public static final int PUT_KV_CONFIG = 100;
public static final int GET_KV_CONFIG = 101;
@@ -162,4 +164,6 @@ public class RequestCode {
public static final int SEND_BATCH_MESSAGE = 320;
+
+ public static final int QUERY_CONSUME_QUEUE = 321;
}
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/ResponseCode.java b/common/src/main/java/org/apache/rocketmq/common/protocol/ResponseCode.java
index 90b182b1e54784025877a3d98237b9603483ec7a..f62c4ea61874528ad8d922972ccf358c603c62b4 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/ResponseCode.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/ResponseCode.java
@@ -53,6 +53,10 @@ public class ResponseCode extends RemotingSysResponseCode {
public static final int SUBSCRIPTION_GROUP_NOT_EXIST = 26;
+ public static final int FILTER_DATA_NOT_EXIST = 27;
+
+ public static final int FILTER_DATA_NOT_LATEST = 28;
+
public static final int TRANSACTION_SHOULD_COMMIT = 200;
public static final int TRANSACTION_SHOULD_ROLLBACK = 201;
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBody.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBody.java
new file mode 100644
index 0000000000000000000000000000000000000000..a78ce554560a478cb6fb3629e42f008699e5a5d5
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBody.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.common.protocol.body;
+
+import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
+import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
+
+public class CheckClientRequestBody extends RemotingSerializable {
+
+ private String clientId;
+ private String group;
+ private SubscriptionData subscriptionData;
+
+ public String getClientId() {
+ return clientId;
+ }
+
+ public void setClientId(String clientId) {
+ this.clientId = clientId;
+ }
+
+ public String getGroup() {
+ return group;
+ }
+
+ public void setGroup(String group) {
+ this.group = group;
+ }
+
+ public SubscriptionData getSubscriptionData() {
+ return subscriptionData;
+ }
+
+ public void setSubscriptionData(SubscriptionData subscriptionData) {
+ this.subscriptionData = subscriptionData;
+ }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumeQueueData.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumeQueueData.java
new file mode 100644
index 0000000000000000000000000000000000000000..7268dcda56b8ba8f9fd7914fb73c7e6015d8eab9
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumeQueueData.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.common.protocol.body;
+
+public class ConsumeQueueData {
+
+ private long physicOffset;
+ private int physicSize;
+ private long tagsCode;
+ private String extendDataJson;
+ private String bitMap;
+ private boolean eval;
+ private String msg;
+
+ public long getPhysicOffset() {
+ return physicOffset;
+ }
+
+ public void setPhysicOffset(long physicOffset) {
+ this.physicOffset = physicOffset;
+ }
+
+ public int getPhysicSize() {
+ return physicSize;
+ }
+
+ public void setPhysicSize(int physicSize) {
+ this.physicSize = physicSize;
+ }
+
+ public long getTagsCode() {
+ return tagsCode;
+ }
+
+ public void setTagsCode(long tagsCode) {
+ this.tagsCode = tagsCode;
+ }
+
+ public String getExtendDataJson() {
+ return extendDataJson;
+ }
+
+ public void setExtendDataJson(String extendDataJson) {
+ this.extendDataJson = extendDataJson;
+ }
+
+ public String getBitMap() {
+ return bitMap;
+ }
+
+ public void setBitMap(String bitMap) {
+ this.bitMap = bitMap;
+ }
+
+ public boolean isEval() {
+ return eval;
+ }
+
+ public void setEval(boolean eval) {
+ this.eval = eval;
+ }
+
+ public String getMsg() {
+ return msg;
+ }
+
+ public void setMsg(String msg) {
+ this.msg = msg;
+ }
+
+ @Override
+ public String toString() {
+ return "ConsumeQueueData{" +
+ "physicOffset=" + physicOffset +
+ ", physicSize=" + physicSize +
+ ", tagsCode=" + tagsCode +
+ ", extendDataJson='" + extendDataJson + '\'' +
+ ", bitMap='" + bitMap + '\'' +
+ ", eval=" + eval +
+ ", msg='" + msg + '\'' +
+ '}';
+ }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/QueryConsumeQueueResponseBody.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/QueryConsumeQueueResponseBody.java
new file mode 100644
index 0000000000000000000000000000000000000000..be93da993d025bece0b281363dbb81c229eb2520
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/QueryConsumeQueueResponseBody.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.common.protocol.body;
+
+import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
+import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
+
+import java.util.List;
+
+public class QueryConsumeQueueResponseBody extends RemotingSerializable {
+
+ private SubscriptionData subscriptionData;
+ private String filterData;
+ private List queueData;
+ private long maxQueueIndex;
+ private long minQueueIndex;
+
+ public SubscriptionData getSubscriptionData() {
+ return subscriptionData;
+ }
+
+ public void setSubscriptionData(SubscriptionData subscriptionData) {
+ this.subscriptionData = subscriptionData;
+ }
+
+ public String getFilterData() {
+ return filterData;
+ }
+
+ public void setFilterData(String filterData) {
+ this.filterData = filterData;
+ }
+
+ public List getQueueData() {
+ return queueData;
+ }
+
+ public void setQueueData(List queueData) {
+ this.queueData = queueData;
+ }
+
+ public long getMaxQueueIndex() {
+ return maxQueueIndex;
+ }
+
+ public void setMaxQueueIndex(long maxQueueIndex) {
+ this.maxQueueIndex = maxQueueIndex;
+ }
+
+ public long getMinQueueIndex() {
+ return minQueueIndex;
+ }
+
+ public void setMinQueueIndex(long minQueueIndex) {
+ this.minQueueIndex = minQueueIndex;
+ }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageRequestHeader.java
index 8a592132e4affce0db1c57aa35d168aa7e148c18..106e89e511c159e2b3a4aa5aa6d1bd2fe4e85487 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageRequestHeader.java
@@ -46,6 +46,7 @@ public class PullMessageRequestHeader implements CommandCustomHeader {
private String subscription;
@CFNotNull
private Long subVersion;
+ private String expressionType;
@Override
public void checkFields() throws RemotingCommandException {
@@ -130,4 +131,12 @@ public class PullMessageRequestHeader implements CommandCustomHeader {
public void setSubVersion(Long subVersion) {
this.subVersion = subVersion;
}
+
+ public String getExpressionType() {
+ return expressionType;
+ }
+
+ public void setExpressionType(String expressionType) {
+ this.expressionType = expressionType;
+ }
}
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/QueryConsumeQueueRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/QueryConsumeQueueRequestHeader.java
new file mode 100644
index 0000000000000000000000000000000000000000..642fe17cb63c5bed56a6bfffb01d8a65daf0476e
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/QueryConsumeQueueRequestHeader.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.common.protocol.header;
+
+import org.apache.rocketmq.remoting.CommandCustomHeader;
+import org.apache.rocketmq.remoting.exception.RemotingCommandException;
+
+public class QueryConsumeQueueRequestHeader implements CommandCustomHeader {
+
+ private String topic;
+ private int queueId;
+ private long index;
+ private int count;
+ private String consumerGroup;
+
+ public String getTopic() {
+ return topic;
+ }
+
+ public void setTopic(String topic) {
+ this.topic = topic;
+ }
+
+ public int getQueueId() {
+ return queueId;
+ }
+
+ public void setQueueId(int queueId) {
+ this.queueId = queueId;
+ }
+
+ public long getIndex() {
+ return index;
+ }
+
+ public void setIndex(long index) {
+ this.index = index;
+ }
+
+ public int getCount() {
+ return count;
+ }
+
+ public void setCount(int count) {
+ this.count = count;
+ }
+
+ public String getConsumerGroup() {
+ return consumerGroup;
+ }
+
+ public void setConsumerGroup(String consumerGroup) {
+ this.consumerGroup = consumerGroup;
+ }
+
+ @Override
+ public void checkFields() throws RemotingCommandException {
+
+ }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/heartbeat/SubscriptionData.java b/common/src/main/java/org/apache/rocketmq/common/protocol/heartbeat/SubscriptionData.java
index 81f59544c29b1a8ba4a53e8a67c9547430eb6fcf..e456b7e606a24dc8fcaa5379692f668dedf21efc 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/heartbeat/SubscriptionData.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/heartbeat/SubscriptionData.java
@@ -32,6 +32,7 @@ public class SubscriptionData implements Comparable {
private Set tagsSet = new HashSet();
private Set codeSet = new HashSet();
private long subVersion = System.currentTimeMillis();
+ private String expressionType;
@JSONField(serialize = false)
private String filterClassSource;
@@ -102,6 +103,14 @@ public class SubscriptionData implements Comparable {
this.classFilterMode = classFilterMode;
}
+ public String getExpressionType() {
+ return expressionType;
+ }
+
+ public void setExpressionType(String expressionType) {
+ this.expressionType = expressionType;
+ }
+
@Override
public int hashCode() {
final int prime = 31;
@@ -111,6 +120,7 @@ public class SubscriptionData implements Comparable {
result = prime * result + ((subString == null) ? 0 : subString.hashCode());
result = prime * result + ((tagsSet == null) ? 0 : tagsSet.hashCode());
result = prime * result + ((topic == null) ? 0 : topic.hashCode());
+ result = prime * result + ((expressionType == null) ? 0 : expressionType.hashCode());
return result;
}
@@ -147,6 +157,11 @@ public class SubscriptionData implements Comparable {
return false;
} else if (!topic.equals(other.topic))
return false;
+ if (expressionType == null) {
+ if (other.expressionType != null)
+ return false;
+ } else if (!expressionType.equals(other.expressionType))
+ return false;
return true;
}
@@ -154,7 +169,7 @@ public class SubscriptionData implements Comparable {
public String toString() {
return "SubscriptionData [classFilterMode=" + classFilterMode + ", topic=" + topic + ", subString="
+ subString + ", tagsSet=" + tagsSet + ", codeSet=" + codeSet + ", subVersion=" + subVersion
- + "]";
+ + ", expressionType=" + expressionType + "]";
}
@Override
diff --git a/common/src/test/java/org/apache/rocketmq/common/filter/FilterAPITest.java b/common/src/test/java/org/apache/rocketmq/common/filter/FilterAPITest.java
index 5137f3282ee91cc61a55336a9bb47d11cc0b2b34..c5f8460eeb3467c166423d9bb3465780f441ba04 100644
--- a/common/src/test/java/org/apache/rocketmq/common/filter/FilterAPITest.java
+++ b/common/src/test/java/org/apache/rocketmq/common/filter/FilterAPITest.java
@@ -42,4 +42,53 @@ public class FilterAPITest {
}
assertThat(subscriptionData.getTagsSet()).isEqualTo(tagSet);
}
+
+ @Test
+ public void testBuildTagSome() {
+ try {
+ SubscriptionData subscriptionData = FilterAPI.build(
+ "TOPIC", "A || B", ExpressionType.TAG
+ );
+
+ assertThat(subscriptionData).isNotNull();
+ assertThat(subscriptionData.getTopic()).isEqualTo("TOPIC");
+ assertThat(subscriptionData.getSubString()).isEqualTo("A || B");
+ assertThat(ExpressionType.isTagType(subscriptionData.getExpressionType())).isTrue();
+
+ assertThat(subscriptionData.getTagsSet()).isNotNull();
+ assertThat(subscriptionData.getTagsSet()).containsExactly("A", "B");
+ } catch (Exception e) {
+ e.printStackTrace();
+ assertThat(Boolean.FALSE).isTrue();
+ }
+ }
+
+ @Test
+ public void testBuildSQL() {
+ try {
+ SubscriptionData subscriptionData = FilterAPI.build(
+ "TOPIC", "a is not null", ExpressionType.SQL92
+ );
+
+ assertThat(subscriptionData).isNotNull();
+ assertThat(subscriptionData.getTopic()).isEqualTo("TOPIC");
+ assertThat(subscriptionData.getExpressionType()).isEqualTo(ExpressionType.SQL92);
+ } catch (Exception e) {
+ e.printStackTrace();
+ assertThat(Boolean.FALSE).isTrue();
+ }
+ }
+
+ @Test
+ public void testBuildSQLWithNullSubString() {
+ try {
+ FilterAPI.build(
+ "TOPIC", null, ExpressionType.SQL92
+ );
+
+ assertThat(Boolean.FALSE).isTrue();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
}
diff --git a/common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java b/common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java
new file mode 100644
index 0000000000000000000000000000000000000000..d14d6b060c89168cde510531da86740ec8b33ad5
--- /dev/null
+++ b/common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.common.message;
+
+import org.junit.Test;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class MessageDecoderTest {
+
+ @Test
+ public void testDecodeProperties() {
+ MessageExt messageExt = new MessageExt();
+
+ messageExt.setMsgId("645100FA00002A9F000000489A3AA09E");
+ messageExt.setTopic("abc");
+ messageExt.setBody("hello!q!".getBytes());
+ try {
+ messageExt.setBornHost(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 0));
+ } catch (UnknownHostException e) {
+ e.printStackTrace();
+ assertThat(Boolean.FALSE).isTrue();
+ }
+ messageExt.setBornTimestamp(System.currentTimeMillis());
+ messageExt.setCommitLogOffset(123456);
+ messageExt.setPreparedTransactionOffset(0);
+ messageExt.setQueueId(0);
+ messageExt.setQueueOffset(123);
+ messageExt.setReconsumeTimes(0);
+ try {
+ messageExt.setStoreHost(new InetSocketAddress(InetAddress.getLocalHost(), 0));
+ } catch (UnknownHostException e) {
+ e.printStackTrace();
+ assertThat(Boolean.FALSE).isTrue();
+ }
+
+ messageExt.putUserProperty("a", "123");
+ messageExt.putUserProperty("b", "hello");
+ messageExt.putUserProperty("c", "3.14");
+
+ byte[] msgBytes = new byte[0];
+ try {
+ msgBytes = MessageDecoder.encode(messageExt, false);
+ } catch (Exception e) {
+ e.printStackTrace();
+ assertThat(Boolean.FALSE).isTrue();
+ }
+
+ ByteBuffer byteBuffer = ByteBuffer.allocate(msgBytes.length);
+ byteBuffer.put(msgBytes);
+
+ Map properties = MessageDecoder.decodeProperties(byteBuffer);
+
+ assertThat(properties).isNotNull();
+ assertThat("123").isEqualTo(properties.get("a"));
+ assertThat("hello").isEqualTo(properties.get("b"));
+ assertThat("3.14").isEqualTo(properties.get("c"));
+ }
+}
diff --git a/distribution/conf/logback_broker.xml b/distribution/conf/logback_broker.xml
index 05c0ee4fb08d772aa2098570913d1f6e939751b3..dd5c63f5de0378d04464e1a2c977599ea8e46176 100644
--- a/distribution/conf/logback_broker.xml
+++ b/distribution/conf/logback_broker.xml
@@ -222,6 +222,29 @@
+
+ ${user.home}/logs/rocketmqlogs/filter.log
+ true
+
+ ${user.home}/logs/rocketmqlogs/otherdays/filter.%i.log
+
+ 1
+ 10
+
+
+ 100MB
+
+
+ %d{yyy-MM-dd HH:mm:ss,GMT+8} %p %t - %m%n
+ UTF-8
+
+
+
+
+
+
${user.home}/logs/rocketmqlogs/stats.log
@@ -321,6 +344,11 @@
+
+
+
+
+
diff --git a/distribution/release.xml b/distribution/release.xml
index 2d3ec1ecc23001052df8fcacde36f6828c841002..9e4ef2a017a9c874721f81f436dd6870cf0ac88f 100644
--- a/distribution/release.xml
+++ b/distribution/release.xml
@@ -67,6 +67,7 @@
org.apache.rocketmq:rocketmq-namesrvorg.apache.rocketmq:rocketmq-filtersrvorg.apache.rocketmq:rocketmq-example
+ org.apache.rocketmq:rocketmq-filterlib/
diff --git a/example/src/main/java/org/apache/rocketmq/example/benchmark/Consumer.java b/example/src/main/java/org/apache/rocketmq/example/benchmark/Consumer.java
index 473e4c77004cef3fcd3b14b4ca27f88fc91eb600..3e1b79b0cd80102d107edc523b334da3e64e6a28 100644
--- a/example/src/main/java/org/apache/rocketmq/example/benchmark/Consumer.java
+++ b/example/src/main/java/org/apache/rocketmq/example/benchmark/Consumer.java
@@ -27,10 +27,13 @@ import org.apache.commons.cli.Option;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.PosixParser;
import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
+import org.apache.rocketmq.client.consumer.MessageSelector;
import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext;
import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.common.filter.ExpressionType;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.srvutil.ServerUtil;
@@ -46,12 +49,14 @@ public class Consumer {
final String topic = commandLine.hasOption('t') ? commandLine.getOptionValue('t').trim() : "BenchmarkTest";
final String groupPrefix = commandLine.hasOption('g') ? commandLine.getOptionValue('g').trim() : "benchmark_consumer";
final String isPrefixEnable = commandLine.hasOption('p') ? commandLine.getOptionValue('p').trim() : "true";
+ final String filterType = commandLine.hasOption('f') ? commandLine.getOptionValue('f').trim() : null;
+ final String expression = commandLine.hasOption('e') ? commandLine.getOptionValue('e').trim() : null;
String group = groupPrefix;
if (Boolean.parseBoolean(isPrefixEnable)) {
group = groupPrefix + "_" + Long.toString(System.currentTimeMillis() % 100);
}
- System.out.printf("topic %s group %s prefix %s%n", topic, group, isPrefixEnable);
+ System.out.printf("topic: %s, group: %s, prefix: %s, filterType: %s, expression: %s%n", topic, group, isPrefixEnable, filterType, expression);
final StatsBenchmarkConsumer statsBenchmarkConsumer = new StatsBenchmarkConsumer();
@@ -99,7 +104,21 @@ public class Consumer {
DefaultMQPushConsumer consumer = new DefaultMQPushConsumer(group);
consumer.setInstanceName(Long.toString(System.currentTimeMillis()));
- consumer.subscribe(topic, "*");
+ if (filterType == null || expression == null) {
+ consumer.subscribe(topic, "*");
+ } else {
+ if (ExpressionType.TAG.equals(filterType)) {
+ String expr = MixAll.file2String(expression);
+ System.out.printf("Expression: %s%n", expr);
+ consumer.subscribe(topic, MessageSelector.byTag(expr));
+ } else if (ExpressionType.SQL92.equals(filterType)) {
+ String expr = MixAll.file2String(expression);
+ System.out.printf("Expression: %s%n", expr);
+ consumer.subscribe(topic, MessageSelector.bySql(expr));
+ } else {
+ throw new IllegalArgumentException("Not support filter type! " + filterType);
+ }
+ }
consumer.registerMessageListener(new MessageListenerConcurrently() {
@Override
@@ -142,6 +161,14 @@ public class Consumer {
opt.setRequired(false);
options.addOption(opt);
+ opt = new Option("f", "filterType", true, "TAG, SQL92");
+ opt.setRequired(false);
+ options.addOption(opt);
+
+ opt = new Option("e", "expression", true, "filter expression content file path.ie: ./test/expr");
+ opt.setRequired(false);
+ options.addOption(opt);
+
return options;
}
diff --git a/example/src/main/java/org/apache/rocketmq/example/benchmark/Producer.java b/example/src/main/java/org/apache/rocketmq/example/benchmark/Producer.java
index 50d750df4dc4cb6512f4a8d70a3dde567fe47c98..2d8d0f6fa185915d1a5128070ba2a6ec3af44934 100644
--- a/example/src/main/java/org/apache/rocketmq/example/benchmark/Producer.java
+++ b/example/src/main/java/org/apache/rocketmq/example/benchmark/Producer.java
@@ -18,11 +18,13 @@ package org.apache.rocketmq.example.benchmark;
import java.io.UnsupportedEncodingException;
import java.util.LinkedList;
+import java.util.Random;
import java.util.Timer;
import java.util.TimerTask;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicLong;
+
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.Options;
@@ -50,13 +52,12 @@ public class Producer {
final int threadCount = commandLine.hasOption('w') ? Integer.parseInt(commandLine.getOptionValue('w')) : 64;
final int messageSize = commandLine.hasOption('s') ? Integer.parseInt(commandLine.getOptionValue('s')) : 128;
final boolean keyEnable = commandLine.hasOption('k') && Boolean.parseBoolean(commandLine.getOptionValue('k'));
+ final int propertySize = commandLine.hasOption('p') ? Integer.parseInt(commandLine.getOptionValue('p')) : 0;
System.out.printf("topic %s threadCount %d messageSize %d keyEnable %s%n", topic, threadCount, messageSize, keyEnable);
final Logger log = ClientLogger.getLog();
- final Message msg = buildMessage(messageSize, topic);
-
final ExecutorService sendThreadPool = Executors.newFixedThreadPool(threadCount);
final StatsBenchmarkProducer statsBenchmark = new StatsBenchmarkProducer();
@@ -117,10 +118,37 @@ public class Producer {
public void run() {
while (true) {
try {
+ final Message msg;
+ try {
+ msg = buildMessage(messageSize, topic);
+ } catch (UnsupportedEncodingException e) {
+ e.printStackTrace();
+ return;
+ }
final long beginTimestamp = System.currentTimeMillis();
if (keyEnable) {
msg.setKeys(String.valueOf(beginTimestamp / 1000));
}
+ if (propertySize > 0) {
+ if (msg.getProperties() != null) {
+ msg.getProperties().clear();
+ }
+ int i = 0;
+ int startValue = (new Random(System.currentTimeMillis())).nextInt(100);
+ int size = 0;
+ while (true) {
+ String prop1 = "prop" + i, prop1V = "hello" + startValue;
+ String prop2 = "prop" + (i + 1), prop2V = String.valueOf(startValue);
+ msg.putUserProperty(prop1, prop1V);
+ msg.putUserProperty(prop2, prop2V);
+ size += prop1.length() + prop2.length() + prop1V.length() + prop2V.length();
+ if (size > propertySize) {
+ break;
+ }
+ i += 2;
+ startValue += 2;
+ }
+ }
producer.send(msg);
statsBenchmark.getSendRequestSuccessCount().incrementAndGet();
statsBenchmark.getReceiveResponseSuccessCount().incrementAndGet();
@@ -214,7 +242,7 @@ class StatsBenchmarkProducer {
private final AtomicLong sendMessageMaxRT = new AtomicLong(0L);
public Long[] createSnapshot() {
- Long[] snap = new Long[] {
+ Long[] snap = new Long[]{
System.currentTimeMillis(),
this.sendRequestSuccessCount.get(),
this.sendRequestFailedCount.get(),
diff --git a/example/src/main/java/org/apache/rocketmq/example/filter/SqlConsumer.java b/example/src/main/java/org/apache/rocketmq/example/filter/SqlConsumer.java
new file mode 100644
index 0000000000000000000000000000000000000000..9a3b8131588972d9674659ee1f575d58d870b1c9
--- /dev/null
+++ b/example/src/main/java/org/apache/rocketmq/example/filter/SqlConsumer.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.example.filter;
+
+import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
+import org.apache.rocketmq.client.consumer.MessageSelector;
+import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext;
+import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
+import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.common.message.MessageExt;
+
+import java.util.List;
+
+public class SqlConsumer {
+
+ public static void main(String[] args) {
+ DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("please_rename_unique_group_name_4");
+
+ try {
+ consumer.subscribe("TopicTest",
+ MessageSelector.bySql("(TAGS is not null and TAGS in ('TagA', 'TagB'))" +
+ "and (a is not null and a between 0 3)"));
+ } catch (MQClientException e) {
+ e.printStackTrace();
+ return;
+ }
+
+ consumer.registerMessageListener(new MessageListenerConcurrently() {
+
+ @Override
+ public ConsumeConcurrentlyStatus consumeMessage(List msgs,
+ ConsumeConcurrentlyContext context) {
+ System.out.printf(Thread.currentThread().getName() + " Receive New Messages: " + msgs + "%n");
+ return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
+ }
+ });
+
+ try {
+ consumer.start();
+ } catch (MQClientException e) {
+ e.printStackTrace();
+ return;
+ }
+ System.out.printf("Consumer Started.%n");
+ }
+}
diff --git a/example/src/main/java/org/apache/rocketmq/example/filter/SqlProducer.java b/example/src/main/java/org/apache/rocketmq/example/filter/SqlProducer.java
new file mode 100644
index 0000000000000000000000000000000000000000..3f3a0e65215b0d1f53720c26a25f9f53bdeac68b
--- /dev/null
+++ b/example/src/main/java/org/apache/rocketmq/example/filter/SqlProducer.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.example.filter;
+
+import org.apache.rocketmq.client.exception.MQClientException;
+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 SqlProducer {
+
+ public static void main(String[] args) {
+ DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name");
+ try {
+ producer.start();
+ } catch (MQClientException e) {
+ e.printStackTrace();
+ return;
+ }
+
+ for (int i = 0; i < 10; i++) {
+ try {
+ String tag;
+ int div = i % 3;
+ if (div == 0) {
+ tag = "TagA";
+ } else if (div == 1) {
+ tag = "TagB";
+ } else {
+ tag = "TagC";
+ }
+ Message msg = new Message("TopicTest",
+ tag,
+ ("Hello RocketMQ " + i).getBytes(RemotingHelper.DEFAULT_CHARSET)
+ );
+ msg.putUserProperty("a", String.valueOf(i));
+
+ SendResult sendResult = producer.send(msg);
+ System.out.printf("%s%n", sendResult);
+ } catch (Exception e) {
+ e.printStackTrace();
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e1) {
+ e1.printStackTrace();
+ }
+ }
+ }
+ producer.shutdown();
+ }
+}
diff --git a/filter/pom.xml b/filter/pom.xml
new file mode 100644
index 0000000000000000000000000000000000000000..7978f0552e315d87546a69ed8b072ceb1f9c9b60
--- /dev/null
+++ b/filter/pom.xml
@@ -0,0 +1,43 @@
+
+
+
+
+
+ rocketmq-all
+ org.apache.rocketmq
+ 4.1.0-incubating-SNAPSHOT
+
+
+ 4.0.0
+ jar
+ rocketmq-filter
+ rocketmq-filter ${project.version}
+
+
+
+ ${project.groupId}
+ rocketmq-common
+
+
+ ${project.groupId}
+ rocketmq-srvutil
+
+
+
\ No newline at end of file
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/FilterFactory.java b/filter/src/main/java/org/apache/rocketmq/filter/FilterFactory.java
new file mode 100644
index 0000000000000000000000000000000000000000..a31854840c7ccab060825c2e9b9756f5d51ccb7c
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/FilterFactory.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Filter factory: support other filter to register.
+ */
+public class FilterFactory {
+
+ public static final FilterFactory INSTANCE = new FilterFactory();
+
+ protected static final Map FILTER_SPI_HOLDER = new HashMap(4);
+
+ static {
+ FilterFactory.INSTANCE.register(new SqlFilter());
+ }
+
+ /**
+ * Register a filter.
+ *
+ * Note:
+ *
1. Filter registered will be used in broker server, so take care of it's reliability and performance.
+ *
+ * @param filterSpi
+ */
+ public void register(FilterSpi filterSpi) {
+ if (FILTER_SPI_HOLDER.containsKey(filterSpi.ofType())) {
+ throw new IllegalArgumentException(String.format("Filter spi type(%s) already exist!", filterSpi.ofType()));
+ }
+
+ FILTER_SPI_HOLDER.put(filterSpi.ofType(), filterSpi);
+ }
+
+ /**
+ * Un register a filter.
+ *
+ * @param type
+ * @return
+ */
+ public FilterSpi unRegister(String type) {
+ return FILTER_SPI_HOLDER.remove(type);
+ }
+
+ /**
+ * Get a filter registered, null if none exist.
+ *
+ * @param type
+ * @return
+ */
+ public FilterSpi get(String type) {
+ return FILTER_SPI_HOLDER.get(type);
+ }
+
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/FilterSpi.java b/filter/src/main/java/org/apache/rocketmq/filter/FilterSpi.java
new file mode 100644
index 0000000000000000000000000000000000000000..fcc39fac6077096cd0a4e67c4680af1cd1819730
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/FilterSpi.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter;
+
+import org.apache.rocketmq.filter.expression.Expression;
+import org.apache.rocketmq.filter.expression.MQFilterException;
+
+/**
+ * Filter spi interface.
+ */
+public interface FilterSpi {
+
+ /**
+ * Compile.
+ *
+ * @param expr
+ * @return
+ * @throws org.apache.rocketmq.filter.expression.MQFilterException
+ */
+ Expression compile(final String expr) throws MQFilterException;
+
+ /**
+ * Which type.
+ *
+ * @return
+ */
+ String ofType();
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/SqlFilter.java b/filter/src/main/java/org/apache/rocketmq/filter/SqlFilter.java
new file mode 100644
index 0000000000000000000000000000000000000000..0c1ffb8db8656da85cd61caf23aa6891a2176201
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/SqlFilter.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter;
+
+import org.apache.rocketmq.common.filter.ExpressionType;
+import org.apache.rocketmq.filter.expression.Expression;
+import org.apache.rocketmq.filter.expression.MQFilterException;
+import org.apache.rocketmq.filter.parser.SelectorParser;
+
+/**
+ * SQL92 Filter, just a wrapper of {@link org.apache.rocketmq.filter.parser.SelectorParser}.
+ *
+ *
+ * Do not use this filter directly.Use {@link FilterFactory#get} to select a filter.
+ *
+ */
+public class SqlFilter implements FilterSpi {
+
+ @Override
+ public Expression compile(final String expr) throws MQFilterException {
+ return SelectorParser.parse(expr);
+ }
+
+ @Override
+ public String ofType() {
+ return ExpressionType.SQL92;
+ }
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/constant/UnaryType.java b/filter/src/main/java/org/apache/rocketmq/filter/constant/UnaryType.java
new file mode 100644
index 0000000000000000000000000000000000000000..d2d04cdb7bf225624729f8200597ac6c52004508
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/constant/UnaryType.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter.constant;
+
+public enum UnaryType {
+ NEGATE,
+ IN,
+ NOT,
+ BOOLEANCAST,
+ LIKE
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/BinaryExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/BinaryExpression.java
new file mode 100644
index 0000000000000000000000000000000000000000..0f172e305f9f67a0d3f49fd7bfdd2a825657c034
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/BinaryExpression.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter.expression;
+
+/**
+ * An expression which performs an operation on two expression values.
+ *
+ * This class was taken from ActiveMQ org.apache.activemq.filter.BinaryExpression,
+ *
+ */
+public abstract class BinaryExpression implements Expression {
+ protected Expression left;
+ protected Expression right;
+
+ public BinaryExpression(Expression left, Expression right) {
+ this.left = left;
+ this.right = right;
+ }
+
+ public Expression getLeft() {
+ return left;
+ }
+
+ public Expression getRight() {
+ return right;
+ }
+
+ /**
+ * @see Object#toString()
+ */
+ public String toString() {
+ return "(" + left.toString() + " " + getExpressionSymbol() + " " + right.toString() + ")";
+ }
+
+ /**
+ * @see Object#hashCode()
+ */
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ /**
+ * @see Object#equals(Object)
+ */
+ public boolean equals(Object o) {
+
+ if (o == null || !this.getClass().equals(o.getClass())) {
+ return false;
+ }
+ return toString().equals(o.toString());
+
+ }
+
+ /**
+ * Returns the symbol that represents this binary expression. For example, addition is
+ * represented by "+"
+ *
+ * @return
+ */
+ public abstract String getExpressionSymbol();
+
+ /**
+ * @param expression
+ */
+ public void setRight(Expression expression) {
+ right = expression;
+ }
+
+ /**
+ * @param expression
+ */
+ public void setLeft(Expression expression) {
+ left = expression;
+ }
+
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/BooleanExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/BooleanExpression.java
new file mode 100644
index 0000000000000000000000000000000000000000..bb54632e640df17527f211d7c72eb872aa04ccbf
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/BooleanExpression.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter.expression;
+
+/**
+ * A BooleanExpression is an expression that always
+ * produces a Boolean result.
+ *
+ * This class was taken from ActiveMQ org.apache.activemq.filter.BooleanExpression,
+ * but the parameter is changed to an interface.
+ *
+ *
+ * @see org.apache.rocketmq.filter.expression.EvaluationContext
+ */
+public interface BooleanExpression extends Expression {
+
+ /**
+ * @param context
+ * @return true if the expression evaluates to Boolean.TRUE.
+ * @throws Exception
+ */
+ boolean matches(EvaluationContext context) throws Exception;
+
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/ComparisonExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/ComparisonExpression.java
new file mode 100644
index 0000000000000000000000000000000000000000..8b82e5739775fdd15ac241194f77c0e70e29e474
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/ComparisonExpression.java
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter.expression;
+
+import java.util.List;
+
+/**
+ * A filter performing a comparison of two objects
+ *
+ * This class was taken from ActiveMQ org.apache.activemq.filter.ComparisonExpression,
+ * but:
+ * 1. Remove LIKE expression, and related methods;
+ * 2. Extract a new method __compare which has int return value;
+ * 3. When create between expression, check whether left value is less or equal than right value;
+ * 4. For string type value(can not convert to number), only equal or unequal comparison are supported.
+ *
+ */
+public abstract class ComparisonExpression extends BinaryExpression implements BooleanExpression {
+
+ public static final ThreadLocal CONVERT_STRING_EXPRESSIONS = new ThreadLocal();
+
+ boolean convertStringExpressions = false;
+
+ /**
+ * @param left
+ * @param right
+ */
+ public ComparisonExpression(Expression left, Expression right) {
+ super(left, right);
+ convertStringExpressions = CONVERT_STRING_EXPRESSIONS.get() != null;
+ }
+
+ public static BooleanExpression createBetween(Expression value, Expression left, Expression right) {
+ // check
+ if (left instanceof ConstantExpression && right instanceof ConstantExpression) {
+ Object lv = ((ConstantExpression) left).getValue();
+ Object rv = ((ConstantExpression) right).getValue();
+ if (lv == null || rv == null) {
+ throw new RuntimeException("Illegal values of between, values can not be null!");
+ }
+ if (lv instanceof Comparable && rv instanceof Comparable) {
+ int ret = __compare((Comparable) rv, (Comparable) lv, true);
+ if (ret < 0)
+ throw new RuntimeException(
+ String.format("Illegal values of between, left value(%s) must less than or equal to right value(%s)", lv, rv)
+ );
+ }
+ }
+
+ return LogicExpression.createAND(createGreaterThanEqual(value, left), createLessThanEqual(value, right));
+ }
+
+ public static BooleanExpression createNotBetween(Expression value, Expression left, Expression right) {
+ return LogicExpression.createOR(createLessThan(value, left), createGreaterThan(value, right));
+ }
+
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ public static BooleanExpression createInFilter(Expression left, List elements) {
+
+ if (!(left instanceof PropertyExpression)) {
+ throw new RuntimeException("Expected a property for In expression, got: " + left);
+ }
+ return UnaryExpression.createInExpression((PropertyExpression) left, elements, false);
+
+ }
+
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ public static BooleanExpression createNotInFilter(Expression left, List elements) {
+
+ if (!(left instanceof PropertyExpression)) {
+ throw new RuntimeException("Expected a property for In expression, got: " + left);
+ }
+ return UnaryExpression.createInExpression((PropertyExpression) left, elements, true);
+
+ }
+
+ public static BooleanExpression createIsNull(Expression left) {
+ return doCreateEqual(left, ConstantExpression.NULL);
+ }
+
+ public static BooleanExpression createIsNotNull(Expression left) {
+ return UnaryExpression.createNOT(doCreateEqual(left, ConstantExpression.NULL));
+ }
+
+ public static BooleanExpression createNotEqual(Expression left, Expression right) {
+ return UnaryExpression.createNOT(createEqual(left, right));
+ }
+
+ public static BooleanExpression createEqual(Expression left, Expression right) {
+ checkEqualOperand(left);
+ checkEqualOperand(right);
+ checkEqualOperandCompatability(left, right);
+ return doCreateEqual(left, right);
+ }
+
+ @SuppressWarnings({"rawtypes"})
+ private static BooleanExpression doCreateEqual(Expression left, Expression right) {
+ return new ComparisonExpression(left, right) {
+
+ public Object evaluate(EvaluationContext context) throws Exception {
+ Object lv = left.evaluate(context);
+ Object rv = right.evaluate(context);
+
+ // If one of the values is null
+ if (lv == null ^ rv == null) {
+ if (lv == null) {
+ return null;
+ }
+ return Boolean.FALSE;
+ }
+ if (lv == rv || lv.equals(rv)) {
+ return Boolean.TRUE;
+ }
+ if (lv instanceof Comparable && rv instanceof Comparable) {
+ return compare((Comparable) lv, (Comparable) rv);
+ }
+ return Boolean.FALSE;
+ }
+
+ protected boolean asBoolean(int answer) {
+ return answer == 0;
+ }
+
+ public String getExpressionSymbol() {
+ return "==";
+ }
+ };
+ }
+
+ public static BooleanExpression createGreaterThan(final Expression left, final Expression right) {
+ checkLessThanOperand(left);
+ checkLessThanOperand(right);
+ return new ComparisonExpression(left, right) {
+ protected boolean asBoolean(int answer) {
+ return answer > 0;
+ }
+
+ public String getExpressionSymbol() {
+ return ">";
+ }
+ };
+ }
+
+ public static BooleanExpression createGreaterThanEqual(final Expression left, final Expression right) {
+ checkLessThanOperand(left);
+ checkLessThanOperand(right);
+ return new ComparisonExpression(left, right) {
+ protected boolean asBoolean(int answer) {
+ return answer >= 0;
+ }
+
+ public String getExpressionSymbol() {
+ return ">=";
+ }
+ };
+ }
+
+ public static BooleanExpression createLessThan(final Expression left, final Expression right) {
+ checkLessThanOperand(left);
+ checkLessThanOperand(right);
+ return new ComparisonExpression(left, right) {
+
+ protected boolean asBoolean(int answer) {
+ return answer < 0;
+ }
+
+ public String getExpressionSymbol() {
+ return "<";
+ }
+
+ };
+ }
+
+ public static BooleanExpression createLessThanEqual(final Expression left, final Expression right) {
+ checkLessThanOperand(left);
+ checkLessThanOperand(right);
+ return new ComparisonExpression(left, right) {
+
+ protected boolean asBoolean(int answer) {
+ return answer <= 0;
+ }
+
+ public String getExpressionSymbol() {
+ return "<=";
+ }
+ };
+ }
+
+ /**
+ * Only Numeric expressions can be used in >, >=, < or <= expressions.s
+ *
+ * @param expr
+ */
+ public static void checkLessThanOperand(Expression expr) {
+ if (expr instanceof ConstantExpression) {
+ Object value = ((ConstantExpression) expr).getValue();
+ if (value instanceof Number) {
+ return;
+ }
+
+ // Else it's boolean or a String..
+ throw new RuntimeException("Value '" + expr + "' cannot be compared.");
+ }
+ if (expr instanceof BooleanExpression) {
+ throw new RuntimeException("Value '" + expr + "' cannot be compared.");
+ }
+ }
+
+ /**
+ * Validates that the expression can be used in == or <> expression. Cannot
+ * not be NULL TRUE or FALSE litterals.
+ *
+ * @param expr
+ */
+ public static void checkEqualOperand(Expression expr) {
+ if (expr instanceof ConstantExpression) {
+ Object value = ((ConstantExpression) expr).getValue();
+ if (value == null) {
+ throw new RuntimeException("'" + expr + "' cannot be compared.");
+ }
+ }
+ }
+
+ /**
+ * @param left
+ * @param right
+ */
+ private static void checkEqualOperandCompatability(Expression left, Expression right) {
+ if (left instanceof ConstantExpression && right instanceof ConstantExpression) {
+ if (left instanceof BooleanExpression && !(right instanceof BooleanExpression)) {
+ throw new RuntimeException("'" + left + "' cannot be compared with '" + right + "'");
+ }
+ }
+ }
+
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ public Object evaluate(EvaluationContext context) throws Exception {
+ Comparable lv = (Comparable) left.evaluate(context);
+ if (lv == null) {
+ return null;
+ }
+ Comparable rv = (Comparable) right.evaluate(context);
+ if (rv == null) {
+ return null;
+ }
+ if (getExpressionSymbol().equals(">=") || getExpressionSymbol().equals(">")
+ || getExpressionSymbol().equals("<") || getExpressionSymbol().equals("<=")) {
+ Class extends Comparable> lc = lv.getClass();
+ Class extends Comparable> rc = rv.getClass();
+ if (lc == rc && lc == String.class) {
+ // Compare String is illegal
+ // first try to convert to double
+ try {
+ Comparable lvC = Double.valueOf((String) (Comparable) lv);
+ Comparable rvC = Double.valueOf((String) rv);
+
+ return compare(lvC, rvC);
+ } catch (Exception e) {
+ throw new RuntimeException("It's illegal to compare string by '>=', '>', '<', '<='. lv=" + lv + ", rv=" + rv, e);
+ }
+ }
+ }
+ return compare(lv, rv);
+ }
+
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ protected static int __compare(Comparable lv, Comparable rv, boolean convertStringExpressions) {
+ Class extends Comparable> lc = lv.getClass();
+ Class extends Comparable> rc = rv.getClass();
+ // If the the objects are not of the same type,
+ // try to convert up to allow the comparison.
+ if (lc != rc) {
+ try {
+ if (lc == Boolean.class) {
+ if (convertStringExpressions && rc == String.class) {
+ lv = Boolean.valueOf((String) lv).booleanValue();
+ } else {
+ return -1;
+ }
+ } else if (lc == Byte.class) {
+ if (rc == Short.class) {
+ lv = Short.valueOf(((Number) lv).shortValue());
+ } else if (rc == Integer.class) {
+ lv = Integer.valueOf(((Number) lv).intValue());
+ } else if (rc == Long.class) {
+ lv = Long.valueOf(((Number) lv).longValue());
+ } else if (rc == Float.class) {
+ lv = new Float(((Number) lv).floatValue());
+ } else if (rc == Double.class) {
+ lv = new Double(((Number) lv).doubleValue());
+ } else if (convertStringExpressions && rc == String.class) {
+ rv = Byte.valueOf((String) rv);
+ } else {
+ return -1;
+ }
+ } else if (lc == Short.class) {
+ if (rc == Integer.class) {
+ lv = Integer.valueOf(((Number) lv).intValue());
+ } else if (rc == Long.class) {
+ lv = Long.valueOf(((Number) lv).longValue());
+ } else if (rc == Float.class) {
+ lv = new Float(((Number) lv).floatValue());
+ } else if (rc == Double.class) {
+ lv = new Double(((Number) lv).doubleValue());
+ } else if (convertStringExpressions && rc == String.class) {
+ rv = Short.valueOf((String) rv);
+ } else {
+ return -1;
+ }
+ } else if (lc == Integer.class) {
+ if (rc == Long.class) {
+ lv = Long.valueOf(((Number) lv).longValue());
+ } else if (rc == Float.class) {
+ lv = new Float(((Number) lv).floatValue());
+ } else if (rc == Double.class) {
+ lv = new Double(((Number) lv).doubleValue());
+ } else if (convertStringExpressions && rc == String.class) {
+ rv = Integer.valueOf((String) rv);
+ } else {
+ return -1;
+ }
+ } else if (lc == Long.class) {
+ if (rc == Integer.class) {
+ rv = Long.valueOf(((Number) rv).longValue());
+ } else if (rc == Float.class) {
+ lv = new Float(((Number) lv).floatValue());
+ } else if (rc == Double.class) {
+ lv = new Double(((Number) lv).doubleValue());
+ } else if (convertStringExpressions && rc == String.class) {
+ rv = Long.valueOf((String) rv);
+ } else {
+ return -1;
+ }
+ } else if (lc == Float.class) {
+ if (rc == Integer.class) {
+ rv = new Float(((Number) rv).floatValue());
+ } else if (rc == Long.class) {
+ rv = new Float(((Number) rv).floatValue());
+ } else if (rc == Double.class) {
+ lv = new Double(((Number) lv).doubleValue());
+ } else if (convertStringExpressions && rc == String.class) {
+ rv = Float.valueOf((String) rv);
+ } else {
+ return -1;
+ }
+ } else if (lc == Double.class) {
+ if (rc == Integer.class) {
+ rv = new Double(((Number) rv).doubleValue());
+ } else if (rc == Long.class) {
+ rv = new Double(((Number) rv).doubleValue());
+ } else if (rc == Float.class) {
+ rv = new Float(((Number) rv).doubleValue());
+ } else if (convertStringExpressions && rc == String.class) {
+ rv = Double.valueOf((String) rv);
+ } else {
+ return -1;
+ }
+ } else if (convertStringExpressions && lc == String.class) {
+ if (rc == Boolean.class) {
+ lv = Boolean.valueOf((String) lv);
+ } else if (rc == Byte.class) {
+ lv = Byte.valueOf((String) lv);
+ } else if (rc == Short.class) {
+ lv = Short.valueOf((String) lv);
+ } else if (rc == Integer.class) {
+ lv = Integer.valueOf((String) lv);
+ } else if (rc == Long.class) {
+ lv = Long.valueOf((String) lv);
+ } else if (rc == Float.class) {
+ lv = Float.valueOf((String) lv);
+ } else if (rc == Double.class) {
+ lv = Double.valueOf((String) lv);
+ } else {
+ return -1;
+ }
+ } else {
+ return -1;
+ }
+ } catch (NumberFormatException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ return lv.compareTo(rv);
+ }
+
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ protected Boolean compare(Comparable lv, Comparable rv) {
+ return asBoolean(__compare(lv, rv, convertStringExpressions)) ? Boolean.TRUE : Boolean.FALSE;
+ }
+
+ protected abstract boolean asBoolean(int answer);
+
+ public boolean matches(EvaluationContext context) throws Exception {
+ Object object = evaluate(context);
+ return object != null && object == Boolean.TRUE;
+ }
+
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/ConstantExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/ConstantExpression.java
new file mode 100644
index 0000000000000000000000000000000000000000..ca70f51a8e13b4a62a42da9fdad5ffb7e0777d94
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/ConstantExpression.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter.expression;
+
+/**
+ * Represents a constant expression
+ *
+ * This class was taken from ActiveMQ org.apache.activemq.filter.ConstantExpression,
+ * but:
+ * 1. For long type constant, the range bound by java Long type;
+ * 2. For float type constant, the range bound by java Double type;
+ * 3. Remove Hex and Octal expression;
+ * 4. Add now expression to support to get current time.
+ *
+ */
+public class ConstantExpression implements Expression {
+
+ static class BooleanConstantExpression extends ConstantExpression implements BooleanExpression {
+ public BooleanConstantExpression(Object value) {
+ super(value);
+ }
+
+ public boolean matches(EvaluationContext context) throws Exception {
+ Object object = evaluate(context);
+ return object != null && object == Boolean.TRUE;
+ }
+ }
+
+ public static final BooleanConstantExpression NULL = new BooleanConstantExpression(null);
+ public static final BooleanConstantExpression TRUE = new BooleanConstantExpression(Boolean.TRUE);
+ public static final BooleanConstantExpression FALSE = new BooleanConstantExpression(Boolean.FALSE);
+
+ private Object value;
+
+ public ConstantExpression(Object value) {
+ this.value = value;
+ }
+
+ public static ConstantExpression createFromDecimal(String text) {
+
+ // Strip off the 'l' or 'L' if needed.
+ if (text.endsWith("l") || text.endsWith("L")) {
+ text = text.substring(0, text.length() - 1);
+ }
+
+ // only support Long.MIN_VALUE ~ Long.MAX_VALUE
+ Number value = new Long(text);
+// try {
+// value = new Long(text);
+// } catch (NumberFormatException e) {
+// // The number may be too big to fit in a long.
+// value = new BigDecimal(text);
+// }
+
+ long l = value.longValue();
+ if (Integer.MIN_VALUE <= l && l <= Integer.MAX_VALUE) {
+ value = Integer.valueOf(value.intValue());
+ }
+ return new ConstantExpression(value);
+ }
+
+ public static ConstantExpression createFloat(String text) {
+ Double value = new Double(text);
+ if (value > Double.MAX_VALUE) {
+ throw new RuntimeException(text + " is greater than " + Double.MAX_VALUE);
+ }
+ if (value < Double.MIN_VALUE) {
+ throw new RuntimeException(text + " is less than " + Double.MIN_VALUE);
+ }
+ return new ConstantExpression(value);
+ }
+
+ public static ConstantExpression createNow() {
+ return new NowExpression();
+ }
+
+ public Object evaluate(EvaluationContext context) throws Exception {
+ return value;
+ }
+
+ public Object getValue() {
+ return value;
+ }
+
+ /**
+ * @see Object#toString()
+ */
+ public String toString() {
+ Object value = getValue();
+ if (value == null) {
+ return "NULL";
+ }
+ if (value instanceof Boolean) {
+ return ((Boolean) value).booleanValue() ? "TRUE" : "FALSE";
+ }
+ if (value instanceof String) {
+ return encodeString((String) value);
+ }
+ return value.toString();
+ }
+
+ /**
+ * @see Object#hashCode()
+ */
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ /**
+ * @see Object#equals(Object)
+ */
+ public boolean equals(Object o) {
+
+ if (o == null || !this.getClass().equals(o.getClass())) {
+ return false;
+ }
+ return toString().equals(o.toString());
+
+ }
+
+ /**
+ * Encodes the value of string so that it looks like it would look like when
+ * it was provided in a selector.
+ *
+ * @return
+ */
+ public static String encodeString(String s) {
+ StringBuffer b = new StringBuffer();
+ b.append('\'');
+ for (int i = 0; i < s.length(); i++) {
+ char c = s.charAt(i);
+ if (c == '\'') {
+ b.append(c);
+ }
+ b.append(c);
+ }
+ b.append('\'');
+ return b.toString();
+ }
+
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/EmptyEvaluationContext.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/EmptyEvaluationContext.java
new file mode 100644
index 0000000000000000000000000000000000000000..52af2d0af2de68976e4fd352df1e100256d0a27e
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/EmptyEvaluationContext.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter.expression;
+
+import java.util.Map;
+
+/**
+ * Empty context.
+ */
+public class EmptyEvaluationContext implements EvaluationContext {
+ @Override
+ public Object get(String name) {
+ return null;
+ }
+
+ @Override
+ public Map keyValues() {
+ return null;
+ }
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/EvaluationContext.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/EvaluationContext.java
new file mode 100644
index 0000000000000000000000000000000000000000..094ef53ce36e583737cc9bcab2e570ef73b22b40
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/EvaluationContext.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter.expression;
+
+import java.util.Map;
+
+/**
+ * Context of evaluate expression.
+ *
+ * Compare to org.apache.activemq.filter.MessageEvaluationContext, this is just an interface.
+ */
+public interface EvaluationContext {
+
+ /**
+ * Get value by name from context
+ *
+ * @param name
+ * @return
+ */
+ Object get(String name);
+
+ /**
+ * Context variables.
+ *
+ * @return
+ */
+ Map keyValues();
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/Expression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/Expression.java
new file mode 100644
index 0000000000000000000000000000000000000000..3e6d9b33e6a2ade3757bc8fe285b27d93948e760
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/Expression.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter.expression;
+
+/**
+ * Interface of expression.
+ *
+ * This class was taken from ActiveMQ org.apache.activemq.filter.Expression,
+ * but the parameter is changed to an interface.
+ *
+ *
+ * @see org.apache.rocketmq.filter.expression.EvaluationContext
+ */
+public interface Expression {
+
+ /**
+ * Calculate express result with context.
+ *
+ * @param context context of evaluation
+ * @return the value of this expression
+ */
+ Object evaluate(EvaluationContext context) throws Exception;
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/LogicExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/LogicExpression.java
new file mode 100644
index 0000000000000000000000000000000000000000..1062bb8f51760386acd974341d613db021e63fa6
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/LogicExpression.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter.expression;
+
+/**
+ * A filter performing a comparison of two objects
+ *
+ * This class was taken from ActiveMQ org.apache.activemq.filter.LogicExpression,
+ *
+ */
+public abstract class LogicExpression extends BinaryExpression implements BooleanExpression {
+
+ /**
+ * @param left
+ * @param right
+ */
+ public LogicExpression(BooleanExpression left, BooleanExpression right) {
+ super(left, right);
+ }
+
+ public static BooleanExpression createOR(BooleanExpression lvalue, BooleanExpression rvalue) {
+ return new LogicExpression(lvalue, rvalue) {
+
+ public Object evaluate(EvaluationContext context) throws Exception {
+
+ Boolean lv = (Boolean) left.evaluate(context);
+ if (lv != null && lv.booleanValue()) {
+ return Boolean.TRUE;
+ }
+ Boolean rv = (Boolean) right.evaluate(context);
+ if (rv != null && rv.booleanValue()) {
+ return Boolean.TRUE;
+ }
+ if (lv == null || rv == null) {
+ return null;
+ }
+ return Boolean.FALSE;
+ }
+
+ public String getExpressionSymbol() {
+ return "||";
+ }
+ };
+ }
+
+ public static BooleanExpression createAND(BooleanExpression lvalue, BooleanExpression rvalue) {
+ return new LogicExpression(lvalue, rvalue) {
+
+ public Object evaluate(EvaluationContext context) throws Exception {
+
+ Boolean lv = (Boolean) left.evaluate(context);
+
+ if (lv != null && !lv.booleanValue()) {
+ return Boolean.FALSE;
+ }
+ Boolean rv = (Boolean) right.evaluate(context);
+ if (rv != null && !rv.booleanValue()) {
+ return Boolean.FALSE;
+ }
+ if (lv == null || rv == null) {
+ return null;
+ }
+ return Boolean.TRUE;
+ }
+
+ public String getExpressionSymbol() {
+ return "&&";
+ }
+ };
+ }
+
+ public abstract Object evaluate(EvaluationContext context) throws Exception;
+
+ public boolean matches(EvaluationContext context) throws Exception {
+ Object object = evaluate(context);
+ return object != null && object == Boolean.TRUE;
+ }
+
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/MQFilterException.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/MQFilterException.java
new file mode 100644
index 0000000000000000000000000000000000000000..676a17ba4910e3d20fd5af85eb635a3508c5a3e6
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/MQFilterException.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter.expression;
+
+/**
+ * Exception.
+ */
+public class MQFilterException extends Exception {
+ private static final long serialVersionUID = 1L;
+ private final int responseCode;
+ private final String errorMessage;
+
+ public MQFilterException(String errorMessage, Throwable cause) {
+ super(cause);
+ this.responseCode = -1;
+ this.errorMessage = errorMessage;
+ }
+
+ public MQFilterException(int responseCode, String errorMessage) {
+ this.responseCode = responseCode;
+ this.errorMessage = errorMessage;
+ }
+
+ public int getResponseCode() {
+ return responseCode;
+ }
+
+ public String getErrorMessage() {
+ return errorMessage;
+ }
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/NowExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/NowExpression.java
new file mode 100644
index 0000000000000000000000000000000000000000..d76caca9bd9befe05f3aa2d236ea2474a5f87c06
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/NowExpression.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter.expression;
+
+/**
+ * Current time expression.Just for test.
+ */
+public class NowExpression extends ConstantExpression {
+ public NowExpression() {
+ super("now");
+ }
+
+ @Override
+ public Object evaluate(EvaluationContext context) throws Exception {
+ return new Long(System.currentTimeMillis());
+ }
+
+ public Object getValue() {
+ return new Long(System.currentTimeMillis());
+ }
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/PropertyExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/PropertyExpression.java
new file mode 100644
index 0000000000000000000000000000000000000000..b9657b0464b22223ad167861c7ac06353a0347d0
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/PropertyExpression.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter.expression;
+
+/**
+ * Represents a property expression
+ *
+ * This class was taken from ActiveMQ org.apache.activemq.filter.PropertyExpression,
+ * but more simple and no transfer between expression and message property.
+ *
+ */
+public class PropertyExpression implements Expression {
+ private final String name;
+
+ public PropertyExpression(String name) {
+ this.name = name;
+ }
+
+ @Override
+ public Object evaluate(EvaluationContext context) throws Exception {
+ return context.get(name);
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ /**
+ * @see Object#toString()
+ */
+ @Override
+ public String toString() {
+ return name;
+ }
+
+ /**
+ * @see Object#hashCode()
+ */
+ @Override
+ public int hashCode() {
+ return name.hashCode();
+ }
+
+ /**
+ * @see Object#equals(Object)
+ */
+ @Override
+ public boolean equals(Object o) {
+
+ if (o == null || !this.getClass().equals(o.getClass())) {
+ return false;
+ }
+ return name.equals(((PropertyExpression) o).name);
+ }
+}
diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/UnaryExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/UnaryExpression.java
new file mode 100644
index 0000000000000000000000000000000000000000..0519f4d56d630e2a0ed0b88ad95c5da57589914c
--- /dev/null
+++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/UnaryExpression.java
@@ -0,0 +1,267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.filter.expression;
+
+import org.apache.rocketmq.filter.constant.UnaryType;
+
+import java.math.BigDecimal;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * An expression which performs an operation on two expression values
+ *
+ * This class was taken from ActiveMQ org.apache.activemq.filter.UnaryExpression,
+ * but:
+ * 1. remove XPath and XQuery expression;
+ * 2. Add constant UnaryType to distinguish different unary expression;
+ * 3. Extract UnaryInExpression to an independent class.
+ *