提交 58f1574b 编写于 作者: V vsair 提交者: dongeforever

[ROCKETMQ-121]Support message filtering based on SQL92 closes apache/incubator-rocketmq#82

上级 42f78c28
......@@ -48,6 +48,10 @@
<groupId>${project.groupId}</groupId>
<artifactId>rocketmq-srvutil</artifactId>
</dependency>
<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>rocketmq-filter</artifactId>
</dependency>
<dependency>
<groupId>ch.qos.logback</groupId>
<artifactId>logback-classic</artifactId>
......
......@@ -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() {
......
......@@ -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";
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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
}
......@@ -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<Channel> channels);
void handle(ConsumerGroupEvent event, String group, Object... args);
}
......@@ -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());
}
}
}
......
......@@ -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<Channel> 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<Channel> channels = (List<Channel>) 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<SubscriptionData> subscriptionDataList = (Collection<SubscriptionData>) args[0];
this.brokerController.getConsumerFilterManager().register(group, subscriptionDataList);
break;
default:
throw new RuntimeException("Unknown event " + event);
}
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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<ConsumerFilterData> filterDatas = consumerFilterManager.get(request.getTopic());
if (filterDatas == null || filterDatas.isEmpty()) {
return;
}
Iterator<ConsumerFilterData> 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);
}
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.<String>emptyList());
}
@Override
public int hashCode() {
return HashCodeBuilder.reflectionHashCode(this, Collections.<String>emptyList());
}
@Override
public String toString() {
return ReflectionToStringBuilder.toString(this, ToStringStyle.SHORT_PREFIX_STYLE);
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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<String/*Topic*/, FilterDataMapByTopic>
filterDataByTopic = new ConcurrentHashMap<String/*consumer group*/, FilterDataMapByTopic>(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<SubscriptionData> subList) {
for (SubscriptionData subscriptionData : subList) {
register(
subscriptionData.getTopic(),
consumerGroup,
subscriptionData.getSubString(),
subscriptionData.getExpressionType(),
subscriptionData.getSubVersion()
);
}
// make illegal topic dead.
Collection<ConsumerFilterData> groupFilterData = getByGroup(consumerGroup);
Iterator<ConsumerFilterData> 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<ConsumerFilterData> getByGroup(final String consumerGroup) {
Collection<ConsumerFilterData> ret = new HashSet<ConsumerFilterData>();
Iterator<FilterDataMapByTopic> topicIterator = this.filterDataByTopic.values().iterator();
while (topicIterator.hasNext()) {
FilterDataMapByTopic filterDataMapByTopic = topicIterator.next();
Iterator<ConsumerFilterData> 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<ConsumerFilterData> 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<Map.Entry<String, FilterDataMapByTopic>> topicIterator = this.filterDataByTopic.entrySet().iterator();
while (topicIterator.hasNext()) {
Map.Entry<String, FilterDataMapByTopic> filterDataMapByTopic = topicIterator.next();
Iterator<Map.Entry<String, ConsumerFilterData>> filterDataIterator
= filterDataMapByTopic.getValue().getGroupFilterData().entrySet().iterator();
while (filterDataIterator.hasNext()) {
Map.Entry<String, ConsumerFilterData> 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<String, FilterDataMapByTopic> getFilterDataByTopic() {
return filterDataByTopic;
}
public void setFilterDataByTopic(final ConcurrentHashMap<String, FilterDataMapByTopic> filterDataByTopic) {
this.filterDataByTopic = filterDataByTopic;
}
public static class FilterDataMapByTopic {
private ConcurrentHashMap<String/*consumer group*/, ConsumerFilterData>
groupFilterData = new ConcurrentHashMap<String, ConsumerFilterData>();
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<String, ConsumerFilterData> getGroupFilterData() {
return this.groupFilterData;
}
public void setGroupFilterData(final ConcurrentHashMap<String, ConsumerFilterData> groupFilterData) {
this.groupFilterData = groupFilterData;
}
public String getTopic() {
return topic;
}
public void setTopic(final String topic) {
this.topic = topic;
}
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.
* <br>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<String, String> 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<String, String> 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;
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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<String, String> properties) {
if (subscriptionData == null) {
return true;
}
if (subscriptionData.isClassFilterMode()) {
return true;
}
if (ExpressionType.isTagType(subscriptionData.getExpressionType())) {
return true;
}
ConsumerFilterData realFilterData = this.consumerFilterData;
Map<String, String> 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;
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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<String, String> properties;
public MessageEvaluationContext(Map<String, String> 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<String, Object> keyValues() {
if (properties == null) {
return null;
}
Map<String, Object> copy = new HashMap<String, Object>(properties.size(), 1);
for (String key : properties.keySet()) {
copy.put(key, properties.get(key));
}
return copy;
}
}
......@@ -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<String, String> properties) {
this.pullRequestHoldService.notifyMessageArriving(topic, queueId, logicOffset, tagsCode,
msgStoreTime, filterBitMap, properties);
}
}
......@@ -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;
}
}
......@@ -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<String/* topic@queueId */, ManyPullRequest> pullRequestTable =
new ConcurrentHashMap<String, ManyPullRequest>(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<String, String> 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());
......
......@@ -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) {
......
......@@ -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<CommitLogDispatcher> getDispatcherList() {
return next.getDispatcherList();
}
@Override
public ConsumeQueue getConsumeQueue(String topic, int queueId) {
return next.getConsumeQueue(topic, queueId);
}
}
......@@ -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<ConsumeQueueData> 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;
}
}
......@@ -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;
}
}
......@@ -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;
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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<String, String> properties = new HashMap<String, String>(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<String, String> properties = new HashMap<String, String>(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<String, String> properties = new HashMap<String, String>(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<ConsumerFilterData> 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();
}
}
}
}
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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<SubscriptionData> 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<ConsumerFilterData> filterDatas = filterManager.getByGroup("CID_0");
assertThat(filterDatas).isNotNull();
assertThat(filterDatas.size()).isEqualTo(10);
Iterator<ConsumerFilterData> 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<ConsumerFilterData> 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<ConsumerFilterData> 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();
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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<String, String> 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<MessageExtBrokerInner> putMsg(DefaultMessageStore master, int topicCount, int msgCountPerTopic) throws Exception {
List<MessageExtBrokerInner> msgs = new ArrayList<MessageExtBrokerInner>();
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<MessageExtBrokerInner> filtered(List<MessageExtBrokerInner> msgs, ConsumerFilterData filterData) {
List<MessageExtBrokerInner> filteredMsgs = new ArrayList<MessageExtBrokerInner>();
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<MessageExtBrokerInner> 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<MessageExtBrokerInner> 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<MessageExtBrokerInner> 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<MessageExtBrokerInner> 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<MessageExtBrokerInner> 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);
}
}
}
......@@ -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<ConsumeMessageHook> 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);
......
......@@ -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
......
......@@ -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.
* <p>
* 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}.
* </p>
* <p/>
* <p>
* Choose Tag: {@link MessageSelector#byTag(java.lang.String)}
* </p>
* <p/>
* <p>
* Choose SQL92: {@link MessageSelector#bySql(java.lang.String)}
* </p>
*
* @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
*
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.
* <p>
* Now, support:
* <li>Tag: {@link org.apache.rocketmq.common.filter.ExpressionType#TAG}
* </li>
* <li>SQL92: {@link org.apache.rocketmq.common.filter.ExpressionType#SQL92}
* </li>
* </p>
*/
public class MessageSelector {
/**
* @see org.apache.rocketmq.common.filter.ExpressionType
*/
private String type;
/**
* expression content.
*/
private String expression;
private MessageSelector(String type, String expression) {
this.type = type;
this.expression = expression;
}
/**
* Use SLQ92 to select message.
*
* @param sql if null or empty, will be treated as select all message.
* @return
*/
public static MessageSelector bySql(String sql) {
return new MessageSelector(ExpressionType.SQL92, sql);
}
/**
* Use tag to select message.
*
* @param tag if null or empty or "*", will be treated as select all message.
* @return
*/
public static MessageSelector byTag(String tag) {
return new MessageSelector(ExpressionType.TAG, tag);
}
public String getExpressionType() {
return type;
}
public String getExpression() {
return expression;
}
}
......@@ -19,10 +19,18 @@ package org.apache.rocketmq.client.impl;
public class FindBrokerResult {
private final String brokerAddr;
private final boolean slave;
private final int brokerVersion;
public FindBrokerResult(String brokerAddr, boolean slave) {
this.brokerAddr = brokerAddr;
this.slave = slave;
this.brokerVersion = 0;
}
public FindBrokerResult(String brokerAddr, boolean slave, int brokerVersion) {
this.brokerAddr = brokerAddr;
this.slave = slave;
this.brokerVersion = brokerVersion;
}
public String getBrokerAddr() {
......@@ -32,4 +40,8 @@ public class FindBrokerResult {
public boolean isSlave() {
return slave;
}
public int getBrokerVersion() {
return brokerVersion;
}
}
......@@ -59,6 +59,7 @@ import org.apache.rocketmq.common.namesrv.TopAddressing;
import org.apache.rocketmq.common.protocol.RequestCode;
import org.apache.rocketmq.common.protocol.ResponseCode;
import org.apache.rocketmq.common.protocol.body.BrokerStatsData;
import org.apache.rocketmq.common.protocol.body.CheckClientRequestBody;
import org.apache.rocketmq.common.protocol.body.ClusterInfo;
import org.apache.rocketmq.common.protocol.body.ConsumeMessageDirectlyResult;
import org.apache.rocketmq.common.protocol.body.ConsumeStatsList;
......@@ -70,6 +71,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;
......@@ -103,6 +105,7 @@ import org.apache.rocketmq.common.protocol.header.GetTopicStatsInfoRequestHeader
import org.apache.rocketmq.common.protocol.header.GetTopicsByClusterRequestHeader;
import org.apache.rocketmq.common.protocol.header.PullMessageRequestHeader;
import org.apache.rocketmq.common.protocol.header.PullMessageResponseHeader;
import org.apache.rocketmq.common.protocol.header.QueryConsumeQueueRequestHeader;
import org.apache.rocketmq.common.protocol.header.QueryConsumeTimeSpanRequestHeader;
import org.apache.rocketmq.common.protocol.header.QueryConsumerOffsetRequestHeader;
import org.apache.rocketmq.common.protocol.header.QueryConsumerOffsetResponseHeader;
......@@ -129,6 +132,7 @@ import org.apache.rocketmq.common.protocol.header.namesrv.PutKVConfigRequestHead
import org.apache.rocketmq.common.protocol.header.namesrv.WipeWritePermOfBrokerRequestHeader;
import org.apache.rocketmq.common.protocol.header.namesrv.WipeWritePermOfBrokerResponseHeader;
import org.apache.rocketmq.common.protocol.heartbeat.HeartbeatData;
import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
import org.apache.rocketmq.common.protocol.route.TopicRouteData;
import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig;
import org.apache.rocketmq.remoting.InvokeCallback;
......@@ -168,7 +172,7 @@ public class MQClientAPIImpl {
public MQClientAPIImpl(final NettyClientConfig nettyClientConfig, final ClientRemotingProcessor clientRemotingProcessor,
RPCHook rpcHook, final ClientConfig clientConfig) {
this.clientConfig = clientConfig;
topAddressing = new TopAddressing(MixAll.WS_ADDR, clientConfig.getUnitName());
topAddressing = new TopAddressing(MixAll.getWSAddr(), clientConfig.getUnitName());
this.remotingClient = new NettyRemotingClient(nettyClientConfig, null);
this.clientRemotingProcessor = clientRemotingProcessor;
......@@ -843,7 +847,7 @@ public class MQClientAPIImpl {
this.remotingClient.invokeOneway(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis);
}
public void sendHearbeat(//
public int sendHearbeat(//
final String addr, //
final HeartbeatData heartbeatData, //
final long timeoutMillis//
......@@ -855,7 +859,7 @@ public class MQClientAPIImpl {
assert response != null;
switch (response.getCode()) {
case ResponseCode.SUCCESS: {
return;
return response.getVersion();
}
default:
break;
......@@ -2024,4 +2028,51 @@ public class MQClientAPIImpl {
return configMap;
}
public QueryConsumeQueueResponseBody queryConsumeQueue(final String brokerAddr, final String topic, final int queueId,
final long index, final int count, final String consumerGroup,
final long timeoutMillis) throws InterruptedException,
RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException {
QueryConsumeQueueRequestHeader requestHeader = new QueryConsumeQueueRequestHeader();
requestHeader.setTopic(topic);
requestHeader.setQueueId(queueId);
requestHeader.setIndex(index);
requestHeader.setCount(count);
requestHeader.setConsumerGroup(consumerGroup);
RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_CONSUME_QUEUE, requestHeader);
RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis);
assert response != null;
if (ResponseCode.SUCCESS == response.getCode()) {
return QueryConsumeQueueResponseBody.decode(response.getBody(), QueryConsumeQueueResponseBody.class);
}
throw new MQClientException(response.getCode(), response.getRemark());
}
public void checkClientInBroker(final String brokerAddr, final String consumerGroup,
final String clientId, final SubscriptionData subscriptionData,
final long timeoutMillis)
throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException,
RemotingConnectException, MQClientException {
RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CHECK_CLIENT_CONFIG, null);
CheckClientRequestBody requestBody = new CheckClientRequestBody();
requestBody.setClientId(clientId);
requestBody.setGroup(consumerGroup);
requestBody.setSubscriptionData(subscriptionData);
request.setBody(requestBody.encode());
RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis);
assert response != null;
if (ResponseCode.SUCCESS != response.getCode()) {
throw new MQClientException(response.getCode(), response.getRemark());
}
}
}
\ No newline at end of file
......@@ -30,6 +30,7 @@ import java.util.concurrent.ConcurrentHashMap;
import org.apache.rocketmq.client.QueryResult;
import org.apache.rocketmq.client.Validators;
import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
import org.apache.rocketmq.client.consumer.MessageSelector;
import org.apache.rocketmq.client.consumer.PullCallback;
import org.apache.rocketmq.client.consumer.PullResult;
import org.apache.rocketmq.client.consumer.listener.MessageListener;
......@@ -405,15 +406,16 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
this.pullAPIWrapper.pullKernelImpl(//
pullRequest.getMessageQueue(), // 1
subExpression, // 2
subscriptionData.getSubVersion(), // 3
pullRequest.getNextOffset(), // 4
this.defaultMQPushConsumer.getPullBatchSize(), // 5
sysFlag, // 6
commitOffsetValue, // 7
BROKER_SUSPEND_MAX_TIME_MILLIS, // 8
CONSUMER_TIMEOUT_MILLIS_WHEN_SUSPEND, // 9
CommunicationMode.ASYNC, // 10
pullCallback// 11
subscriptionData.getExpressionType(), // 3
subscriptionData.getSubVersion(), // 4
pullRequest.getNextOffset(), // 5
this.defaultMQPushConsumer.getPullBatchSize(), // 6
sysFlag, // 7
commitOffsetValue, // 8
BROKER_SUSPEND_MAX_TIME_MILLIS, // 9
CONSUMER_TIMEOUT_MILLIS_WHEN_SUSPEND, // 10
CommunicationMode.ASYNC, // 11
pullCallback // 12
);
} catch (Exception e) {
log.error("pullKernelImpl exception", e);
......@@ -615,6 +617,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
}
this.updateTopicSubscribeInfoWhenSubscriptionChanged();
this.mQClientFactory.checkClientInBroker();
this.mQClientFactory.sendHeartbeatToAllBrokerWithLock();
this.mQClientFactory.rebalanceImmediately();
}
......@@ -836,6 +839,25 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
}
}
public void subscribe(final String topic, final MessageSelector messageSelector) throws MQClientException {
try {
if (messageSelector == null) {
subscribe(topic, SubscriptionData.SUB_ALL);
return;
}
SubscriptionData subscriptionData = FilterAPI.build(topic,
messageSelector.getExpression(), messageSelector.getExpressionType());
this.rebalanceImpl.getSubscriptionInner().put(topic, subscriptionData);
if (this.mQClientFactory != null) {
this.mQClientFactory.sendHeartbeatToAllBrokerWithLock();
}
} catch (Exception e) {
throw new MQClientException("subscription exception", e);
}
}
public void suspend() {
this.pause = true;
log.info("suspend this consumer, {}", this.defaultMQPushConsumer.getConsumerGroup());
......
......@@ -33,7 +33,9 @@ import org.apache.rocketmq.client.impl.CommunicationMode;
import org.apache.rocketmq.client.impl.FindBrokerResult;
import org.apache.rocketmq.client.impl.factory.MQClientInstance;
import org.apache.rocketmq.client.log.ClientLogger;
import org.apache.rocketmq.common.MQVersion;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.filter.ExpressionType;
import org.apache.rocketmq.common.message.MessageAccessor;
import org.apache.rocketmq.common.message.MessageConst;
import org.apache.rocketmq.common.message.MessageDecoder;
......@@ -135,6 +137,7 @@ public class PullAPIWrapper {
public PullResult pullKernelImpl(
final MessageQueue mq,
final String subExpression,
final String expressionType,
final long subVersion,
final long offset,
final int maxNums,
......@@ -156,6 +159,14 @@ public class PullAPIWrapper {
}
if (findBrokerResult != null) {
{
// check version
if (!ExpressionType.isTagType(expressionType)
&& findBrokerResult.getBrokerVersion() < MQVersion.Version.V4_1_0_SNAPSHOT.ordinal()) {
throw new MQClientException("The broker[" + mq.getBrokerName() + ", "
+ findBrokerResult.getBrokerVersion() + "] does not upgrade to support for filter message by " + expressionType, null);
}
}
int sysFlagInner = sysFlag;
if (findBrokerResult.isSlave()) {
......@@ -173,6 +184,7 @@ public class PullAPIWrapper {
requestHeader.setSuspendTimeoutMillis(brokerSuspendMaxTimeMillis);
requestHeader.setSubscription(subExpression);
requestHeader.setSubVersion(subVersion);
requestHeader.setExpressionType(expressionType);
String brokerAddr = findBrokerResult.getBrokerAddr();
if (PullSysFlag.hasClassFilterFlag(sysFlagInner)) {
......@@ -192,6 +204,34 @@ public class PullAPIWrapper {
throw new MQClientException("The broker[" + mq.getBrokerName() + "] not exist", null);
}
public PullResult pullKernelImpl(
final MessageQueue mq,
final String subExpression,
final long subVersion,
final long offset,
final int maxNums,
final int sysFlag,
final long commitOffset,
final long brokerSuspendMaxTimeMillis,
final long timeoutMillis,
final CommunicationMode communicationMode,
final PullCallback pullCallback
) throws MQClientException, RemotingException, MQBrokerException, InterruptedException {
return pullKernelImpl(
mq,
subExpression,
ExpressionType.TAG,
subVersion, offset,
maxNums,
sysFlag,
commitOffset,
brokerSuspendMaxTimeMillis,
timeoutMillis,
communicationMode,
pullCallback
);
}
public long recalculatePullFromWhichNode(final MessageQueue mq) {
if (this.isConnectBrokerByUser()) {
return this.defaultBrokerId;
......
......@@ -61,6 +61,7 @@ import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.ServiceState;
import org.apache.rocketmq.common.UtilAll;
import org.apache.rocketmq.common.constant.PermName;
import org.apache.rocketmq.common.filter.ExpressionType;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.common.message.MessageQueue;
import org.apache.rocketmq.common.protocol.body.ConsumeMessageDirectlyResult;
......@@ -98,6 +99,8 @@ public class MQClientInstance {
private final Lock lockHeartbeat = new ReentrantLock();
private final ConcurrentHashMap<String/* Broker Name */, HashMap<Long/* brokerId */, String/* address */>> brokerAddrTable =
new ConcurrentHashMap<String, HashMap<Long, String>>();
private final ConcurrentHashMap<String/* Broker Name */, HashMap<String/* address */, Integer>> brokerVersionTable =
new ConcurrentHashMap<String, HashMap<String, Integer>>();
private final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() {
@Override
public Thread newThread(Runnable r) {
......@@ -404,6 +407,44 @@ public class MQClientInstance {
}
}
public void checkClientInBroker() throws MQClientException {
Iterator<Entry<String, MQConsumerInner>> it = this.consumerTable.entrySet().iterator();
while (it.hasNext()) {
Entry<String, MQConsumerInner> entry = it.next();
Set<SubscriptionData> subscriptionInner = entry.getValue().subscriptions();
if (subscriptionInner == null || subscriptionInner.isEmpty()) {
return;
}
for (SubscriptionData subscriptionData : subscriptionInner) {
if (ExpressionType.isTagType(subscriptionData.getExpressionType())) {
continue;
}
// may need to check one broker every cluster...
// assume that the configs of every broker in cluster are the the same.
String addr = findBrokerAddrByTopic(subscriptionData.getTopic());
if (addr != null) {
try {
this.getMQClientAPIImpl().checkClientInBroker(
addr, entry.getKey(), this.clientId, subscriptionData, 3 * 1000
);
} catch (Exception e) {
if (e instanceof MQClientException) {
throw (MQClientException) e;
} else {
throw new MQClientException("Check client in broker error, maybe because you use "
+ subscriptionData.getExpressionType() + " to filter message, but server has not been upgraded to support!"
+ "This error would not affect the launch of consumer, but may has impact on message receiving if you " +
"have use the new features which are not supported by server, please check the log!", e);
}
}
}
}
}
}
public void sendHeartbeatToAllBrokerWithLock() {
if (this.lockHeartbeat.tryLock()) {
try {
......@@ -493,7 +534,11 @@ public class MQClientInstance {
}
try {
this.mQClientAPIImpl.sendHearbeat(addr, heartbeatData, 3000);
int version = this.mQClientAPIImpl.sendHearbeat(addr, heartbeatData, 3000);
if (!this.brokerVersionTable.containsKey(brokerName)) {
this.brokerVersionTable.put(brokerName, new HashMap<String, Integer>(4));
}
this.brokerVersionTable.get(brokerName).put(addr, version);
if (times % 20 == 0) {
log.info("send heart beat to broker[{} {} {}] success", brokerName, id, addr);
log.info(heartbeatData.toString());
......@@ -943,7 +988,7 @@ public class MQClientInstance {
}
if (found) {
return new FindBrokerResult(brokerAddr, slave);
return new FindBrokerResult(brokerAddr, slave, findBrokerVersion(brokerName, brokerAddr));
}
return null;
......@@ -982,12 +1027,21 @@ public class MQClientInstance {
}
if (found) {
return new FindBrokerResult(brokerAddr, slave);
return new FindBrokerResult(brokerAddr, slave, findBrokerVersion(brokerName, brokerAddr));
}
return null;
}
public int findBrokerVersion(String brokerName, String brokerAddr) {
if (this.brokerVersionTable.containsKey(brokerName)) {
if (this.brokerVersionTable.get(brokerName).containsKey(brokerAddr)) {
return this.brokerVersionTable.get(brokerName).get(brokerAddr);
}
}
return 0;
}
public List<String> findConsumerIdList(final String topic, final String group) {
String brokerAddr = this.findBrokerAddrByTopic(topic);
if (null == brokerAddr) {
......
......@@ -99,6 +99,25 @@ public class BrokerConfig {
private boolean traceOn = true;
// Switch of filter bit map calculation.
// If switch on:
// 1. Calculate filter bit map when construct queue.
// 2. Filter bit map will be saved to consume queue extend file if allowed.
private boolean enableCalcFilterBitMap = false;
// Expect num of consumers will use filter.
private int expectConsumerNumUseFilter = 32;
// Error rate of bloom filter, 1~100.
private int maxErrorRateOfBloomFilter = 20;
//how long to clean filter data after dead.Default: 24h
private long filterDataCleanTimeSpan = 24 * 3600 * 1000;
// whether do filter when retry.
private boolean filterSupportRetry = false;
private boolean enablePropertyFilter = false;
public static String localHostName() {
try {
return InetAddress.getLocalHost().getHostName();
......@@ -484,4 +503,52 @@ public class BrokerConfig {
public void setCommercialBaseCount(int commercialBaseCount) {
this.commercialBaseCount = commercialBaseCount;
}
public boolean isEnableCalcFilterBitMap() {
return enableCalcFilterBitMap;
}
public void setEnableCalcFilterBitMap(boolean enableCalcFilterBitMap) {
this.enableCalcFilterBitMap = enableCalcFilterBitMap;
}
public int getExpectConsumerNumUseFilter() {
return expectConsumerNumUseFilter;
}
public void setExpectConsumerNumUseFilter(int expectConsumerNumUseFilter) {
this.expectConsumerNumUseFilter = expectConsumerNumUseFilter;
}
public int getMaxErrorRateOfBloomFilter() {
return maxErrorRateOfBloomFilter;
}
public void setMaxErrorRateOfBloomFilter(int maxErrorRateOfBloomFilter) {
this.maxErrorRateOfBloomFilter = maxErrorRateOfBloomFilter;
}
public long getFilterDataCleanTimeSpan() {
return filterDataCleanTimeSpan;
}
public void setFilterDataCleanTimeSpan(long filterDataCleanTimeSpan) {
this.filterDataCleanTimeSpan = filterDataCleanTimeSpan;
}
public boolean isFilterSupportRetry() {
return filterSupportRetry;
}
public void setFilterSupportRetry(boolean filterSupportRetry) {
this.filterSupportRetry = filterSupportRetry;
}
public boolean isEnablePropertyFilter() {
return enablePropertyFilter;
}
public void setEnablePropertyFilter(boolean enablePropertyFilter) {
this.enablePropertyFilter = enablePropertyFilter;
}
}
......@@ -55,8 +55,8 @@ public class MixAll {
public static final String DEFAULT_NAMESRV_ADDR_LOOKUP = "jmenv.tbsite.net";
public static final String WS_DOMAIN_NAME = System.getProperty("rocketmq.namesrv.domain", DEFAULT_NAMESRV_ADDR_LOOKUP);
public static final String WS_DOMAIN_SUBGROUP = System.getProperty("rocketmq.namesrv.domain.subgroup", "nsaddr");
// http://jmenv.tbsite.net:8080/rocketmq/nsaddr
public static final String WS_ADDR = "http://" + WS_DOMAIN_NAME + ":8080/rocketmq/" + WS_DOMAIN_SUBGROUP;
// // http://jmenv.tbsite.net:8080/rocketmq/nsaddr
// public static final String WS_ADDR = "http://" + WS_DOMAIN_NAME + ":8080/rocketmq/" + WS_DOMAIN_SUBGROUP;
public static final String DEFAULT_TOPIC = "TBW102";
public static final String BENCHMARK_TOPIC = "BenchmarkTest";
public static final String DEFAULT_PRODUCER_GROUP = "DEFAULT_PRODUCER";
......@@ -89,6 +89,16 @@ public class MixAll {
public static final String DEFAULT_TRACE_REGION_ID = "DefaultRegion";
public static final String CONSUME_CONTEXT_TYPE = "ConsumeContextType";
public static String getWSAddr() {
String wsDomainName = System.getProperty("rocketmq.namesrv.domain", DEFAULT_NAMESRV_ADDR_LOOKUP);
String wsDomainSubgroup = System.getProperty("rocketmq.namesrv.domain.subgroup", "nsaddr");
String wsAddr = "http://" + wsDomainName + ":8080/rocketmq/" + wsDomainSubgroup;
if (wsDomainName.indexOf(":") > 0) {
wsAddr = "http://" + wsDomainName + "/rocketmq/" + wsDomainSubgroup;
}
return wsAddr;
}
public static String getRetryTopic(final String consumerGroup) {
return RETRY_GROUP_TOPIC_PREFIX + consumerGroup;
}
......
......@@ -34,4 +34,5 @@ public class LoggerName {
public static final String DUPLICATION_LOGGER_NAME = "RocketmqDuplication";
public static final String PROTECTION_LOGGER_NAME = "RocketmqProtection";
public static final String WATER_MARK_LOGGER_NAME = "RocketmqWaterMark";
public static final String FILTER_LOGGER_NAME = "RocketmqFilter";
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.filter;
public class ExpressionType {
/**
* <ul>
* Keywords:
* <li>{@code AND, OR, NOT, BETWEEN, IN, TRUE, FALSE, IS, NULL}</li>
* </ul>
* <p/>
* <ul>
* Data type:
* <li>Boolean, like: TRUE, FALSE</li>
* <li>String, like: 'abc'</li>
* <li>Decimal, like: 123</li>
* <li>Float number, like: 3.1415</li>
* </ul>
* <p/>
* <ul>
* Grammar:
* <li>{@code AND, OR}</li>
* <li>{@code >, >=, <, <=, =}</li>
* <li>{@code BETWEEN A AND B}, equals to {@code >=A AND <=B}</li>
* <li>{@code NOT BETWEEN A AND B}, equals to {@code >B OR <A}</li>
* <li>{@code IN ('a', 'b')}, equals to {@code ='a' OR ='b'}, this operation only support String type.</li>
* <li>{@code IS NULL}, {@code IS NOT NULL}, check parameter whether is null, or not.</li>
* <li>{@code =TRUE}, {@code =FALSE}, check parameter whether is true, or false.</li>
* </ul>
* <p/>
* <p>
* Example:
* (a > 10 AND a < 100) OR (b IS NOT NULL AND b=TRUE)
* </p>
*/
public static final String SQL92 = "SQL92";
/**
* Only support or operation such as
* "tag1 || tag2 || tag3", <br>
* 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;
}
}
......@@ -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;
}
}
......@@ -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);
}
......
......@@ -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<String, String> 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<String, String> map = string2messageProperties(propertiesString);
return map;
}
return null;
}
public static MessageExt decode(java.nio.ByteBuffer byteBuffer) {
return decode(byteBuffer, true, true, false);
}
......
......@@ -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);
......
......@@ -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;
}
......@@ -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;
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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;
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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 + '\'' +
'}';
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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<ConsumeQueueData> 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<ConsumeQueueData> getQueueData() {
return queueData;
}
public void setQueueData(List<ConsumeQueueData> 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;
}
}
......@@ -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;
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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 {
}
}
......@@ -32,6 +32,7 @@ public class SubscriptionData implements Comparable<SubscriptionData> {
private Set<String> tagsSet = new HashSet<String>();
private Set<Integer> codeSet = new HashSet<Integer>();
private long subVersion = System.currentTimeMillis();
private String expressionType;
@JSONField(serialize = false)
private String filterClassSource;
......@@ -102,6 +103,14 @@ public class SubscriptionData implements Comparable<SubscriptionData> {
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<SubscriptionData> {
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<SubscriptionData> {
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<SubscriptionData> {
public String toString() {
return "SubscriptionData [classFilterMode=" + classFilterMode + ", topic=" + topic + ", subString="
+ subString + ", tagsSet=" + tagsSet + ", codeSet=" + codeSet + ", subVersion=" + subVersion
+ "]";
+ ", expressionType=" + expressionType + "]";
}
@Override
......
......@@ -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();
}
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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<String, String> 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"));
}
}
......@@ -222,6 +222,29 @@
<appender-ref ref="RocketmqRebalanceLockAppender_inner"/>
</appender>
<appender name="RocketmqFilterAppender_inner"
class="ch.qos.logback.core.rolling.RollingFileAppender">
<file>${user.home}/logs/rocketmqlogs/filter.log</file>
<append>true</append>
<rollingPolicy class="ch.qos.logback.core.rolling.FixedWindowRollingPolicy">
<fileNamePattern>${user.home}/logs/rocketmqlogs/otherdays/filter.%i.log
</fileNamePattern>
<minIndex>1</minIndex>
<maxIndex>10</maxIndex>
</rollingPolicy>
<triggeringPolicy
class="ch.qos.logback.core.rolling.SizeBasedTriggeringPolicy">
<maxFileSize>100MB</maxFileSize>
</triggeringPolicy>
<encoder>
<pattern>%d{yyy-MM-dd HH:mm:ss,GMT+8} %p %t - %m%n</pattern>
<charset class="java.nio.charset.Charset">UTF-8</charset>
</encoder>
</appender>
<appender name="RocketmqFilterAppender" class="ch.qos.logback.classic.AsyncAppender">
<appender-ref ref="RocketmqFilterAppender_inner"/>
</appender>
<appender name="RocketmqStatsAppender"
class="ch.qos.logback.core.rolling.RollingFileAppender">
<file>${user.home}/logs/rocketmqlogs/stats.log</file>
......@@ -321,6 +344,11 @@
<appender-ref ref="RocketmqCommercialAppender"/>
</logger>
<logger name="RocketmqFilter" additivity="false">
<level value="INFO"/>
<appender-ref ref="RocketmqFilterAppender"/>
</logger>
<root>
<level value="INFO"/>
<appender-ref ref="DefaultAppender"/>
......
......@@ -67,6 +67,7 @@
<include>org.apache.rocketmq:rocketmq-namesrv</include>
<include>org.apache.rocketmq:rocketmq-filtersrv</include>
<include>org.apache.rocketmq:rocketmq-example</include>
<include>org.apache.rocketmq:rocketmq-filter</include>
</includes>
<binaries>
<outputDirectory>lib/</outputDirectory>
......
......@@ -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;
}
......
......@@ -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(),
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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<MessageExt> 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");
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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();
}
}
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ Licensed to the Apache Software Foundation (ASF) under one or more
~ contributor license agreements. See the NOTICE file distributed with
~ this work for additional information regarding copyright ownership.
~ The ASF licenses this file to You under the Apache License, Version 2.0
~ (the "License"); you may not use this file except in compliance with
~ the License. You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing, software
~ distributed under the License is distributed on an "AS IS" BASIS,
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
~ See the License for the specific language governing permissions and
~ limitations under the License.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>rocketmq-all</artifactId>
<groupId>org.apache.rocketmq</groupId>
<version>4.1.0-incubating-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<packaging>jar</packaging>
<artifactId>rocketmq-filter</artifactId>
<name>rocketmq-filter ${project.version}</name>
<dependencies>
<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>rocketmq-common</artifactId>
</dependency>
<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>rocketmq-srvutil</artifactId>
</dependency>
</dependencies>
</project>
\ No newline at end of file
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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<String, FilterSpi> FILTER_SPI_HOLDER = new HashMap<String, FilterSpi>(4);
static {
FilterFactory.INSTANCE.register(new SqlFilter());
}
/**
* Register a filter.
* <br>
* Note:
* <li>1. Filter registered will be used in broker server, so take care of it's reliability and performance.</li>
*
* @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);
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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();
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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}.
* <p/>
* <p>
* Do not use this filter directly.Use {@link FilterFactory#get} to select a filter.
* </p>
*/
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;
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.
* <p>
* This class was taken from ActiveMQ org.apache.activemq.filter.BinaryExpression,
* </p>
*/
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;
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.
* <p>
* This class was taken from ActiveMQ org.apache.activemq.filter.BooleanExpression,
* but the parameter is changed to an interface.
* </p>
*
* @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;
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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
* <p>
* 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.
* </p>
*/
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();
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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<String, Object> keyValues() {
return null;
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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<String, Object> keyValues();
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.
* <p>
* This class was taken from ActiveMQ org.apache.activemq.filter.Expression,
* but the parameter is changed to an interface.
* </p>
*
* @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;
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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
* <p>
* This class was taken from ActiveMQ org.apache.activemq.filter.LogicExpression,
* </p>
*/
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;
}
}
此差异已折叠。
......@@ -41,5 +41,9 @@
<groupId>commons-cli</groupId>
<artifactId>commons-cli</artifactId>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
</dependencies>
</project>
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册