From 9eeb2f7e4e321c2f4688a3d3470731c0314c7cf3 Mon Sep 17 00:00:00 2001 From: vsair Date: Fri, 21 Apr 2017 18:17:58 +0800 Subject: [PATCH] [ROCKETMQ-121]Support message filtering based on SQL92 closes apache/incubator-rocketmq#82 --- broker/pom.xml | 4 + .../rocketmq/broker/BrokerController.java | 27 + .../broker/BrokerPathConfigHelper.java | 3 + .../broker/client/ConsumerGroupEvent.java | 33 + .../client/ConsumerIdsChangeListener.java | 6 +- .../broker/client/ConsumerManager.java | 11 +- .../DefaultConsumerIdsChangeListener.java | 37 +- .../filter/CommitLogDispatcherCalcBitMap.java | 110 ++ .../broker/filter/ConsumerFilterData.java | 151 ++ .../broker/filter/ConsumerFilterManager.java | 471 ++++++ .../ExpressionForRetryMessageFilter.java | 97 ++ .../filter/ExpressionMessageFilter.java | 162 ++ .../filter/MessageEvaluationContext.java | 58 + .../NotifyMessageArrivingListener.java | 8 +- .../broker/longpolling/PullRequest.java | 10 +- .../longpolling/PullRequestHoldService.java | 19 +- .../rocketmq/broker/out/BrokerOuterAPI.java | 2 +- .../plugin/AbstractPluginMessageStore.java | 18 +- .../processor/AdminBrokerProcessor.java | 91 ++ .../processor/ClientManageProcessor.java | 44 + .../processor/PullMessageProcessor.java | 59 +- .../CommitLogDispatcherCalcBitMapTest.java | 192 +++ .../filter/ConsumerFilterManagerTest.java | 291 ++++ .../filter/MessageStoreWithFilterTest.java | 392 +++++ .../processor/PullMessageProcessorTest.java | 9 +- .../consumer/DefaultMQPushConsumer.java | 15 + .../client/consumer/MQPushConsumer.java | 21 + .../client/consumer/MessageSelector.java | 77 + .../client/impl/FindBrokerResult.java | 12 + .../rocketmq/client/impl/MQClientAPIImpl.java | 57 +- .../consumer/DefaultMQPushConsumerImpl.java | 40 +- .../client/impl/consumer/PullAPIWrapper.java | 40 + .../client/impl/factory/MQClientInstance.java | 60 +- .../apache/rocketmq/common/BrokerConfig.java | 67 + .../org/apache/rocketmq/common/MixAll.java | 14 +- .../rocketmq/common/constant/LoggerName.java | 1 + .../common/filter/ExpressionType.java | 67 + .../rocketmq/common/filter/FilterAPI.java | 18 + .../rocketmq/common/message/Message.java | 6 + .../common/message/MessageDecoder.java | 39 + .../common/namesrv/TopAddressing.java | 2 +- .../rocketmq/common/protocol/RequestCode.java | 4 + .../common/protocol/ResponseCode.java | 4 + .../protocol/body/CheckClientRequestBody.java | 52 + .../protocol/body/ConsumeQueueData.java | 98 ++ .../body/QueryConsumeQueueResponseBody.java | 72 + .../header/PullMessageRequestHeader.java | 9 + .../QueryConsumeQueueRequestHeader.java | 75 + .../protocol/heartbeat/SubscriptionData.java | 17 +- .../rocketmq/common/filter/FilterAPITest.java | 49 + .../common/message/MessageDecoderTest.java | 80 + distribution/conf/logback_broker.xml | 28 + distribution/release.xml | 1 + .../rocketmq/example/benchmark/Consumer.java | 31 +- .../rocketmq/example/benchmark/Producer.java | 34 +- .../rocketmq/example/filter/SqlConsumer.java | 62 + .../rocketmq/example/filter/SqlProducer.java | 67 + filter/pom.xml | 43 + .../apache/rocketmq/filter/FilterFactory.java | 72 + .../org/apache/rocketmq/filter/FilterSpi.java | 43 + .../org/apache/rocketmq/filter/SqlFilter.java | 43 + .../rocketmq/filter/constant/UnaryType.java | 26 + .../filter/expression/BinaryExpression.java | 91 ++ .../filter/expression/BooleanExpression.java | 39 + .../expression/ComparisonExpression.java | 413 +++++ .../filter/expression/ConstantExpression.java | 156 ++ .../expression/EmptyEvaluationContext.java | 35 + .../filter/expression/EvaluationContext.java | 43 + .../filter/expression/Expression.java | 38 + .../filter/expression/LogicExpression.java | 94 ++ .../filter/expression/MQFilterException.java | 46 + .../filter/expression/NowExpression.java | 36 + .../filter/expression/PropertyExpression.java | 70 + .../filter/expression/UnaryExpression.java | 267 ++++ .../filter/expression/UnaryInExpression.java | 61 + .../filter/parser/ParseException.java | 204 +++ .../filter/parser/SelectorParser.java | 1354 +++++++++++++++++ .../rocketmq/filter/parser/SelectorParser.jj | 524 +++++++ .../parser/SelectorParserConstants.java | 140 ++ .../parser/SelectorParserTokenManager.java | 919 +++++++++++ .../filter/parser/SimpleCharStream.java | 502 ++++++ .../apache/rocketmq/filter/parser/Token.java | 152 ++ .../rocketmq/filter/parser/TokenMgrError.java | 174 +++ .../rocketmq/filter/util/BitsArray.java | 260 ++++ .../rocketmq/filter/util/BloomFilter.java | 338 ++++ .../rocketmq/filter/util/BloomFilterData.java | 83 + .../apache/rocketmq/filter/BitsArrayTest.java | 123 ++ .../rocketmq/filter/BloomFilterTest.java | 172 +++ .../rocketmq/filter/ExpressionTest.java | 594 ++++++++ .../apache/rocketmq/filter/FilterSpiTest.java | 84 + .../apache/rocketmq/filter/ParserTest.java | 129 ++ pom.xml | 11 + srvutil/pom.xml | 4 + .../org/apache/rocketmq/store/CommitLog.java | 8 +- .../rocketmq/store/CommitLogDispatcher.java | 26 + .../apache/rocketmq/store/ConsumeQueue.java | 122 +- .../rocketmq/store/ConsumeQueueExt.java | 638 ++++++++ .../rocketmq/store/DefaultMessageFilter.java | 29 +- .../rocketmq/store/DefaultMessageStore.java | 132 +- .../rocketmq/store/DispatchRequest.java | 21 +- .../org/apache/rocketmq/store/MappedFile.java | 25 + .../rocketmq/store/MappedFileQueue.java | 2 +- .../store/MessageArrivingListener.java | 5 +- .../apache/rocketmq/store/MessageFilter.java | 26 +- .../apache/rocketmq/store/MessageStore.java | 8 +- .../store/config/MessageStoreConfig.java | 31 + .../store/config/StorePathConfigHelper.java | 4 + .../schedule/ScheduleMessageService.java | 14 + .../rocketmq/store/ConsumeQueueExtTest.java | 251 +++ .../rocketmq/store/ConsumeQueueTest.java | 226 +++ .../store/DefaultMessageStoreTest.java | 4 +- .../tools/admin/DefaultMQAdminExt.java | 9 + .../tools/admin/DefaultMQAdminExtImpl.java | 8 + .../rocketmq/tools/admin/MQAdminExt.java | 22 + .../tools/command/MQAdminStartup.java | 3 + .../queue/QueryConsumeQueueCommand.java | 159 ++ 116 files changed, 12552 insertions(+), 128 deletions(-) create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupEvent.java create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/filter/CommitLogDispatcherCalcBitMap.java create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterData.java create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterManager.java create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/filter/ExpressionForRetryMessageFilter.java create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/filter/ExpressionMessageFilter.java create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/filter/MessageEvaluationContext.java create mode 100644 broker/src/test/java/org/apache/rocketmq/broker/filter/CommitLogDispatcherCalcBitMapTest.java create mode 100644 broker/src/test/java/org/apache/rocketmq/broker/filter/ConsumerFilterManagerTest.java create mode 100644 broker/src/test/java/org/apache/rocketmq/broker/filter/MessageStoreWithFilterTest.java create mode 100644 client/src/main/java/org/apache/rocketmq/client/consumer/MessageSelector.java create mode 100644 common/src/main/java/org/apache/rocketmq/common/filter/ExpressionType.java create mode 100644 common/src/main/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBody.java create mode 100644 common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumeQueueData.java create mode 100644 common/src/main/java/org/apache/rocketmq/common/protocol/body/QueryConsumeQueueResponseBody.java create mode 100644 common/src/main/java/org/apache/rocketmq/common/protocol/header/QueryConsumeQueueRequestHeader.java create mode 100644 common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java create mode 100644 example/src/main/java/org/apache/rocketmq/example/filter/SqlConsumer.java create mode 100644 example/src/main/java/org/apache/rocketmq/example/filter/SqlProducer.java create mode 100644 filter/pom.xml create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/FilterFactory.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/FilterSpi.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/SqlFilter.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/constant/UnaryType.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/expression/BinaryExpression.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/expression/BooleanExpression.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/expression/ComparisonExpression.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/expression/ConstantExpression.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/expression/EmptyEvaluationContext.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/expression/EvaluationContext.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/expression/Expression.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/expression/LogicExpression.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/expression/MQFilterException.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/expression/NowExpression.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/expression/PropertyExpression.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/expression/UnaryExpression.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/expression/UnaryInExpression.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/parser/ParseException.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParser.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParser.jj create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParserConstants.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParserTokenManager.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/parser/SimpleCharStream.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/parser/Token.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/parser/TokenMgrError.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/util/BitsArray.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/util/BloomFilter.java create mode 100644 filter/src/main/java/org/apache/rocketmq/filter/util/BloomFilterData.java create mode 100644 filter/src/test/java/org/apache/rocketmq/filter/BitsArrayTest.java create mode 100644 filter/src/test/java/org/apache/rocketmq/filter/BloomFilterTest.java create mode 100644 filter/src/test/java/org/apache/rocketmq/filter/ExpressionTest.java create mode 100644 filter/src/test/java/org/apache/rocketmq/filter/FilterSpiTest.java create mode 100644 filter/src/test/java/org/apache/rocketmq/filter/ParserTest.java create mode 100644 store/src/main/java/org/apache/rocketmq/store/CommitLogDispatcher.java create mode 100644 store/src/main/java/org/apache/rocketmq/store/ConsumeQueueExt.java create mode 100644 store/src/test/java/org/apache/rocketmq/store/ConsumeQueueExtTest.java create mode 100644 store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java create mode 100644 tools/src/main/java/org/apache/rocketmq/tools/command/queue/QueryConsumeQueueCommand.java diff --git a/broker/pom.xml b/broker/pom.xml index 8cdafea3..0f8ad0a1 100644 --- a/broker/pom.xml +++ b/broker/pom.xml @@ -48,6 +48,10 @@ ${project.groupId} rocketmq-srvutil + + ${project.groupId} + rocketmq-filter + ch.qos.logback logback-classic diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java index 6acd40ca..bacd25cc 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java @@ -37,6 +37,8 @@ import org.apache.rocketmq.broker.client.DefaultConsumerIdsChangeListener; import org.apache.rocketmq.broker.client.ProducerManager; import org.apache.rocketmq.broker.client.net.Broker2Client; import org.apache.rocketmq.broker.client.rebalance.RebalanceLockManager; +import org.apache.rocketmq.broker.filter.CommitLogDispatcherCalcBitMap; +import org.apache.rocketmq.broker.filter.ConsumerFilterManager; import org.apache.rocketmq.broker.filtersrv.FilterServerManager; import org.apache.rocketmq.broker.latency.BrokerFastFailure; import org.apache.rocketmq.broker.latency.BrokerFixedThreadPoolExecutor; @@ -96,6 +98,7 @@ public class BrokerController { private final MessageStoreConfig messageStoreConfig; private final ConsumerOffsetManager consumerOffsetManager; private final ConsumerManager consumerManager; + private final ConsumerFilterManager consumerFilterManager; private final ProducerManager producerManager; private final ClientHousekeepingService clientHousekeepingService; private final PullMessageProcessor pullMessageProcessor; @@ -149,6 +152,7 @@ public class BrokerController { this.messageArrivingListener = new NotifyMessageArrivingListener(this.pullRequestHoldService); this.consumerIdsChangeListener = new DefaultConsumerIdsChangeListener(this); this.consumerManager = new ConsumerManager(this.consumerIdsChangeListener); + this.consumerFilterManager = new ConsumerFilterManager(this); this.producerManager = new ProducerManager(); this.clientHousekeepingService = new ClientHousekeepingService(this); this.broker2Client = new Broker2Client(this); @@ -192,6 +196,7 @@ public class BrokerController { result = result && this.consumerOffsetManager.load(); result = result && this.subscriptionGroupManager.load(); + result = result && this.consumerFilterManager.load(); if (result) { try { @@ -202,6 +207,7 @@ public class BrokerController { //load plugin MessageStorePluginContext context = new MessageStorePluginContext(messageStoreConfig, brokerStatsManager, messageArrivingListener, brokerConfig); this.messageStore = MessageStoreFactory.build(context, this.messageStore); + this.messageStore.getDispatcherList().addFirst(new CommitLogDispatcherCalcBitMap(this.brokerConfig, this.consumerFilterManager)); } catch (IOException e) { result = false; e.printStackTrace(); @@ -274,6 +280,17 @@ public class BrokerController { } }, 1000 * 10, this.brokerConfig.getFlushConsumerOffsetInterval(), TimeUnit.MILLISECONDS); + this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { + @Override + public void run() { + try { + BrokerController.this.consumerFilterManager.persist(); + } catch (Throwable e) { + log.error("schedule persist consumer filter error.", e); + } + } + }, 1000 * 10, 1000 * 10, TimeUnit.MILLISECONDS); + this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override public void run() { @@ -400,9 +417,11 @@ public class BrokerController { ClientManageProcessor clientProcessor = new ClientManageProcessor(this); this.remotingServer.registerProcessor(RequestCode.HEART_BEAT, clientProcessor, this.clientManageExecutor); this.remotingServer.registerProcessor(RequestCode.UNREGISTER_CLIENT, clientProcessor, this.clientManageExecutor); + this.remotingServer.registerProcessor(RequestCode.CHECK_CLIENT_CONFIG, clientProcessor, this.clientManageExecutor); this.fastRemotingServer.registerProcessor(RequestCode.HEART_BEAT, clientProcessor, this.clientManageExecutor); this.fastRemotingServer.registerProcessor(RequestCode.UNREGISTER_CLIENT, clientProcessor, this.clientManageExecutor); + this.remotingServer.registerProcessor(RequestCode.CHECK_CLIENT_CONFIG, clientProcessor, this.clientManageExecutor); /** * ConsumerManageProcessor @@ -504,6 +523,10 @@ public class BrokerController { return consumerManager; } + public ConsumerFilterManager getConsumerFilterManager() { + return consumerFilterManager; + } + public ConsumerOffsetManager getConsumerOffsetManager() { return consumerOffsetManager; } @@ -590,6 +613,10 @@ public class BrokerController { if (this.brokerFastFailure != null) { this.brokerFastFailure.shutdown(); } + + if (this.consumerFilterManager != null) { + this.consumerFilterManager.persist(); + } } private void unregisterBrokerAll() { diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerPathConfigHelper.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerPathConfigHelper.java index 24876df4..0a323ee8 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerPathConfigHelper.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerPathConfigHelper.java @@ -44,4 +44,7 @@ public class BrokerPathConfigHelper { return rootDir + File.separator + "config" + File.separator + "subscriptionGroup.json"; } + public static String getConsumerFilterPath(final String rootDir) { + return rootDir + File.separator + "config" + File.separator + "consumerFilter.json"; + } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupEvent.java b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupEvent.java new file mode 100644 index 00000000..717fb708 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupEvent.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.broker.client; + +public enum ConsumerGroupEvent { + + /** + * Some consumers in the group are changed. + */ + CHANGE, + /** + * The group of consumer is unregistered. + */ + UNREGISTER, + /** + * The group of consumer is registered. + */ + REGISTER +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java index 07d28dca..831e2932 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java @@ -16,9 +16,7 @@ */ package org.apache.rocketmq.broker.client; -import io.netty.channel.Channel; -import java.util.List; - public interface ConsumerIdsChangeListener { - void consumerIdsChanged(final String group, final List channels); + + void handle(ConsumerGroupEvent event, String group, Object... args); } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerManager.java b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerManager.java index a2d88d5d..a5ddec8c 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerManager.java @@ -85,10 +85,11 @@ public class ConsumerManager { if (remove != null) { log.info("unregister consumer ok, no any connection, and remove consumer group, {}", next.getKey()); + this.consumerIdsChangeListener.handle(ConsumerGroupEvent.UNREGISTER, next.getKey()); } } - this.consumerIdsChangeListener.consumerIdsChanged(next.getKey(), info.getAllChannel()); + this.consumerIdsChangeListener.handle(ConsumerGroupEvent.CHANGE, next.getKey(), info.getAllChannel()); } } } @@ -111,10 +112,12 @@ public class ConsumerManager { if (r1 || r2) { if (isNotifyConsumerIdsChangedEnable) { - this.consumerIdsChangeListener.consumerIdsChanged(group, consumerGroupInfo.getAllChannel()); + this.consumerIdsChangeListener.handle(ConsumerGroupEvent.CHANGE, group, consumerGroupInfo.getAllChannel()); } } + this.consumerIdsChangeListener.handle(ConsumerGroupEvent.REGISTER, group, subList); + return r1 || r2; } @@ -126,10 +129,12 @@ public class ConsumerManager { ConsumerGroupInfo remove = this.consumerTable.remove(group); if (remove != null) { log.info("unregister consumer ok, no any connection, and remove consumer group, {}", group); + + this.consumerIdsChangeListener.handle(ConsumerGroupEvent.UNREGISTER, group); } } if (isNotifyConsumerIdsChangedEnable) { - this.consumerIdsChangeListener.consumerIdsChanged(group, consumerGroupInfo.getAllChannel()); + this.consumerIdsChangeListener.handle(ConsumerGroupEvent.CHANGE, group, consumerGroupInfo.getAllChannel()); } } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java b/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java index a1b2d8ac..d716a339 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java @@ -17,8 +17,12 @@ package org.apache.rocketmq.broker.client; import io.netty.channel.Channel; + +import java.util.Collection; import java.util.List; + import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; public class DefaultConsumerIdsChangeListener implements ConsumerIdsChangeListener { private final BrokerController brokerController; @@ -28,11 +32,34 @@ public class DefaultConsumerIdsChangeListener implements ConsumerIdsChangeListen } @Override - public void consumerIdsChanged(String group, List channels) { - if (channels != null && brokerController.getBrokerConfig().isNotifyConsumerIdsChangedEnable()) { - for (Channel chl : channels) { - this.brokerController.getBroker2Client().notifyConsumerIdsChanged(chl, group); - } + public void handle(ConsumerGroupEvent event, String group, Object... args) { + if (event == null) { + return; + } + switch (event) { + case CHANGE: + if (args == null || args.length < 1) { + return; + } + List channels = (List) args[0]; + if (channels != null && brokerController.getBrokerConfig().isNotifyConsumerIdsChangedEnable()) { + for (Channel chl : channels) { + this.brokerController.getBroker2Client().notifyConsumerIdsChanged(chl, group); + } + } + break; + case UNREGISTER: + this.brokerController.getConsumerFilterManager().unRegister(group); + break; + case REGISTER: + if (args == null || args.length < 1) { + return; + } + Collection subscriptionDataList = (Collection) args[0]; + this.brokerController.getConsumerFilterManager().register(group, subscriptionDataList); + break; + default: + throw new RuntimeException("Unknown event " + event); } } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filter/CommitLogDispatcherCalcBitMap.java b/broker/src/main/java/org/apache/rocketmq/broker/filter/CommitLogDispatcherCalcBitMap.java new file mode 100644 index 00000000..85415d62 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/filter/CommitLogDispatcherCalcBitMap.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.broker.filter; + +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.filter.util.BitsArray; +import org.apache.rocketmq.store.CommitLogDispatcher; +import org.apache.rocketmq.store.DispatchRequest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Iterator; + +/** + * Calculate bit map of filter. + */ +public class CommitLogDispatcherCalcBitMap implements CommitLogDispatcher { + + private static final Logger log = LoggerFactory.getLogger(LoggerName.FILTER_LOGGER_NAME); + + protected final BrokerConfig brokerConfig; + protected final ConsumerFilterManager consumerFilterManager; + + public CommitLogDispatcherCalcBitMap(BrokerConfig brokerConfig, ConsumerFilterManager consumerFilterManager) { + this.brokerConfig = brokerConfig; + this.consumerFilterManager = consumerFilterManager; + } + + @Override + public void dispatch(DispatchRequest request) { + if (!this.brokerConfig.isEnableCalcFilterBitMap()) { + return; + } + + try { + + Collection filterDatas = consumerFilterManager.get(request.getTopic()); + + if (filterDatas == null || filterDatas.isEmpty()) { + return; + } + + Iterator iterator = filterDatas.iterator(); + BitsArray filterBitMap = BitsArray.create( + this.consumerFilterManager.getBloomFilter().getM() + ); + + long startTime = System.currentTimeMillis(); + while (iterator.hasNext()) { + ConsumerFilterData filterData = iterator.next(); + + if (filterData.getCompiledExpression() == null) { + log.error("[BUG] Consumer in filter manager has no compiled expression! {}", filterData); + continue; + } + + if (filterData.getBloomFilterData() == null) { + log.error("[BUG] Consumer in filter manager has no bloom data! {}", filterData); + continue; + } + + Object ret = null; + try { + MessageEvaluationContext context = new MessageEvaluationContext(request.getPropertiesMap()); + + ret = filterData.getCompiledExpression().evaluate(context); + } catch (Throwable e) { + log.error("Calc filter bit map error!commitLogOffset={}, consumer={}, {}", request.getCommitLogOffset(), filterData, e); + } + + log.debug("Result of Calc bit map:ret={}, data={}, props={}, offset={}", ret, filterData, request.getPropertiesMap(), request.getCommitLogOffset()); + + // eval true + if (ret != null && ret instanceof Boolean && (Boolean) ret) { + consumerFilterManager.getBloomFilter().hashTo( + filterData.getBloomFilterData(), + filterBitMap + ); + } + } + + request.setBitMap(filterBitMap.bytes()); + + long eclipseTime = System.currentTimeMillis() - startTime; + // 1ms + if (eclipseTime >= 1) { + log.warn("Spend {} ms to calc bit map, consumerNum={}, topic={}", eclipseTime, filterDatas.size(), request.getTopic()); + } + } catch (Throwable e) { + log.error("Calc bit map error! topic={}, offset={}, queueId={}, {}", request.getTopic(), request.getCommitLogOffset(), request.getQueueId(), e); + } + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterData.java b/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterData.java new file mode 100644 index 00000000..4db02e26 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterData.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.broker.filter; + +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.commons.lang3.builder.ReflectionToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; +import org.apache.rocketmq.filter.expression.Expression; +import org.apache.rocketmq.filter.util.BloomFilterData; + +import java.util.Collections; + +/** + * Filter data of consumer. + */ +public class ConsumerFilterData { + + private String consumerGroup; + private String topic; + private String expression; + private String expressionType; + private transient Expression compiledExpression; + private long bornTime; + private long deadTime = 0; + private BloomFilterData bloomFilterData; + private long clientVersion; + + public boolean isDead() { + return this.deadTime >= this.bornTime; + } + + public long howLongAfterDeath() { + if (isDead()) { + return System.currentTimeMillis() - getDeadTime(); + } + return -1; + } + + /** + * Check this filter data has been used to calculate bit map when msg was stored in server. + * + * @param msgStoreTime + * @return + */ + public boolean isMsgInLive(long msgStoreTime) { + return msgStoreTime > getBornTime(); + } + + public String getConsumerGroup() { + return consumerGroup; + } + + public void setConsumerGroup(final String consumerGroup) { + this.consumerGroup = consumerGroup; + } + + public String getTopic() { + return topic; + } + + public void setTopic(final String topic) { + this.topic = topic; + } + + public String getExpression() { + return expression; + } + + public void setExpression(final String expression) { + this.expression = expression; + } + + public String getExpressionType() { + return expressionType; + } + + public void setExpressionType(final String expressionType) { + this.expressionType = expressionType; + } + + public Expression getCompiledExpression() { + return compiledExpression; + } + + public void setCompiledExpression(final Expression compiledExpression) { + this.compiledExpression = compiledExpression; + } + + public long getBornTime() { + return bornTime; + } + + public void setBornTime(final long bornTime) { + this.bornTime = bornTime; + } + + public long getDeadTime() { + return deadTime; + } + + public void setDeadTime(final long deadTime) { + this.deadTime = deadTime; + } + + public BloomFilterData getBloomFilterData() { + return bloomFilterData; + } + + public void setBloomFilterData(final BloomFilterData bloomFilterData) { + this.bloomFilterData = bloomFilterData; + } + + public long getClientVersion() { + return clientVersion; + } + + public void setClientVersion(long clientVersion) { + this.clientVersion = clientVersion; + } + + @Override + public boolean equals(Object o) { + return EqualsBuilder.reflectionEquals(this, o, Collections.emptyList()); + } + + @Override + public int hashCode() { + return HashCodeBuilder.reflectionHashCode(this, Collections.emptyList()); + } + + @Override + public String toString() { + return ReflectionToStringBuilder.toString(this, ToStringStyle.SHORT_PREFIX_STYLE); + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterManager.java b/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterManager.java new file mode 100644 index 00000000..7f790af6 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterManager.java @@ -0,0 +1,471 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.broker.filter; + +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.BrokerPathConfigHelper; +import org.apache.rocketmq.common.ConfigManager; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.filter.FilterFactory; +import org.apache.rocketmq.common.filter.ExpressionType; +import org.apache.rocketmq.filter.util.BloomFilter; +import org.apache.rocketmq.filter.util.BloomFilterData; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Consumer filter data manager.Just manage the consumers use expression filter. + */ +public class ConsumerFilterManager extends ConfigManager { + + private static final Logger log = LoggerFactory.getLogger(LoggerName.FILTER_LOGGER_NAME); + + private static final long MS_24_HOUR = 24 * 3600 * 1000; + + private ConcurrentHashMap + filterDataByTopic = new ConcurrentHashMap(256); + + private transient BrokerController brokerController; + private transient BloomFilter bloomFilter; + + public ConsumerFilterManager() { + // just for test + this.bloomFilter = BloomFilter.createByFn(20, 64); + } + + public ConsumerFilterManager(BrokerController brokerController) { + this.brokerController = brokerController; + this.bloomFilter = BloomFilter.createByFn( + brokerController.getBrokerConfig().getMaxErrorRateOfBloomFilter(), + brokerController.getBrokerConfig().getExpectConsumerNumUseFilter() + ); + // then set bit map length of store config. + brokerController.getMessageStoreConfig().setBitMapLengthConsumeQueueExt( + this.bloomFilter.getM() + ); + } + + /** + * Build consumer filter data.Be care, bloom filter data is not included. + * + * @param topic + * @param consumerGroup + * @param expression + * @param type + * @param clientVersion + * @return maybe null + */ + public static ConsumerFilterData build(final String topic, final String consumerGroup, + final String expression, final String type, + final long clientVersion) { + if (ExpressionType.isTagType(type)) { + return null; + } + + ConsumerFilterData consumerFilterData = new ConsumerFilterData(); + consumerFilterData.setTopic(topic); + consumerFilterData.setConsumerGroup(consumerGroup); + consumerFilterData.setBornTime(System.currentTimeMillis()); + consumerFilterData.setDeadTime(0); + consumerFilterData.setExpression(expression); + consumerFilterData.setExpressionType(type); + consumerFilterData.setClientVersion(clientVersion); + try { + consumerFilterData.setCompiledExpression( + FilterFactory.INSTANCE.get(type).compile(expression) + ); + } catch (Throwable e) { + log.error("parse error: expr={}, topic={}, group={}, error={}", expression, topic, consumerGroup, e.getMessage()); + return null; + } + + return consumerFilterData; + } + + public void register(final String consumerGroup, final Collection subList) { + for (SubscriptionData subscriptionData : subList) { + register( + subscriptionData.getTopic(), + consumerGroup, + subscriptionData.getSubString(), + subscriptionData.getExpressionType(), + subscriptionData.getSubVersion() + ); + } + + // make illegal topic dead. + Collection groupFilterData = getByGroup(consumerGroup); + + Iterator iterator = groupFilterData.iterator(); + while (iterator.hasNext()) { + ConsumerFilterData filterData = iterator.next(); + + boolean exist = false; + for (SubscriptionData subscriptionData : subList) { + if (subscriptionData.getTopic().equals(filterData.getTopic())) { + exist = true; + break; + } + } + + if (!exist && !filterData.isDead()) { + filterData.setDeadTime(System.currentTimeMillis()); + log.info("Consumer filter changed: {}, make illegal topic dead:{}", consumerGroup, filterData); + } + } + } + + public boolean register(final String topic, final String consumerGroup, final String expression, + final String type, final long clientVersion) { + if (ExpressionType.isTagType(type)) { + return false; + } + + if (expression == null || expression.length() == 0) { + return false; + } + + FilterDataMapByTopic filterDataMapByTopic = this.filterDataByTopic.get(topic); + + if (filterDataMapByTopic == null) { + FilterDataMapByTopic temp = new FilterDataMapByTopic(topic); + FilterDataMapByTopic prev = this.filterDataByTopic.putIfAbsent(topic, temp); + filterDataMapByTopic = prev != null ? prev : temp; + } + + BloomFilterData bloomFilterData = bloomFilter.generate(consumerGroup + "#" + topic); + + return filterDataMapByTopic.register(consumerGroup, expression, type, bloomFilterData, clientVersion); + } + + public void unRegister(final String consumerGroup) { + for (String topic : filterDataByTopic.keySet()) { + this.filterDataByTopic.get(topic).unRegister(consumerGroup); + } + } + + public ConsumerFilterData get(final String topic, final String consumerGroup) { + if (!this.filterDataByTopic.containsKey(topic)) { + return null; + } + if (this.filterDataByTopic.get(topic).getGroupFilterData().isEmpty()) { + return null; + } + + return this.filterDataByTopic.get(topic).getGroupFilterData().get(consumerGroup); + } + + public Collection getByGroup(final String consumerGroup) { + Collection ret = new HashSet(); + + Iterator topicIterator = this.filterDataByTopic.values().iterator(); + while (topicIterator.hasNext()) { + FilterDataMapByTopic filterDataMapByTopic = topicIterator.next(); + + Iterator filterDataIterator = filterDataMapByTopic.getGroupFilterData().values().iterator(); + + while (filterDataIterator.hasNext()) { + ConsumerFilterData filterData = filterDataIterator.next(); + + if (filterData.getConsumerGroup().equals(consumerGroup)) { + ret.add(filterData); + } + } + } + + return ret; + } + + public final Collection get(final String topic) { + if (!this.filterDataByTopic.containsKey(topic)) { + return null; + } + if (this.filterDataByTopic.get(topic).getGroupFilterData().isEmpty()) { + return null; + } + + return this.filterDataByTopic.get(topic).getGroupFilterData().values(); + } + + public BloomFilter getBloomFilter() { + return bloomFilter; + } + + @Override + public String encode() { + return encode(false); + } + + @Override + public String configFilePath() { + if (this.brokerController != null) { + return BrokerPathConfigHelper.getConsumerFilterPath( + this.brokerController.getMessageStoreConfig().getStorePathRootDir() + ); + } + return BrokerPathConfigHelper.getConsumerFilterPath("./unit_test"); + } + + @Override + public void decode(final String jsonString) { + ConsumerFilterManager load = RemotingSerializable.fromJson(jsonString, ConsumerFilterManager.class); + if (load != null && load.filterDataByTopic != null) { + boolean bloomChanged = false; + for (String topic : load.filterDataByTopic.keySet()) { + FilterDataMapByTopic dataMapByTopic = load.filterDataByTopic.get(topic); + if (dataMapByTopic == null) { + continue; + } + + for (String group : dataMapByTopic.getGroupFilterData().keySet()) { + + ConsumerFilterData filterData = dataMapByTopic.getGroupFilterData().get(group); + + if (filterData == null) { + continue; + } + + try { + filterData.setCompiledExpression( + FilterFactory.INSTANCE.get(filterData.getExpressionType()).compile(filterData.getExpression()) + ); + } catch (Exception e) { + log.error("load filter data error, " + filterData, e); + } + + // check whether bloom filter is changed + // if changed, ignore the bit map calculated before. + if (!this.bloomFilter.isValid(filterData.getBloomFilterData())) { + bloomChanged = true; + log.info("Bloom filter is changed!So ignore all filter data persisted! {}, {}", this.bloomFilter, filterData.getBloomFilterData()); + break; + } + + log.info("load exist consumer filter data: {}", filterData); + + if (filterData.getDeadTime() == 0) { + // we think all consumers are dead when load + long deadTime = System.currentTimeMillis() - 30 * 1000; + filterData.setDeadTime( + deadTime <= filterData.getBornTime() ? filterData.getBornTime() : deadTime + ); + } + } + } + + if (!bloomChanged) { + this.filterDataByTopic = load.filterDataByTopic; + } + } + } + + @Override + public String encode(final boolean prettyFormat) { + // clean + { + clean(); + } + return RemotingSerializable.toJson(this, prettyFormat); + } + + public void clean() { + Iterator> topicIterator = this.filterDataByTopic.entrySet().iterator(); + while (topicIterator.hasNext()) { + Map.Entry filterDataMapByTopic = topicIterator.next(); + + Iterator> filterDataIterator + = filterDataMapByTopic.getValue().getGroupFilterData().entrySet().iterator(); + + while (filterDataIterator.hasNext()) { + Map.Entry filterDataByGroup = filterDataIterator.next(); + + ConsumerFilterData filterData = filterDataByGroup.getValue(); + if (filterData.howLongAfterDeath() >= (this.brokerController == null ? MS_24_HOUR : this.brokerController.getBrokerConfig().getFilterDataCleanTimeSpan())) { + log.info("Remove filter consumer {}, died too long!", filterDataByGroup.getValue()); + filterDataIterator.remove(); + } + } + + if (filterDataMapByTopic.getValue().getGroupFilterData().isEmpty()) { + log.info("Topic has no consumer, remove it! {}", filterDataMapByTopic.getKey()); + topicIterator.remove(); + } + } + } + + public ConcurrentHashMap getFilterDataByTopic() { + return filterDataByTopic; + } + + public void setFilterDataByTopic(final ConcurrentHashMap filterDataByTopic) { + this.filterDataByTopic = filterDataByTopic; + } + + public static class FilterDataMapByTopic { + + private ConcurrentHashMap + groupFilterData = new ConcurrentHashMap(); + + private String topic; + + public FilterDataMapByTopic() { + } + + public FilterDataMapByTopic(String topic) { + this.topic = topic; + } + + public void unRegister(String consumerGroup) { + if (!this.groupFilterData.containsKey(consumerGroup)) { + return; + } + + ConsumerFilterData data = this.groupFilterData.get(consumerGroup); + + if (data == null || data.isDead()) { + return; + } + + long now = System.currentTimeMillis(); + + log.info("Unregister consumer filter: {}, deadTime: {}", data, now); + + data.setDeadTime(now); + } + + public boolean register(String consumerGroup, String expression, String type, BloomFilterData bloomFilterData, long clientVersion) { + ConsumerFilterData old = this.groupFilterData.get(consumerGroup); + + if (old == null) { + ConsumerFilterData consumerFilterData = build(topic, consumerGroup, expression, type, clientVersion); + if (consumerFilterData == null) { + return false; + } + consumerFilterData.setBloomFilterData(bloomFilterData); + + old = this.groupFilterData.putIfAbsent(consumerGroup, consumerFilterData); + if (old == null) { + log.info("New consumer filter registered: {}", consumerFilterData); + return true; + } else { + if (clientVersion <= old.getClientVersion()) { + if (!type.equals(old.getExpressionType()) || !expression.equals(old.getExpression())) { + log.warn("Ignore consumer({} : {}) filter(concurrent), because of version {} <= {}, but maybe info changed!old={}:{}, ignored={}:{}", + consumerGroup, topic, + clientVersion, old.getClientVersion(), + old.getExpressionType(), old.getExpression(), + type, expression); + } + if (clientVersion == old.getClientVersion() && old.isDead()) { + reAlive(old); + return true; + } + + return false; + } else { + this.groupFilterData.put(consumerGroup, consumerFilterData); + log.info("New consumer filter registered(concurrent): {}, old: {}", consumerFilterData, old); + return true; + } + } + } else { + if (clientVersion <= old.getClientVersion()) { + if (!type.equals(old.getExpressionType()) || !expression.equals(old.getExpression())) { + log.info("Ignore consumer({}:{}) filter, because of version {} <= {}, but maybe info changed!old={}:{}, ignored={}:{}", + consumerGroup, topic, + clientVersion, old.getClientVersion(), + old.getExpressionType(), old.getExpression(), + type, expression); + } + if (clientVersion == old.getClientVersion() && old.isDead()) { + reAlive(old); + return true; + } + + return false; + } + + boolean change = !old.getExpression().equals(expression) || !old.getExpressionType().equals(type); + if (old.getBloomFilterData() == null && bloomFilterData != null) { + change = true; + } + if (old.getBloomFilterData() != null && !old.getBloomFilterData().equals(bloomFilterData)) { + change = true; + } + + // if subscribe data is changed, or consumer is died too long. + if (change) { + ConsumerFilterData consumerFilterData = build(topic, consumerGroup, expression, type, clientVersion); + if (consumerFilterData == null) { + // new expression compile error, remove old, let client report error. + this.groupFilterData.remove(consumerGroup); + return false; + } + consumerFilterData.setBloomFilterData(bloomFilterData); + + this.groupFilterData.put(consumerGroup, consumerFilterData); + + log.info("Consumer filter info change, old: {}, new: {}, change: {}", + old, consumerFilterData, change); + + return true; + } else { + old.setClientVersion(clientVersion); + if (old.isDead()) { + reAlive(old); + } + return true; + } + } + } + + protected void reAlive(ConsumerFilterData filterData) { + long oldDeadTime = filterData.getDeadTime(); + filterData.setDeadTime(0); + log.info("Re alive consumer filter: {}, oldDeadTime: {}", filterData, oldDeadTime); + } + + public final ConsumerFilterData get(String consumerGroup) { + return this.groupFilterData.get(consumerGroup); + } + + public final ConcurrentHashMap getGroupFilterData() { + return this.groupFilterData; + } + + public void setGroupFilterData(final ConcurrentHashMap groupFilterData) { + this.groupFilterData = groupFilterData; + } + + public String getTopic() { + return topic; + } + + public void setTopic(final String topic) { + this.topic = topic; + } + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filter/ExpressionForRetryMessageFilter.java b/broker/src/main/java/org/apache/rocketmq/broker/filter/ExpressionForRetryMessageFilter.java new file mode 100644 index 00000000..95181789 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/filter/ExpressionForRetryMessageFilter.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.broker.filter; + + +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.filter.ExpressionType; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; + +import java.nio.ByteBuffer; +import java.util.Map; + +/** + * Support filter to retry topic. + *
It will decode properties first in order to get real topic. + */ +public class ExpressionForRetryMessageFilter extends ExpressionMessageFilter { + public ExpressionForRetryMessageFilter(SubscriptionData subscriptionData, ConsumerFilterData consumerFilterData, ConsumerFilterManager consumerFilterManager) { + super(subscriptionData, consumerFilterData, consumerFilterManager); + } + + @Override + public boolean isMatchedByCommitLog(ByteBuffer msgBuffer, Map properties) { + if (subscriptionData == null) { + return true; + } + + if (subscriptionData.isClassFilterMode()) { + return true; + } + + boolean isRetryTopic = subscriptionData.getTopic().startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX); + + if (!isRetryTopic && ExpressionType.isTagType(subscriptionData.getExpressionType())) { + return true; + } + + ConsumerFilterData realFilterData = this.consumerFilterData; + Map tempProperties = properties; + boolean decoded = false; + if (isRetryTopic) { + // retry topic, use original filter data. + // poor performance to support retry filter. + if (tempProperties == null && msgBuffer != null) { + decoded = true; + tempProperties = MessageDecoder.decodeProperties(msgBuffer); + } + String realTopic = tempProperties.get(MessageConst.PROPERTY_RETRY_TOPIC); + String group = subscriptionData.getTopic().substring(MixAll.RETRY_GROUP_TOPIC_PREFIX.length()); + realFilterData = this.consumerFilterManager.get(realTopic, group); + } + + // no expression + if (realFilterData == null || realFilterData.getExpression() == null + || realFilterData.getCompiledExpression() == null) { + return true; + } + + if (!decoded && tempProperties == null && msgBuffer != null) { + tempProperties = MessageDecoder.decodeProperties(msgBuffer); + } + + Object ret = null; + try { + MessageEvaluationContext context = new MessageEvaluationContext(tempProperties); + + ret = realFilterData.getCompiledExpression().evaluate(context); + } catch (Throwable e) { + log.error("Message Filter error, " + realFilterData + ", " + tempProperties, e); + } + + log.debug("Pull eval result: {}, {}, {}", ret, realFilterData, tempProperties); + + if (ret == null || !(ret instanceof Boolean)) { + return false; + } + + return (Boolean) ret; + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filter/ExpressionMessageFilter.java b/broker/src/main/java/org/apache/rocketmq/broker/filter/ExpressionMessageFilter.java new file mode 100644 index 00000000..893df0d7 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/filter/ExpressionMessageFilter.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.broker.filter; + +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.filter.ExpressionType; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.filter.util.BitsArray; +import org.apache.rocketmq.filter.util.BloomFilter; +import org.apache.rocketmq.store.ConsumeQueueExt; +import org.apache.rocketmq.store.MessageFilter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.Map; + +public class ExpressionMessageFilter implements MessageFilter { + + protected static final Logger log = LoggerFactory.getLogger(LoggerName.FILTER_LOGGER_NAME); + + protected final SubscriptionData subscriptionData; + protected final ConsumerFilterData consumerFilterData; + protected final ConsumerFilterManager consumerFilterManager; + protected final boolean bloomDataValid; + + public ExpressionMessageFilter(SubscriptionData subscriptionData, ConsumerFilterData consumerFilterData, + ConsumerFilterManager consumerFilterManager) { + this.subscriptionData = subscriptionData; + this.consumerFilterData = consumerFilterData; + this.consumerFilterManager = consumerFilterManager; + if (consumerFilterData == null) { + bloomDataValid = false; + return; + } + BloomFilter bloomFilter = this.consumerFilterManager.getBloomFilter(); + if (bloomFilter != null && bloomFilter.isValid(consumerFilterData.getBloomFilterData())) { + bloomDataValid = true; + } else { + bloomDataValid = false; + } + } + + @Override + public boolean isMatchedByConsumeQueue(Long tagsCode, ConsumeQueueExt.CqExtUnit cqExtUnit) { + if (null == subscriptionData) { + return true; + } + + if (subscriptionData.isClassFilterMode()) { + return true; + } + + // by tags code. + if (ExpressionType.isTagType(subscriptionData.getExpressionType())) { + + if (tagsCode == null || tagsCode < 0L) { + return true; + } + + if (subscriptionData.getSubString().equals(SubscriptionData.SUB_ALL)) { + return true; + } + + return subscriptionData.getCodeSet().contains(tagsCode.intValue()); + } else { + // no expression or no bloom + if (consumerFilterData == null || consumerFilterData.getExpression() == null + || consumerFilterData.getCompiledExpression() == null || consumerFilterData.getBloomFilterData() == null) { + return true; + } + + // message is before consumer + if (cqExtUnit == null || !consumerFilterData.isMsgInLive(cqExtUnit.getMsgStoreTime())) { + log.debug("Pull matched because not in live: {}, {}", consumerFilterData, cqExtUnit); + return true; + } + + byte[] filterBitMap = cqExtUnit.getFilterBitMap(); + BloomFilter bloomFilter = this.consumerFilterManager.getBloomFilter(); + if (filterBitMap == null || !this.bloomDataValid + || filterBitMap.length * Byte.SIZE != consumerFilterData.getBloomFilterData().getBitNum()) { + return true; + } + + BitsArray bitsArray = null; + try { + bitsArray = BitsArray.create(filterBitMap); + boolean ret = bloomFilter.isHit(consumerFilterData.getBloomFilterData(), bitsArray); + log.debug("Pull {} by bit map:{}, {}, {}", ret, consumerFilterData, bitsArray, cqExtUnit); + return ret; + } catch (Throwable e) { + log.error("bloom filter error, sub=" + subscriptionData + + ", filter=" + consumerFilterData + ", bitMap=" + bitsArray, e); + } + } + + return true; + } + + @Override + public boolean isMatchedByCommitLog(ByteBuffer msgBuffer, Map properties) { + if (subscriptionData == null) { + return true; + } + + if (subscriptionData.isClassFilterMode()) { + return true; + } + + if (ExpressionType.isTagType(subscriptionData.getExpressionType())) { + return true; + } + + ConsumerFilterData realFilterData = this.consumerFilterData; + Map tempProperties = properties; + + // no expression + if (realFilterData == null || realFilterData.getExpression() == null + || realFilterData.getCompiledExpression() == null) { + return true; + } + + if (tempProperties == null && msgBuffer != null) { + tempProperties = MessageDecoder.decodeProperties(msgBuffer); + } + + Object ret = null; + try { + MessageEvaluationContext context = new MessageEvaluationContext(tempProperties); + + ret = realFilterData.getCompiledExpression().evaluate(context); + } catch (Throwable e) { + log.error("Message Filter error, " + realFilterData + ", " + tempProperties, e); + } + + log.debug("Pull eval result: {}, {}, {}", ret, realFilterData, tempProperties); + + if (ret == null || !(ret instanceof Boolean)) { + return false; + } + + return (Boolean) ret; + } + +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filter/MessageEvaluationContext.java b/broker/src/main/java/org/apache/rocketmq/broker/filter/MessageEvaluationContext.java new file mode 100644 index 00000000..879d1790 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/filter/MessageEvaluationContext.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.broker.filter; + +import org.apache.rocketmq.filter.expression.EvaluationContext; + +import java.util.HashMap; +import java.util.Map; + +/** + * Evaluation context from message. + */ +public class MessageEvaluationContext implements EvaluationContext { + + private Map properties; + + public MessageEvaluationContext(Map properties) { + this.properties = properties; + } + + @Override + public Object get(final String name) { + if (this.properties == null) { + return null; + } + return this.properties.get(name); + } + + @Override + public Map keyValues() { + if (properties == null) { + return null; + } + + Map copy = new HashMap(properties.size(), 1); + + for (String key : properties.keySet()) { + copy.put(key, properties.get(key)); + } + + return copy; + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java index 2dec9f74..fd38c4fb 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java @@ -19,6 +19,8 @@ package org.apache.rocketmq.broker.longpolling; import org.apache.rocketmq.store.MessageArrivingListener; +import java.util.Map; + public class NotifyMessageArrivingListener implements MessageArrivingListener { private final PullRequestHoldService pullRequestHoldService; @@ -27,7 +29,9 @@ public class NotifyMessageArrivingListener implements MessageArrivingListener { } @Override - public void arriving(String topic, int queueId, long logicOffset, long tagsCode) { - this.pullRequestHoldService.notifyMessageArriving(topic, queueId, logicOffset, tagsCode); + public void arriving(String topic, int queueId, long logicOffset, long tagsCode, + long msgStoreTime, byte[] filterBitMap, Map properties) { + this.pullRequestHoldService.notifyMessageArriving(topic, queueId, logicOffset, tagsCode, + msgStoreTime, filterBitMap, properties); } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequest.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequest.java index b66344f3..045ab9b1 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequest.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequest.java @@ -19,6 +19,7 @@ package org.apache.rocketmq.broker.longpolling; import io.netty.channel.Channel; import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.store.MessageFilter; public class PullRequest { private final RemotingCommand requestCommand; @@ -27,15 +28,18 @@ public class PullRequest { private final long suspendTimestamp; private final long pullFromThisOffset; private final SubscriptionData subscriptionData; + private final MessageFilter messageFilter; public PullRequest(RemotingCommand requestCommand, Channel clientChannel, long timeoutMillis, long suspendTimestamp, - long pullFromThisOffset, SubscriptionData subscriptionData) { + long pullFromThisOffset, SubscriptionData subscriptionData, + MessageFilter messageFilter) { this.requestCommand = requestCommand; this.clientChannel = clientChannel; this.timeoutMillis = timeoutMillis; this.suspendTimestamp = suspendTimestamp; this.pullFromThisOffset = pullFromThisOffset; this.subscriptionData = subscriptionData; + this.messageFilter = messageFilter; } public RemotingCommand getRequestCommand() { @@ -61,4 +65,8 @@ public class PullRequest { public SubscriptionData getSubscriptionData() { return subscriptionData; } + + public MessageFilter getMessageFilter() { + return messageFilter; + } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java index fdba50da..1a53db19 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java @@ -18,13 +18,13 @@ package org.apache.rocketmq.broker.longpolling; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.common.ServiceThread; import org.apache.rocketmq.common.SystemClock; import org.apache.rocketmq.common.constant.LoggerName; -import org.apache.rocketmq.store.DefaultMessageFilter; -import org.apache.rocketmq.store.MessageFilter; +import org.apache.rocketmq.store.ConsumeQueueExt; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,7 +33,6 @@ public class PullRequestHoldService extends ServiceThread { private static final String TOPIC_QUEUEID_SEPARATOR = "@"; private final BrokerController brokerController; private final SystemClock systemClock = new SystemClock(); - private final MessageFilter messageFilter = new DefaultMessageFilter(); private ConcurrentHashMap pullRequestTable = new ConcurrentHashMap(1024); @@ -110,10 +109,11 @@ public class PullRequestHoldService extends ServiceThread { } public void notifyMessageArriving(final String topic, final int queueId, final long maxOffset) { - notifyMessageArriving(topic, queueId, maxOffset, null); + notifyMessageArriving(topic, queueId, maxOffset, null, 0, null, null); } - public void notifyMessageArriving(final String topic, final int queueId, final long maxOffset, final Long tagsCode) { + public void notifyMessageArriving(final String topic, final int queueId, final long maxOffset, final Long tagsCode, + long msgStoreTime, byte[] filterBitMap, Map properties) { String key = this.buildKey(topic, queueId); ManyPullRequest mpr = this.pullRequestTable.get(key); if (mpr != null) { @@ -128,7 +128,14 @@ public class PullRequestHoldService extends ServiceThread { } if (newestOffset > request.getPullFromThisOffset()) { - if (this.messageFilter.isMessageMatched(request.getSubscriptionData(), tagsCode)) { + boolean match = request.getMessageFilter().isMatchedByConsumeQueue(tagsCode, + new ConsumeQueueExt.CqExtUnit(tagsCode, msgStoreTime, filterBitMap)); + // match by bit map, need eval again when properties is not null. + if (match && properties != null) { + match = request.getMessageFilter().isMatchedByCommitLog(null, properties); + } + + if (match) { try { this.brokerController.getPullMessageProcessor().executeRequestWhenWakeup(request.getClientChannel(), request.getRequestCommand()); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java index 039c9429..6c2a987d 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java @@ -50,7 +50,7 @@ import org.slf4j.LoggerFactory; public class BrokerOuterAPI { private static final Logger log = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); private final RemotingClient remotingClient; - private final TopAddressing topAddressing = new TopAddressing(MixAll.WS_ADDR); + private final TopAddressing topAddressing = new TopAddressing(MixAll.getWSAddr()); private String nameSrvAddr = null; public BrokerOuterAPI(final NettyClientConfig nettyClientConfig) { diff --git a/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java b/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java index 00257fd6..8ded973e 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java @@ -18,11 +18,14 @@ package org.apache.rocketmq.broker.plugin; import java.util.HashMap; +import java.util.LinkedList; import java.util.Set; import org.apache.rocketmq.common.message.MessageExt; -import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.store.CommitLogDispatcher; +import org.apache.rocketmq.store.ConsumeQueue; import org.apache.rocketmq.store.GetMessageResult; import org.apache.rocketmq.store.MessageExtBrokerInner; +import org.apache.rocketmq.store.MessageFilter; import org.apache.rocketmq.store.MessageStore; import org.apache.rocketmq.store.PutMessageResult; import org.apache.rocketmq.store.QueryMessageResult; @@ -84,8 +87,8 @@ public abstract class AbstractPluginMessageStore implements MessageStore { @Override public GetMessageResult getMessage(String group, String topic, int queueId, long offset, - int maxMsgNums, SubscriptionData subscriptionData) { - return next.getMessage(group, topic, queueId, offset, maxMsgNums, subscriptionData); + int maxMsgNums, final MessageFilter messageFilter) { + return next.getMessage(group, topic, queueId, offset, maxMsgNums, messageFilter); } @Override @@ -234,4 +237,13 @@ public abstract class AbstractPluginMessageStore implements MessageStore { next.setConfirmOffset(phyOffset); } + @Override + public LinkedList getDispatcherList() { + return next.getDispatcherList(); + } + + @Override + public ConsumeQueue getConsumeQueue(String topic, int queueId) { + return next.getConsumeQueue(topic, queueId); + } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java index e35316d6..daea53c7 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java @@ -16,6 +16,7 @@ */ package org.apache.rocketmq.broker.processor; +import com.alibaba.fastjson.JSON; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; import java.io.UnsupportedEncodingException; @@ -32,6 +33,8 @@ import java.util.concurrent.ConcurrentHashMap; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.client.ClientChannelInfo; import org.apache.rocketmq.broker.client.ConsumerGroupInfo; +import org.apache.rocketmq.broker.filter.ConsumerFilterData; +import org.apache.rocketmq.broker.filter.ExpressionMessageFilter; import org.apache.rocketmq.common.MQVersion; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.TopicConfig; @@ -49,6 +52,7 @@ import org.apache.rocketmq.common.protocol.ResponseCode; import org.apache.rocketmq.common.protocol.body.BrokerStatsData; import org.apache.rocketmq.common.protocol.body.BrokerStatsItem; import org.apache.rocketmq.common.protocol.body.Connection; +import org.apache.rocketmq.common.protocol.body.ConsumeQueueData; import org.apache.rocketmq.common.protocol.body.ConsumeStatsList; import org.apache.rocketmq.common.protocol.body.ConsumerConnection; import org.apache.rocketmq.common.protocol.body.GroupList; @@ -56,6 +60,7 @@ import org.apache.rocketmq.common.protocol.body.KVTable; import org.apache.rocketmq.common.protocol.body.LockBatchRequestBody; import org.apache.rocketmq.common.protocol.body.LockBatchResponseBody; import org.apache.rocketmq.common.protocol.body.ProducerConnection; +import org.apache.rocketmq.common.protocol.body.QueryConsumeQueueResponseBody; import org.apache.rocketmq.common.protocol.body.QueryConsumeTimeSpanBody; import org.apache.rocketmq.common.protocol.body.QueryCorrectionOffsetBody; import org.apache.rocketmq.common.protocol.body.QueueTimeSpan; @@ -81,6 +86,7 @@ import org.apache.rocketmq.common.protocol.header.GetMinOffsetRequestHeader; import org.apache.rocketmq.common.protocol.header.GetMinOffsetResponseHeader; import org.apache.rocketmq.common.protocol.header.GetProducerConnectionListRequestHeader; import org.apache.rocketmq.common.protocol.header.GetTopicStatsInfoRequestHeader; +import org.apache.rocketmq.common.protocol.header.QueryConsumeQueueRequestHeader; import org.apache.rocketmq.common.protocol.header.QueryConsumeTimeSpanRequestHeader; import org.apache.rocketmq.common.protocol.header.QueryCorrectionOffsetHeader; import org.apache.rocketmq.common.protocol.header.QueryTopicConsumeByWhoRequestHeader; @@ -94,6 +100,7 @@ import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; import org.apache.rocketmq.common.stats.StatsItem; import org.apache.rocketmq.common.stats.StatsSnapshot; import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig; +import org.apache.rocketmq.filter.util.BitsArray; import org.apache.rocketmq.remoting.common.RemotingHelper; import org.apache.rocketmq.remoting.exception.RemotingCommandException; import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; @@ -101,7 +108,10 @@ import org.apache.rocketmq.remoting.netty.NettyRequestProcessor; import org.apache.rocketmq.remoting.protocol.LanguageCode; import org.apache.rocketmq.remoting.protocol.RemotingCommand; import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.apache.rocketmq.store.ConsumeQueue; +import org.apache.rocketmq.store.ConsumeQueueExt; import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.MessageFilter; import org.apache.rocketmq.store.SelectMappedBufferResult; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -187,6 +197,8 @@ public class AdminBrokerProcessor implements NettyRequestProcessor { return ViewBrokerStatsData(ctx, request); case RequestCode.GET_BROKER_CONSUME_STATS: return fetchAllConsumeStatsInBroker(ctx, request); + case RequestCode.QUERY_CONSUME_QUEUE: + return queryConsumeQueue(ctx, request); default: break; } @@ -1244,4 +1256,83 @@ public class AdminBrokerProcessor implements NettyRequestProcessor { } } + private RemotingCommand queryConsumeQueue(ChannelHandlerContext ctx, RemotingCommand request) throws RemotingCommandException { + QueryConsumeQueueRequestHeader requestHeader = + (QueryConsumeQueueRequestHeader) request.decodeCommandCustomHeader(QueryConsumeQueueRequestHeader.class); + + RemotingCommand response = RemotingCommand.createResponseCommand(null); + + ConsumeQueue consumeQueue = this.brokerController.getMessageStore().getConsumeQueue(requestHeader.getTopic(), + requestHeader.getQueueId()); + if (consumeQueue == null) { + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark(String.format("%d@%s is not exist!", requestHeader.getQueueId(), requestHeader.getTopic())); + return response; + } + + QueryConsumeQueueResponseBody body = new QueryConsumeQueueResponseBody(); + response.setCode(ResponseCode.SUCCESS); + response.setBody(body.encode()); + + body.setMaxQueueIndex(consumeQueue.getMaxOffsetInQueue()); + body.setMinQueueIndex(consumeQueue.getMinOffsetInQueue()); + + MessageFilter messageFilter = null; + if (requestHeader.getConsumerGroup() != null) { + SubscriptionData subscriptionData = this.brokerController.getConsumerManager().findSubscriptionData( + requestHeader.getConsumerGroup(), requestHeader.getTopic() + ); + body.setSubscriptionData(subscriptionData); + if (subscriptionData == null) { + body.setFilterData(String.format("%s@%s is not online!", requestHeader.getConsumerGroup(), requestHeader.getTopic())); + } else { + ConsumerFilterData filterData = this.brokerController.getConsumerFilterManager() + .get(requestHeader.getTopic(), requestHeader.getConsumerGroup()); + body.setFilterData(JSON.toJSONString(filterData, true)); + + messageFilter = new ExpressionMessageFilter(subscriptionData, filterData, + this.brokerController.getConsumerFilterManager()); + } + } + + SelectMappedBufferResult result = consumeQueue.getIndexBuffer(requestHeader.getIndex()); + if (result == null) { + response.setRemark(String.format("Index %d of %d@%s is not exist!", requestHeader.getIndex(), requestHeader.getQueueId(), requestHeader.getTopic())); + return response; + } + try { + List queues = new ArrayList<>(); + for (int i = 0; i < result.getSize() && i < requestHeader.getCount() * ConsumeQueue.CQ_STORE_UNIT_SIZE; i += ConsumeQueue.CQ_STORE_UNIT_SIZE) { + ConsumeQueueData one = new ConsumeQueueData(); + one.setPhysicOffset(result.getByteBuffer().getLong()); + one.setPhysicSize(result.getByteBuffer().getInt()); + one.setTagsCode(result.getByteBuffer().getLong()); + + if (!consumeQueue.isExtAddr(one.getTagsCode())) { + queues.add(one); + continue; + } + + ConsumeQueueExt.CqExtUnit cqExtUnit = consumeQueue.getExt(one.getTagsCode()); + if (cqExtUnit != null) { + one.setExtendDataJson(JSON.toJSONString(cqExtUnit)); + if (cqExtUnit.getFilterBitMap() != null) { + one.setBitMap(BitsArray.create(cqExtUnit.getFilterBitMap()).toString()); + } + if (messageFilter != null) { + one.setEval(messageFilter.isMatchedByConsumeQueue(cqExtUnit.getTagsCode(), cqExtUnit)); + } + } else { + one.setMsg("Cq extend not exist!addr: " + one.getTagsCode()); + } + + queues.add(one); + } + body.setQueueData(queues); + } finally { + result.release(); + } + + return response; + } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java index 6349ffc9..67807a86 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java @@ -22,15 +22,19 @@ import org.apache.rocketmq.broker.client.ClientChannelInfo; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.constant.PermName; +import org.apache.rocketmq.common.filter.ExpressionType; import org.apache.rocketmq.common.protocol.RequestCode; import org.apache.rocketmq.common.protocol.ResponseCode; +import org.apache.rocketmq.common.protocol.body.CheckClientRequestBody; import org.apache.rocketmq.common.protocol.header.UnregisterClientRequestHeader; import org.apache.rocketmq.common.protocol.header.UnregisterClientResponseHeader; import org.apache.rocketmq.common.protocol.heartbeat.ConsumerData; import org.apache.rocketmq.common.protocol.heartbeat.HeartbeatData; import org.apache.rocketmq.common.protocol.heartbeat.ProducerData; +import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig; import org.apache.rocketmq.common.sysflag.TopicSysFlag; +import org.apache.rocketmq.filter.FilterFactory; import org.apache.rocketmq.remoting.common.RemotingHelper; import org.apache.rocketmq.remoting.exception.RemotingCommandException; import org.apache.rocketmq.remoting.netty.NettyRequestProcessor; @@ -54,6 +58,8 @@ public class ClientManageProcessor implements NettyRequestProcessor { return this.heartBeat(ctx, request); case RequestCode.UNREGISTER_CLIENT: return this.unregisterClient(ctx, request); + case RequestCode.CHECK_CLIENT_CONFIG: + return this.checkClientConfig(ctx, request); default: break; } @@ -157,4 +163,42 @@ public class ClientManageProcessor implements NettyRequestProcessor { response.setRemark(null); return response; } + + public RemotingCommand checkClientConfig(ChannelHandlerContext ctx, RemotingCommand request) + throws RemotingCommandException { + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + + CheckClientRequestBody requestBody = CheckClientRequestBody.decode(request.getBody(), + CheckClientRequestBody.class); + + if (requestBody != null && requestBody.getSubscriptionData() != null) { + SubscriptionData subscriptionData = requestBody.getSubscriptionData(); + + if (ExpressionType.isTagType(subscriptionData.getExpressionType())) { + response.setCode(ResponseCode.SUCCESS); + response.setRemark(null); + return response; + } + + if (!this.brokerController.getBrokerConfig().isEnablePropertyFilter()) { + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark("The broker does not support consumer to filter message by " + subscriptionData.getExpressionType()); + return response; + } + + try { + FilterFactory.INSTANCE.get(subscriptionData.getExpressionType()).compile(subscriptionData.getSubString()); + } catch (Exception e) { + log.warn("Client {}@{} filter message, but failed to compile expression! sub={}, error={}", + requestBody.getClientId(), requestBody.getGroup(), requestBody.getSubscriptionData(), e.getMessage()); + response.setCode(ResponseCode.SUBSCRIPTION_PARSE_FAILED); + response.setRemark(e.getMessage()); + return response; + } + } + + response.setCode(ResponseCode.SUCCESS); + response.setRemark(null); + return response; + } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java index 89967d8f..10945da9 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java @@ -25,6 +25,10 @@ import java.nio.ByteBuffer; import java.util.List; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.client.ConsumerGroupInfo; +import org.apache.rocketmq.broker.filter.ConsumerFilterData; +import org.apache.rocketmq.broker.filter.ConsumerFilterManager; +import org.apache.rocketmq.broker.filter.ExpressionForRetryMessageFilter; +import org.apache.rocketmq.broker.filter.ExpressionMessageFilter; import org.apache.rocketmq.broker.longpolling.PullRequest; import org.apache.rocketmq.broker.mqtrace.ConsumeMessageContext; import org.apache.rocketmq.broker.mqtrace.ConsumeMessageHook; @@ -34,6 +38,7 @@ import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.TopicFilterType; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.constant.PermName; +import org.apache.rocketmq.common.filter.ExpressionType; import org.apache.rocketmq.common.filter.FilterAPI; import org.apache.rocketmq.common.help.FAQUrl; import org.apache.rocketmq.common.message.MessageDecoder; @@ -54,6 +59,7 @@ import org.apache.rocketmq.remoting.netty.RequestTask; import org.apache.rocketmq.remoting.protocol.RemotingCommand; import org.apache.rocketmq.store.GetMessageResult; import org.apache.rocketmq.store.MessageExtBrokerInner; +import org.apache.rocketmq.store.MessageFilter; import org.apache.rocketmq.store.PutMessageResult; import org.apache.rocketmq.store.config.BrokerRole; import org.apache.rocketmq.store.stats.BrokerStatsManager; @@ -142,13 +148,22 @@ public class PullMessageProcessor implements NettyRequestProcessor { } SubscriptionData subscriptionData = null; + ConsumerFilterData consumerFilterData = null; if (hasSubscriptionFlag) { try { - subscriptionData = FilterAPI.buildSubscriptionData(requestHeader.getConsumerGroup(), requestHeader.getTopic(), - requestHeader.getSubscription()); + subscriptionData = FilterAPI.build( + requestHeader.getTopic(), requestHeader.getSubscription(), requestHeader.getExpressionType() + ); + if (!ExpressionType.isTagType(subscriptionData.getExpressionType())) { + consumerFilterData = ConsumerFilterManager.build( + requestHeader.getTopic(), requestHeader.getConsumerGroup(), requestHeader.getSubscription(), + requestHeader.getExpressionType(), requestHeader.getSubVersion() + ); + assert consumerFilterData != null; + } } catch (Exception e) { LOG.warn("Parse the consumer's subscription[{}] failed, group: {}", requestHeader.getSubscription(), // - requestHeader.getConsumerGroup()); + requestHeader.getConsumerGroup()); response.setCode(ResponseCode.SUBSCRIPTION_PARSE_FAILED); response.setRemark("parse the consumer's subscription failed"); return response; @@ -180,16 +195,48 @@ public class PullMessageProcessor implements NettyRequestProcessor { if (subscriptionData.getSubVersion() < requestHeader.getSubVersion()) { LOG.warn("The broker's subscription is not latest, group: {} {}", requestHeader.getConsumerGroup(), - subscriptionData.getSubString()); + subscriptionData.getSubString()); response.setCode(ResponseCode.SUBSCRIPTION_NOT_LATEST); response.setRemark("the consumer's subscription not latest"); return response; } + if (!ExpressionType.isTagType(subscriptionData.getExpressionType())) { + consumerFilterData = this.brokerController.getConsumerFilterManager().get(requestHeader.getTopic(), + requestHeader.getConsumerGroup()); + if (consumerFilterData == null) { + response.setCode(ResponseCode.FILTER_DATA_NOT_EXIST); + response.setRemark("The broker's consumer filter data is not exist!Your expression may be wrong!"); + return response; + } + if (consumerFilterData.getClientVersion() < requestHeader.getSubVersion()) { + LOG.warn("The broker's consumer filter data is not latest, group: {}, topic: {}, serverV: {}, clientV: {}", + requestHeader.getConsumerGroup(), requestHeader.getTopic(), consumerFilterData.getClientVersion(), requestHeader.getSubVersion()); + response.setCode(ResponseCode.FILTER_DATA_NOT_LATEST); + response.setRemark("the consumer's consumer filter data not latest"); + return response; + } + } + } + + if (!ExpressionType.isTagType(subscriptionData.getExpressionType()) + && !this.brokerController.getBrokerConfig().isEnablePropertyFilter()) { + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark("The broker does not support consumer to filter message by " + subscriptionData.getExpressionType()); + return response; + } + + MessageFilter messageFilter; + if (this.brokerController.getBrokerConfig().isFilterSupportRetry()) { + messageFilter = new ExpressionForRetryMessageFilter(subscriptionData, consumerFilterData, + this.brokerController.getConsumerFilterManager()); + } else { + messageFilter = new ExpressionMessageFilter(subscriptionData, consumerFilterData, + this.brokerController.getConsumerFilterManager()); } final GetMessageResult getMessageResult = this.brokerController.getMessageStore().getMessage(requestHeader.getConsumerGroup(), requestHeader.getTopic(), - requestHeader.getQueueId(), requestHeader.getQueueOffset(), requestHeader.getMaxMsgNums(), subscriptionData); + requestHeader.getQueueId(), requestHeader.getQueueOffset(), requestHeader.getMaxMsgNums(), messageFilter); if (getMessageResult != null) { response.setRemark(getMessageResult.getStatus().name()); responseHeader.setNextBeginOffset(getMessageResult.getNextBeginOffset()); @@ -368,7 +415,7 @@ public class PullMessageProcessor implements NettyRequestProcessor { long offset = requestHeader.getQueueOffset(); int queueId = requestHeader.getQueueId(); PullRequest pullRequest = new PullRequest(request, channel, pollingTimeMills, - this.brokerController.getMessageStore().now(), offset, subscriptionData); + this.brokerController.getMessageStore().now(), offset, subscriptionData, messageFilter); this.brokerController.getPullRequestHoldService().suspendPullRequest(topic, queueId, pullRequest); response = null; break; diff --git a/broker/src/test/java/org/apache/rocketmq/broker/filter/CommitLogDispatcherCalcBitMapTest.java b/broker/src/test/java/org/apache/rocketmq/broker/filter/CommitLogDispatcherCalcBitMapTest.java new file mode 100644 index 00000000..87f62561 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/filter/CommitLogDispatcherCalcBitMapTest.java @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.broker.filter; + +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.filter.ExpressionType; +import org.apache.rocketmq.filter.util.BitsArray; +import org.apache.rocketmq.store.DispatchRequest; +import org.junit.Test; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +import static org.assertj.core.api.Assertions.assertThat; + +public class CommitLogDispatcherCalcBitMapTest { + + @Test + public void testDispatch_filterDataIllegal() { + BrokerConfig brokerConfig = new BrokerConfig(); + brokerConfig.setEnableCalcFilterBitMap(true); + + ConsumerFilterManager filterManager = new ConsumerFilterManager(); + + filterManager.register("topic0", "CID_0", "a is not null and a >= 5", + ExpressionType.SQL92, System.currentTimeMillis()); + + filterManager.register("topic0", "CID_1", "a is not null and a >= 15", + ExpressionType.SQL92, System.currentTimeMillis()); + + ConsumerFilterData nullExpression = filterManager.get("topic0", "CID_0"); + nullExpression.setExpression(null); + nullExpression.setCompiledExpression(null); + ConsumerFilterData nullBloomData = filterManager.get("topic0", "CID_1"); + nullBloomData.setBloomFilterData(null); + + + CommitLogDispatcherCalcBitMap calcBitMap = new CommitLogDispatcherCalcBitMap(brokerConfig, + filterManager); + + for (int i = 0; i < 1; i++) { + Map properties = new HashMap(4); + properties.put("a", String.valueOf(i * 10 + 5)); + + String topic = "topic" + i; + + DispatchRequest dispatchRequest = new DispatchRequest( + topic, + 0, + i * 100 + 123, + 100, + (long) ("tags" + i).hashCode(), + System.currentTimeMillis(), + i, + null, + UUID.randomUUID().toString(), + 0, + 0, + properties + ); + + calcBitMap.dispatch(dispatchRequest); + + assertThat(dispatchRequest.getBitMap()).isNotNull(); + + BitsArray bitsArray = BitsArray.create(dispatchRequest.getBitMap(), + filterManager.getBloomFilter().getM()); + + for (int j = 0; j < bitsArray.bitLength(); j++) { + assertThat(bitsArray.getBit(j)).isFalse(); + } + } + } + + @Test + public void testDispatch_blankFilterData() { + BrokerConfig brokerConfig = new BrokerConfig(); + brokerConfig.setEnableCalcFilterBitMap(true); + + ConsumerFilterManager filterManager = new ConsumerFilterManager(); + + CommitLogDispatcherCalcBitMap calcBitMap = new CommitLogDispatcherCalcBitMap(brokerConfig, + filterManager); + + for (int i = 0; i < 10; i++) { + Map properties = new HashMap(4); + properties.put("a", String.valueOf(i * 10 + 5)); + + String topic = "topic" + i; + + DispatchRequest dispatchRequest = new DispatchRequest( + topic, + 0, + i * 100 + 123, + 100, + (long) ("tags" + i).hashCode(), + System.currentTimeMillis(), + i, + null, + UUID.randomUUID().toString(), + 0, + 0, + properties + ); + + calcBitMap.dispatch(dispatchRequest); + + assertThat(dispatchRequest.getBitMap()).isNull(); + } + } + + @Test + public void testDispatch() { + BrokerConfig brokerConfig = new BrokerConfig(); + brokerConfig.setEnableCalcFilterBitMap(true); + + ConsumerFilterManager filterManager = ConsumerFilterManagerTest.gen(10, 10); + + CommitLogDispatcherCalcBitMap calcBitMap = new CommitLogDispatcherCalcBitMap(brokerConfig, + filterManager); + + for (int i = 0; i < 10; i++) { + Map properties = new HashMap(4); + properties.put("a", String.valueOf(i * 10 + 5)); + + String topic = "topic" + i; + + DispatchRequest dispatchRequest = new DispatchRequest( + topic, + 0, + i * 100 + 123, + 100, + (long) ("tags" + i).hashCode(), + System.currentTimeMillis(), + i, + null, + UUID.randomUUID().toString(), + 0, + 0, + properties + ); + + calcBitMap.dispatch(dispatchRequest); + + assertThat(dispatchRequest.getBitMap()).isNotNull(); + + BitsArray bits = BitsArray.create(dispatchRequest.getBitMap()); + + Collection filterDatas = filterManager.get(topic); + + for (ConsumerFilterData filterData : filterDatas) { + + if (filterManager.getBloomFilter().isHit(filterData.getBloomFilterData(), bits)) { + try { + assertThat((Boolean) filterData.getCompiledExpression().evaluate( + new MessageEvaluationContext(properties) + )).isTrue(); + } catch (Exception e) { + e.printStackTrace(); + assertThat(true).isFalse(); + } + } else { + try { + assertThat((Boolean) filterData.getCompiledExpression().evaluate( + new MessageEvaluationContext(properties) + )).isFalse(); + } catch (Exception e) { + e.printStackTrace(); + assertThat(true).isFalse(); + } + } + } + } + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/filter/ConsumerFilterManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/filter/ConsumerFilterManagerTest.java new file mode 100644 index 00000000..c8412a8b --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/filter/ConsumerFilterManagerTest.java @@ -0,0 +1,291 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.broker.filter; + +import org.apache.rocketmq.common.filter.ExpressionType; +import org.apache.rocketmq.common.filter.FilterAPI; +import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import org.junit.Test; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +public class ConsumerFilterManagerTest { + + public static ConsumerFilterManager gen(int topicCount, int consumerCount) { + ConsumerFilterManager filterManager = new ConsumerFilterManager(); + + for (int i = 0; i < topicCount; i++) { + String topic = "topic" + i; + + for (int j = 0; j < consumerCount; j++) { + + String consumer = "CID_" + j; + + filterManager.register(topic, consumer, expr(j), ExpressionType.SQL92, System.currentTimeMillis()); + } + } + + return filterManager; + } + + public static String expr(int i) { + return "a is not null and a > " + ((i - 1) * 10) + " and a < " + ((i + 1) * 10); + } + + @Test + public void testRegister_newExpressionCompileErrorAndRemoveOld() { + ConsumerFilterManager filterManager = gen(10, 10); + + assertThat(filterManager.get("topic9", "CID_9")).isNotNull(); + + String newExpr = "a between 10,20"; + + assertThat(filterManager.register("topic9", "CID_9", newExpr, ExpressionType.SQL92, System.currentTimeMillis() + 1)) + .isFalse(); + assertThat(filterManager.get("topic9", "CID_9")).isNull(); + + newExpr = "a between 10 AND 20"; + + assertThat(filterManager.register("topic9", "CID_9", newExpr, ExpressionType.SQL92, System.currentTimeMillis() + 1)) + .isTrue(); + + ConsumerFilterData filterData = filterManager.get("topic9", "CID_9"); + + assertThat(filterData).isNotNull(); + assertThat(newExpr).isEqualTo(filterData.getExpression()); + } + + @Test + public void testRegister_change() { + ConsumerFilterManager filterManager = gen(10, 10); + + ConsumerFilterData filterData = filterManager.get("topic9", "CID_9"); + + System.out.println(filterData.getCompiledExpression()); + + String newExpr = "a > 0 and a < 10"; + + filterManager.register("topic9", "CID_9", newExpr, ExpressionType.SQL92, System.currentTimeMillis() + 1); + + filterData = filterManager.get("topic9", "CID_9"); + + assertThat(newExpr).isEqualTo(filterData.getExpression()); + + System.out.println(filterData.toString()); + } + + @Test + public void testRegister() { + ConsumerFilterManager filterManager = gen(10, 10); + + ConsumerFilterData filterData = filterManager.get("topic9", "CID_9"); + + assertThat(filterData).isNotNull(); + assertThat(filterData.isDead()).isFalse(); + + // new version + assertThat(filterManager.register( + "topic9", "CID_9", "a is not null", ExpressionType.SQL92, System.currentTimeMillis() + 1000 + )).isTrue(); + + ConsumerFilterData newFilter = filterManager.get("topic9", "CID_9"); + + assertThat(newFilter).isNotEqualTo(filterData); + + // same version + assertThat(filterManager.register( + "topic9", "CID_9", "a is null", ExpressionType.SQL92, newFilter.getClientVersion() + )).isFalse(); + + ConsumerFilterData filterData1 = filterManager.get("topic9", "CID_9"); + + assertThat(newFilter).isEqualTo(filterData1); + } + + @Test + public void testRegister_reAlive() { + ConsumerFilterManager filterManager = gen(10, 10); + + ConsumerFilterData filterData = filterManager.get("topic9", "CID_9"); + + assertThat(filterData).isNotNull(); + assertThat(filterData.isDead()).isFalse(); + + //make dead + filterManager.unRegister("CID_9"); + + //reAlive + filterManager.register( + filterData.getTopic(), + filterData.getConsumerGroup(), + filterData.getExpression(), + filterData.getExpressionType(), + System.currentTimeMillis() + ); + + ConsumerFilterData newFilterData = filterManager.get("topic9", "CID_9"); + + assertThat(newFilterData).isNotNull(); + assertThat(newFilterData.isDead()).isFalse(); + } + + @Test + public void testRegister_bySubscriptionData() { + ConsumerFilterManager filterManager = new ConsumerFilterManager(); + List subscriptionDatas = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + try { + subscriptionDatas.add( + FilterAPI.build( + "topic" + i, + "a is not null and a > " + i, + ExpressionType.SQL92 + ) + ); + } catch (Exception e) { + e.printStackTrace(); + assertThat(true).isFalse(); + } + } + + filterManager.register("CID_0", subscriptionDatas); + + Collection filterDatas = filterManager.getByGroup("CID_0"); + + assertThat(filterDatas).isNotNull(); + assertThat(filterDatas.size()).isEqualTo(10); + + Iterator iterator = filterDatas.iterator(); + while (iterator.hasNext()) { + ConsumerFilterData filterData = iterator.next(); + + assertThat(filterData).isNotNull(); + assertThat(filterManager.getBloomFilter().isValid(filterData.getBloomFilterData())).isTrue(); + } + } + + @Test + public void testRegister_tag() { + ConsumerFilterManager filterManager = new ConsumerFilterManager(); + + assertThat(filterManager.register("topic0", "CID_0", "*", null, System.currentTimeMillis())).isFalse(); + + Collection filterDatas = filterManager.getByGroup("CID_0"); + + assertThat(filterDatas).isNullOrEmpty(); + } + + @Test + public void testUnregister() { + ConsumerFilterManager filterManager = gen(10, 10); + + ConsumerFilterData filterData = filterManager.get("topic9", "CID_9"); + + assertThat(filterData).isNotNull(); + assertThat(filterData.isDead()).isFalse(); + + filterManager.unRegister("CID_9"); + + assertThat(filterData.isDead()).isTrue(); + } + + @Test + public void testPersist() { + ConsumerFilterManager filterManager = gen(10, 10); + + try { + filterManager.persist(); + + ConsumerFilterData filterData = filterManager.get("topic9", "CID_9"); + + assertThat(filterData).isNotNull(); + assertThat(filterData.isDead()).isFalse(); + + ConsumerFilterManager loadFilter = new ConsumerFilterManager(); + + assertThat(loadFilter.load()).isTrue(); + + filterData = loadFilter.get("topic9", "CID_9"); + + assertThat(filterData).isNotNull(); + assertThat(filterData.isDead()).isTrue(); + assertThat(filterData.getCompiledExpression()).isNotNull(); + } finally { + deleteDirectory("./unit_test"); + } + } + + @Test + public void testPersist_clean() { + ConsumerFilterManager filterManager = gen(10, 10); + + String topic = "topic9"; + for (int i = 0; i < 10; i++) { + String cid = "CID_" + i; + + ConsumerFilterData filterData = filterManager.get(topic, cid); + + assertThat(filterData).isNotNull(); + assertThat(filterData.isDead()).isFalse(); + + //make dead more than 24h + filterData.setBornTime(System.currentTimeMillis() - 26 * 60 * 60 * 1000); + filterData.setDeadTime(System.currentTimeMillis() - 25 * 60 * 60 * 1000); + } + + try { + filterManager.persist(); + + ConsumerFilterManager loadFilter = new ConsumerFilterManager(); + + assertThat(loadFilter.load()).isTrue(); + + ConsumerFilterData filterData = loadFilter.get(topic, "CID_9"); + + assertThat(filterData).isNull(); + + Collection topicData = loadFilter.get(topic); + + assertThat(topicData).isNullOrEmpty(); + } finally { + deleteDirectory("./unit_test"); + } + } + + protected void deleteDirectory(String rootPath) { + File file = new File(rootPath); + deleteFile(file); + } + + protected void deleteFile(File file) { + File[] subFiles = file.listFiles(); + if (subFiles != null) { + for (File sub : subFiles) { + deleteFile(sub); + } + } + + file.delete(); + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/filter/MessageStoreWithFilterTest.java b/broker/src/test/java/org/apache/rocketmq/broker/filter/MessageStoreWithFilterTest.java new file mode 100644 index 00000000..53e563ed --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/filter/MessageStoreWithFilterTest.java @@ -0,0 +1,392 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.broker.filter; + +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.filter.ExpressionType; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.store.CommitLogDispatcher; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.DispatchRequest; +import org.apache.rocketmq.store.GetMessageResult; +import org.apache.rocketmq.store.GetMessageStatus; +import org.apache.rocketmq.store.MessageArrivingListener; +import org.apache.rocketmq.store.MessageExtBrokerInner; +import org.apache.rocketmq.store.PutMessageResult; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.stats.BrokerStatsManager; +import org.junit.Test; + +import java.io.File; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.net.UnknownHostException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +public class MessageStoreWithFilterTest { + + private static final String msg = "Once, there was a chance for me!"; + private static final byte[] msgBody = msg.getBytes(); + + private static final String topic = "topic"; + private static final int queueId = 0; + private static final String storePath = "." + File.separator + "unit_test_store"; + private static final int commitLogFileSize = 1024 * 1024 * 256; + private static final int cqFileSize = 300000 * 20; + private static final int cqExtFileSize = 300000 * 128; + + private static SocketAddress BornHost; + + private static SocketAddress StoreHost; + + static { + try { + StoreHost = new InetSocketAddress(InetAddress.getLocalHost(), 8123); + } catch (UnknownHostException e) { + e.printStackTrace(); + } + try { + BornHost = new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 0); + } catch (UnknownHostException e) { + e.printStackTrace(); + } + } + + public MessageExtBrokerInner buildMessage() { + MessageExtBrokerInner msg = new MessageExtBrokerInner(); + msg.setTopic(topic); + msg.setTags("TAG1"); + msg.setKeys("Hello"); + msg.setBody(msgBody); + msg.setKeys(String.valueOf(System.currentTimeMillis())); + msg.setQueueId(queueId); + msg.setSysFlag(0); + msg.setBornTimestamp(System.currentTimeMillis()); + msg.setStoreHost(StoreHost); + msg.setBornHost(BornHost); + for (int i = 1; i < 3; i++) { + msg.putUserProperty(String.valueOf(i), "imagoodperson" + i); + } + msg.setPropertiesString(MessageDecoder.messageProperties2String(msg.getProperties())); + + return msg; + } + + public MessageStoreConfig buildStoreConfig(int commitLogFileSize, int cqFileSize, + boolean enableCqExt, int cqExtFileSize) { + MessageStoreConfig messageStoreConfig = new MessageStoreConfig(); + messageStoreConfig.setMapedFileSizeCommitLog(commitLogFileSize); + messageStoreConfig.setMapedFileSizeConsumeQueue(cqFileSize); + messageStoreConfig.setMappedFileSizeConsumeQueueExt(cqExtFileSize); + messageStoreConfig.setMessageIndexEnable(false); + messageStoreConfig.setEnableConsumeQueueExt(enableCqExt); + + messageStoreConfig.setStorePathRootDir(storePath); + messageStoreConfig.setStorePathCommitLog(storePath + File.separator + "commitlog"); + + return messageStoreConfig; + } + + protected DefaultMessageStore gen(ConsumerFilterManager filterManager) throws Exception { + MessageStoreConfig messageStoreConfig = buildStoreConfig( + commitLogFileSize, cqFileSize, true, cqExtFileSize + ); + + BrokerConfig brokerConfig = new BrokerConfig(); + brokerConfig.setEnableCalcFilterBitMap(true); + brokerConfig.setMaxErrorRateOfBloomFilter(20); + brokerConfig.setExpectConsumerNumUseFilter(64); + + DefaultMessageStore master = new DefaultMessageStore( + messageStoreConfig, + new BrokerStatsManager(brokerConfig.getBrokerClusterName()), + new MessageArrivingListener() { + @Override + public void arriving(String topic, int queueId, long logicOffset, long tagsCode, + long msgStoreTime, byte[] filterBitMap, Map properties) { +// System.out.println(String.format("Msg coming: %s, %d, %d, %d", +// topic, queueId, logicOffset, tagsCode)); + } + } + , brokerConfig); + + master.getDispatcherList().addFirst(new CommitLogDispatcher() { + @Override + public void dispatch(DispatchRequest request) { + try { +// System.out.println(String.format("offset:%d, bitMap:%s", request.getCommitLogOffset(), +// BitsArray.create(request.getBitMap()).toString())); + } catch (Throwable e) { + e.printStackTrace(); + } + } + }); + master.getDispatcherList().addFirst(new CommitLogDispatcherCalcBitMap(brokerConfig, filterManager)); + + assertThat(master.load()).isTrue(); + + master.start(); + + return master; + } + + protected List putMsg(DefaultMessageStore master, int topicCount, int msgCountPerTopic) throws Exception { + List msgs = new ArrayList(); + for (int i = 0; i < topicCount; i++) { + String realTopic = topic + i; + for (int j = 0; j < msgCountPerTopic; j++) { + MessageExtBrokerInner msg = buildMessage(); + msg.setTopic(realTopic); + msg.putUserProperty("a", String.valueOf(j * 10 + 5)); + msg.setPropertiesString(MessageDecoder.messageProperties2String(msg.getProperties())); + + PutMessageResult result = master.putMessage(msg); + + msg.setMsgId(result.getAppendMessageResult().getMsgId()); + + msgs.add(msg); + } + } + + return msgs; + } + + protected void deleteDirectory(String rootPath) { + File file = new File(rootPath); + deleteFile(file); + } + + protected void deleteFile(File file) { + File[] subFiles = file.listFiles(); + if (subFiles != null) { + for (File sub : subFiles) { + deleteFile(sub); + } + } + + file.delete(); + } + + protected List filtered(List msgs, ConsumerFilterData filterData) { + List filteredMsgs = new ArrayList(); + + for (MessageExtBrokerInner messageExtBrokerInner : msgs) { + + if (!messageExtBrokerInner.getTopic().equals(filterData.getTopic())) { + continue; + } + + try { + Object evlRet = filterData.getCompiledExpression().evaluate(new MessageEvaluationContext(messageExtBrokerInner.getProperties())); + + if (evlRet == null || !(evlRet instanceof Boolean) || (Boolean) evlRet) { + filteredMsgs.add(messageExtBrokerInner); + } + } catch (Exception e) { + e.printStackTrace(); + assertThat(true).isFalse(); + } + } + + return filteredMsgs; + } + + @Test + public void testGetMessage_withFilterBitMapAndConsumerChanged() { + int topicCount = 10, msgPerTopic = 10; + ConsumerFilterManager filterManager = ConsumerFilterManagerTest.gen(topicCount, msgPerTopic); + + DefaultMessageStore master = null; + try { + master = gen(filterManager); + } catch (Exception e) { + e.printStackTrace(); + assertThat(true).isFalse(); + } + + try { + List msgs = null; + try { + msgs = putMsg(master, topicCount, msgPerTopic); + } catch (Exception e) { + e.printStackTrace(); + assertThat(true).isFalse(); + } + + // sleep to wait for consume queue has been constructed. + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + e.printStackTrace(); + assertThat(true).isFalse(); + } + + // reset consumer; + String topic = "topic" + 0; + String resetGroup = "CID_" + 2; + String normalGroup = "CID_" + 3; + + { + // reset CID_2@topic0 to get all messages. + SubscriptionData resetSubData = new SubscriptionData(); + resetSubData.setExpressionType(ExpressionType.SQL92); + resetSubData.setTopic(topic); + resetSubData.setClassFilterMode(false); + resetSubData.setSubString("a is not null OR a is null"); + + ConsumerFilterData resetFilterData = ConsumerFilterManager.build(topic, + resetGroup, resetSubData.getSubString(), resetSubData.getExpressionType(), + System.currentTimeMillis()); + + GetMessageResult resetGetResult = master.getMessage(resetGroup, topic, queueId, 0, 1000, + new ExpressionMessageFilter(resetSubData, resetFilterData, filterManager)); + + try { + assertThat(resetGetResult).isNotNull(); + + List filteredMsgs = filtered(msgs, resetFilterData); + + assertThat(resetGetResult.getMessageBufferList().size()).isEqualTo(filteredMsgs.size()); + } finally { + resetGetResult.release(); + } + } + + { + ConsumerFilterData normalFilterData = filterManager.get(topic, normalGroup); + assertThat(normalFilterData).isNotNull(); + assertThat(normalFilterData.getBornTime()).isLessThan(System.currentTimeMillis()); + + SubscriptionData normalSubData = new SubscriptionData(); + normalSubData.setExpressionType(normalFilterData.getExpressionType()); + normalSubData.setTopic(topic); + normalSubData.setClassFilterMode(false); + normalSubData.setSubString(normalFilterData.getExpression()); + + List filteredMsgs = filtered(msgs, normalFilterData); + + GetMessageResult normalGetResult = master.getMessage(normalGroup, topic, queueId, 0, 1000, + new ExpressionMessageFilter(normalSubData, normalFilterData, filterManager)); + + try { + assertThat(normalGetResult).isNotNull(); + assertThat(normalGetResult.getMessageBufferList().size()).isEqualTo(filteredMsgs.size()); + } finally { + normalGetResult.release(); + } + } + } finally { + master.shutdown(); + master.destroy(); + deleteDirectory(storePath); + } + } + + @Test + public void testGetMessage_withFilterBitMap() { + int topicCount = 10, msgPerTopic = 500; + ConsumerFilterManager filterManager = ConsumerFilterManagerTest.gen(topicCount, msgPerTopic); + + DefaultMessageStore master = null; + try { + master = gen(filterManager); + } catch (Exception e) { + e.printStackTrace(); + assertThat(true).isFalse(); + } + + try { + List msgs = null; + try { + msgs = putMsg(master, topicCount, msgPerTopic); + // sleep to wait for consume queue has been constructed. + Thread.sleep(1000); + } catch (Exception e) { + e.printStackTrace(); + assertThat(true).isFalse(); + } + + for (int i = 0; i < topicCount; i++) { + String realTopic = topic + i; + + for (int j = 0; j < msgPerTopic; j++) { + String group = "CID_" + j; + + ConsumerFilterData filterData = filterManager.get(realTopic, group); + assertThat(filterData).isNotNull(); + + List filteredMsgs = filtered(msgs, filterData); + + SubscriptionData subscriptionData = new SubscriptionData(); + subscriptionData.setExpressionType(filterData.getExpressionType()); + subscriptionData.setTopic(filterData.getTopic()); + subscriptionData.setClassFilterMode(false); + subscriptionData.setSubString(filterData.getExpression()); + + GetMessageResult getMessageResult = master.getMessage(group, realTopic, queueId, 0, 10000, + new ExpressionMessageFilter(subscriptionData, filterData, filterManager)); + String assertMsg = group + "-" + realTopic; + try { + assertThat(getMessageResult).isNotNull(); + assertThat(GetMessageStatus.FOUND).isEqualTo(getMessageResult.getStatus()); + assertThat(getMessageResult.getMessageBufferList()).isNotNull().isNotEmpty(); + assertThat(getMessageResult.getMessageBufferList().size()).isEqualTo(filteredMsgs.size()); + + for (ByteBuffer buffer : getMessageResult.getMessageBufferList()) { + MessageExt messageExt = MessageDecoder.decode(buffer.slice(), false); + assertThat(messageExt).isNotNull(); + + Object evlRet = null; + try { + evlRet = filterData.getCompiledExpression().evaluate(new MessageEvaluationContext(messageExt.getProperties())); + } catch (Exception e) { + e.printStackTrace(); + assertThat(true).isFalse(); + } + + assertThat(evlRet).isNotNull().isEqualTo(Boolean.TRUE); + + // check + boolean find = false; + for (MessageExtBrokerInner messageExtBrokerInner : filteredMsgs) { + if (messageExtBrokerInner.getMsgId().equals(messageExt.getMsgId())) { + find = true; + } + } + assertThat(find).isTrue(); + } + } finally { + getMessageResult.release(); + } + } + } + } finally { + master.shutdown(); + master.destroy(); + deleteDirectory(storePath); + } + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/PullMessageProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/PullMessageProcessorTest.java index d3d98120..941d4a77 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/processor/PullMessageProcessorTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/PullMessageProcessorTest.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Set; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.client.ClientChannelInfo; +import org.apache.rocketmq.broker.filter.ExpressionMessageFilter; import org.apache.rocketmq.broker.mqtrace.ConsumeMessageContext; import org.apache.rocketmq.broker.mqtrace.ConsumeMessageHook; import org.apache.rocketmq.common.BrokerConfig; @@ -126,7 +127,7 @@ public class PullMessageProcessorTest { @Test public void testProcessRequest_Found() throws RemotingCommandException { GetMessageResult getMessageResult = createGetMessageResult(); - when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(SubscriptionData.class))).thenReturn(getMessageResult); + when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(ExpressionMessageFilter.class))).thenReturn(getMessageResult); final RemotingCommand request = createPullMsgCommand(RequestCode.PULL_MESSAGE); RemotingCommand response = pullMessageProcessor.processRequest(handlerContext, request); @@ -137,7 +138,7 @@ public class PullMessageProcessorTest { @Test public void testProcessRequest_FoundWithHook() throws RemotingCommandException { GetMessageResult getMessageResult = createGetMessageResult(); - when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(SubscriptionData.class))).thenReturn(getMessageResult); + when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(ExpressionMessageFilter.class))).thenReturn(getMessageResult); List consumeMessageHookList = new ArrayList<>(); final ConsumeMessageContext[] messageContext = new ConsumeMessageContext[1]; ConsumeMessageHook consumeMessageHook = new ConsumeMessageHook() { @@ -168,7 +169,7 @@ public class PullMessageProcessorTest { public void testProcessRequest_MsgWasRemoving() throws RemotingCommandException { GetMessageResult getMessageResult = createGetMessageResult(); getMessageResult.setStatus(GetMessageStatus.MESSAGE_WAS_REMOVING); - when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(SubscriptionData.class))).thenReturn(getMessageResult); + when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(ExpressionMessageFilter.class))).thenReturn(getMessageResult); final RemotingCommand request = createPullMsgCommand(RequestCode.PULL_MESSAGE); RemotingCommand response = pullMessageProcessor.processRequest(handlerContext, request); @@ -180,7 +181,7 @@ public class PullMessageProcessorTest { public void testProcessRequest_NoMsgInQueue() throws RemotingCommandException { GetMessageResult getMessageResult = createGetMessageResult(); getMessageResult.setStatus(GetMessageStatus.NO_MESSAGE_IN_QUEUE); - when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(SubscriptionData.class))).thenReturn(getMessageResult); + when(messageStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any(ExpressionMessageFilter.class))).thenReturn(getMessageResult); final RemotingCommand request = createPullMsgCommand(RequestCode.PULL_MESSAGE); RemotingCommand response = pullMessageProcessor.processRequest(handlerContext, request); diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java index 3903fe2b..9c9b59ef 100644 --- a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java @@ -518,6 +518,21 @@ public class DefaultMQPushConsumer extends ClientConfig implements MQPushConsume this.defaultMQPushConsumerImpl.subscribe(topic, fullClassName, filterClassSource); } + /** + * Subscribe a topic by message selector. + * + * @see org.apache.rocketmq.client.consumer.MessageSelector#bySql + * @see org.apache.rocketmq.client.consumer.MessageSelector#byTag + * + * @param topic topic to consume. + * @param messageSelector {@link org.apache.rocketmq.client.consumer.MessageSelector} + * @throws MQClientException + */ + @Override + public void subscribe(final String topic, final MessageSelector messageSelector) throws MQClientException { + this.defaultMQPushConsumerImpl.subscribe(topic, messageSelector); + } + /** * Un-subscribe the specified topic from subscription. * @param topic message topic diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/MQPushConsumer.java b/client/src/main/java/org/apache/rocketmq/client/consumer/MQPushConsumer.java index 92552815..9c6c1f17 100644 --- a/client/src/main/java/org/apache/rocketmq/client/consumer/MQPushConsumer.java +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/MQPushConsumer.java @@ -69,6 +69,27 @@ public interface MQPushConsumer extends MQConsumer { */ void subscribe(final String topic, final String fullClassName, final String filterClassSource) throws MQClientException; + /** + * Subscribe some topic with selector. + *

+ * This interface also has the ability of {@link #subscribe(String, String)}, + * and, support other message selection, such as {@link org.apache.rocketmq.common.filter.ExpressionType#SQL92}. + *

+ *

+ *

+ * Choose Tag: {@link MessageSelector#byTag(java.lang.String)} + *

+ *

+ *

+ * Choose SQL92: {@link MessageSelector#bySql(java.lang.String)} + *

+ * + * @param topic + * @param selector message selector({@link MessageSelector}), can be null. + * @throws MQClientException + */ + void subscribe(final String topic, final MessageSelector selector) throws MQClientException; + /** * Unsubscribe consumption some topic * diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/MessageSelector.java b/client/src/main/java/org/apache/rocketmq/client/consumer/MessageSelector.java new file mode 100644 index 00000000..35a5181f --- /dev/null +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/MessageSelector.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.client.consumer; + +import org.apache.rocketmq.common.filter.ExpressionType; + +/** + * + * Message selector: select message at server. + *

+ * Now, support: + *

  • Tag: {@link org.apache.rocketmq.common.filter.ExpressionType#TAG} + *
  • + *
  • SQL92: {@link org.apache.rocketmq.common.filter.ExpressionType#SQL92} + *
  • + *

    + */ +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; + } +} diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/FindBrokerResult.java b/client/src/main/java/org/apache/rocketmq/client/impl/FindBrokerResult.java index 295060ec..4367a4c1 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/FindBrokerResult.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/FindBrokerResult.java @@ -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; + } } diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java index ff25334f..4244bddc 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java @@ -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 diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java index 67f3ebe2..2cafe29a 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java @@ -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()); diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/PullAPIWrapper.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/PullAPIWrapper.java index 96e21e1f..304a44a6 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/PullAPIWrapper.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/PullAPIWrapper.java @@ -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; diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java index d7e02fe3..a8c65b29 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java @@ -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> brokerAddrTable = new ConcurrentHashMap>(); + private final ConcurrentHashMap> brokerVersionTable = + new ConcurrentHashMap>(); 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> it = this.consumerTable.entrySet().iterator(); + + while (it.hasNext()) { + Entry entry = it.next(); + Set 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(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 findConsumerIdList(final String topic, final String group) { String brokerAddr = this.findBrokerAddrByTopic(topic); if (null == brokerAddr) { diff --git a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java index f79f7267..f0a73bd2 100644 --- a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java +++ b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java @@ -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; + } } diff --git a/common/src/main/java/org/apache/rocketmq/common/MixAll.java b/common/src/main/java/org/apache/rocketmq/common/MixAll.java index 4a54a602..e75efd9a 100644 --- a/common/src/main/java/org/apache/rocketmq/common/MixAll.java +++ b/common/src/main/java/org/apache/rocketmq/common/MixAll.java @@ -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; } diff --git a/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java b/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java index e706e28a..385c1214 100644 --- a/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java +++ b/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java @@ -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"; } diff --git a/common/src/main/java/org/apache/rocketmq/common/filter/ExpressionType.java b/common/src/main/java/org/apache/rocketmq/common/filter/ExpressionType.java new file mode 100644 index 00000000..3b7940ab --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/filter/ExpressionType.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.common.filter; + +public class ExpressionType { + + /** + *
      + * Keywords: + *
    • {@code AND, OR, NOT, BETWEEN, IN, TRUE, FALSE, IS, NULL}
    • + *
    + *

    + *

      + * Data type: + *
    • Boolean, like: TRUE, FALSE
    • + *
    • String, like: 'abc'
    • + *
    • Decimal, like: 123
    • + *
    • Float number, like: 3.1415
    • + *
    + *

    + *

      + * Grammar: + *
    • {@code AND, OR}
    • + *
    • {@code >, >=, <, <=, =}
    • + *
    • {@code BETWEEN A AND B}, equals to {@code >=A AND <=B}
    • + *
    • {@code NOT BETWEEN A AND B}, equals to {@code >B OR + *
    • {@code IN ('a', 'b')}, equals to {@code ='a' OR ='b'}, this operation only support String type.
    • + *
    • {@code IS NULL}, {@code IS NOT NULL}, check parameter whether is null, or not.
    • + *
    • {@code =TRUE}, {@code =FALSE}, check parameter whether is true, or false.
    • + *
    + *

    + *

    + * Example: + * (a > 10 AND a < 100) OR (b IS NOT NULL AND b=TRUE) + *

    + */ + public static final String SQL92 = "SQL92"; + + /** + * Only support or operation such as + * "tag1 || tag2 || tag3",
    + * If null or * expression,meaning subscribe all. + */ + public static final String TAG = "TAG"; + + public static boolean isTagType(String type) { + if (type == null || TAG.equals(type)) { + return true; + } + return false; + } +} diff --git a/common/src/main/java/org/apache/rocketmq/common/filter/FilterAPI.java b/common/src/main/java/org/apache/rocketmq/common/filter/FilterAPI.java index e9bf3fa2..fc8525c1 100644 --- a/common/src/main/java/org/apache/rocketmq/common/filter/FilterAPI.java +++ b/common/src/main/java/org/apache/rocketmq/common/filter/FilterAPI.java @@ -63,4 +63,22 @@ public class FilterAPI { return subscriptionData; } + + public static SubscriptionData build(final String topic, final String subString, + final String type) throws Exception { + if (ExpressionType.TAG.equals(type) || type == null) { + return buildSubscriptionData(null, topic, subString); + } + + if (subString == null || subString.length() < 1) { + throw new IllegalArgumentException("Expression can't be null! " + type); + } + + SubscriptionData subscriptionData = new SubscriptionData(); + subscriptionData.setTopic(topic); + subscriptionData.setSubString(subString); + subscriptionData.setExpressionType(type); + + return subscriptionData; + } } diff --git a/common/src/main/java/org/apache/rocketmq/common/message/Message.java b/common/src/main/java/org/apache/rocketmq/common/message/Message.java index f3bff832..2c81f5cd 100644 --- a/common/src/main/java/org/apache/rocketmq/common/message/Message.java +++ b/common/src/main/java/org/apache/rocketmq/common/message/Message.java @@ -81,6 +81,12 @@ public class Message implements Serializable { throw new RuntimeException(String.format( "The Property<%s> is used by system, input another please", name)); } + if (value == null || value == "" || value.trim() == "" + || name == null || name == "" || name.trim() == "") { + throw new IllegalArgumentException( + "The name or value of property can not be null or blank string!" + ); + } this.putProperty(name, value); } diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java index 90b837a3..e41ec9d2 100644 --- a/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java +++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java @@ -41,6 +41,20 @@ public class MessageDecoder { public final static int MESSAGE_MAGIC_CODE = 0xAABBCCDD ^ 1880681586 + 8; public static final char NAME_VALUE_SEPARATOR = 1; public static final char PROPERTY_SEPARATOR = 2; + public static final int BODY_SIZE_POSITION = 4 // 1 TOTALSIZE + + 4 // 2 MAGICCODE + + 4 // 3 BODYCRC + + 4 // 4 QUEUEID + + 4 // 5 FLAG + + 8 // 6 QUEUEOFFSET + + 8 // 7 PHYSICALOFFSET + + 4 // 8 SYSFLAG + + 8 // 9 BORNTIMESTAMP + + 8 // 10 BORNHOST + + 8 // 11 STORETIMESTAMP + + 8 // 12 STOREHOSTADDRESS + + 4 // 13 RECONSUMETIMES + + 8; // 14 Prepared Transaction Offset public static String createMessageId(final ByteBuffer input, final ByteBuffer addr, final long offset) { input.flip(); @@ -80,6 +94,31 @@ public class MessageDecoder { return new MessageId(address, offset); } + /** + * Just decode properties from msg buffer. + * + * @param byteBuffer msg commit log buffer. + * @return + */ + public static Map decodeProperties(java.nio.ByteBuffer byteBuffer) { + int topicLengthPosition = BODY_SIZE_POSITION + 4 + byteBuffer.getInt(BODY_SIZE_POSITION); + + byte topicLength = byteBuffer.get(topicLengthPosition); + + short propertiesLength = byteBuffer.getShort(topicLengthPosition + 1 + topicLength); + + byteBuffer.position(topicLengthPosition + 1 + topicLength + 2); + + if (propertiesLength > 0) { + byte[] properties = new byte[propertiesLength]; + byteBuffer.get(properties); + String propertiesString = new String(properties, CHARSET_UTF8); + Map map = string2messageProperties(propertiesString); + return map; + } + return null; + } + public static MessageExt decode(java.nio.ByteBuffer byteBuffer) { return decode(byteBuffer, true, true, false); } diff --git a/common/src/main/java/org/apache/rocketmq/common/namesrv/TopAddressing.java b/common/src/main/java/org/apache/rocketmq/common/namesrv/TopAddressing.java index 74fd9654..990e7486 100644 --- a/common/src/main/java/org/apache/rocketmq/common/namesrv/TopAddressing.java +++ b/common/src/main/java/org/apache/rocketmq/common/namesrv/TopAddressing.java @@ -88,7 +88,7 @@ public class TopAddressing { if (verbose) { String errorMsg = - "connect to " + url + " failed, maybe the domain name " + MixAll.WS_DOMAIN_NAME + " not bind in /etc/hosts"; + "connect to " + url + " failed, maybe the domain name " + MixAll.getWSAddr() + " not bind in /etc/hosts"; errorMsg += FAQUrl.suggestTodo(FAQUrl.NAME_SERVER_ADDR_NOT_EXIST_URL); log.warn(errorMsg); diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/RequestCode.java b/common/src/main/java/org/apache/rocketmq/common/protocol/RequestCode.java index c6b09255..6f132f7c 100644 --- a/common/src/main/java/org/apache/rocketmq/common/protocol/RequestCode.java +++ b/common/src/main/java/org/apache/rocketmq/common/protocol/RequestCode.java @@ -68,6 +68,8 @@ public class RequestCode { public static final int GET_ALL_DELAY_OFFSET = 45; + public static final int CHECK_CLIENT_CONFIG = 46; + public static final int PUT_KV_CONFIG = 100; public static final int GET_KV_CONFIG = 101; @@ -162,4 +164,6 @@ public class RequestCode { public static final int SEND_BATCH_MESSAGE = 320; + + public static final int QUERY_CONSUME_QUEUE = 321; } diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/ResponseCode.java b/common/src/main/java/org/apache/rocketmq/common/protocol/ResponseCode.java index 90b182b1..f62c4ea6 100644 --- a/common/src/main/java/org/apache/rocketmq/common/protocol/ResponseCode.java +++ b/common/src/main/java/org/apache/rocketmq/common/protocol/ResponseCode.java @@ -53,6 +53,10 @@ public class ResponseCode extends RemotingSysResponseCode { public static final int SUBSCRIPTION_GROUP_NOT_EXIST = 26; + public static final int FILTER_DATA_NOT_EXIST = 27; + + public static final int FILTER_DATA_NOT_LATEST = 28; + public static final int TRANSACTION_SHOULD_COMMIT = 200; public static final int TRANSACTION_SHOULD_ROLLBACK = 201; diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBody.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBody.java new file mode 100644 index 00000000..a78ce554 --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBody.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.common.protocol.body; + +import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; + +public class CheckClientRequestBody extends RemotingSerializable { + + private String clientId; + private String group; + private SubscriptionData subscriptionData; + + public String getClientId() { + return clientId; + } + + public void setClientId(String clientId) { + this.clientId = clientId; + } + + public String getGroup() { + return group; + } + + public void setGroup(String group) { + this.group = group; + } + + public SubscriptionData getSubscriptionData() { + return subscriptionData; + } + + public void setSubscriptionData(SubscriptionData subscriptionData) { + this.subscriptionData = subscriptionData; + } +} diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumeQueueData.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumeQueueData.java new file mode 100644 index 00000000..7268dcda --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumeQueueData.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.common.protocol.body; + +public class ConsumeQueueData { + + private long physicOffset; + private int physicSize; + private long tagsCode; + private String extendDataJson; + private String bitMap; + private boolean eval; + private String msg; + + public long getPhysicOffset() { + return physicOffset; + } + + public void setPhysicOffset(long physicOffset) { + this.physicOffset = physicOffset; + } + + public int getPhysicSize() { + return physicSize; + } + + public void setPhysicSize(int physicSize) { + this.physicSize = physicSize; + } + + public long getTagsCode() { + return tagsCode; + } + + public void setTagsCode(long tagsCode) { + this.tagsCode = tagsCode; + } + + public String getExtendDataJson() { + return extendDataJson; + } + + public void setExtendDataJson(String extendDataJson) { + this.extendDataJson = extendDataJson; + } + + public String getBitMap() { + return bitMap; + } + + public void setBitMap(String bitMap) { + this.bitMap = bitMap; + } + + public boolean isEval() { + return eval; + } + + public void setEval(boolean eval) { + this.eval = eval; + } + + public String getMsg() { + return msg; + } + + public void setMsg(String msg) { + this.msg = msg; + } + + @Override + public String toString() { + return "ConsumeQueueData{" + + "physicOffset=" + physicOffset + + ", physicSize=" + physicSize + + ", tagsCode=" + tagsCode + + ", extendDataJson='" + extendDataJson + '\'' + + ", bitMap='" + bitMap + '\'' + + ", eval=" + eval + + ", msg='" + msg + '\'' + + '}'; + } +} diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/QueryConsumeQueueResponseBody.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/QueryConsumeQueueResponseBody.java new file mode 100644 index 00000000..be93da99 --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/QueryConsumeQueueResponseBody.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.common.protocol.body; + +import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; + +import java.util.List; + +public class QueryConsumeQueueResponseBody extends RemotingSerializable { + + private SubscriptionData subscriptionData; + private String filterData; + private List queueData; + private long maxQueueIndex; + private long minQueueIndex; + + public SubscriptionData getSubscriptionData() { + return subscriptionData; + } + + public void setSubscriptionData(SubscriptionData subscriptionData) { + this.subscriptionData = subscriptionData; + } + + public String getFilterData() { + return filterData; + } + + public void setFilterData(String filterData) { + this.filterData = filterData; + } + + public List getQueueData() { + return queueData; + } + + public void setQueueData(List queueData) { + this.queueData = queueData; + } + + public long getMaxQueueIndex() { + return maxQueueIndex; + } + + public void setMaxQueueIndex(long maxQueueIndex) { + this.maxQueueIndex = maxQueueIndex; + } + + public long getMinQueueIndex() { + return minQueueIndex; + } + + public void setMinQueueIndex(long minQueueIndex) { + this.minQueueIndex = minQueueIndex; + } +} diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageRequestHeader.java index 8a592132..106e89e5 100644 --- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageRequestHeader.java +++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageRequestHeader.java @@ -46,6 +46,7 @@ public class PullMessageRequestHeader implements CommandCustomHeader { private String subscription; @CFNotNull private Long subVersion; + private String expressionType; @Override public void checkFields() throws RemotingCommandException { @@ -130,4 +131,12 @@ public class PullMessageRequestHeader implements CommandCustomHeader { public void setSubVersion(Long subVersion) { this.subVersion = subVersion; } + + public String getExpressionType() { + return expressionType; + } + + public void setExpressionType(String expressionType) { + this.expressionType = expressionType; + } } diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/QueryConsumeQueueRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/QueryConsumeQueueRequestHeader.java new file mode 100644 index 00000000..642fe17c --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/QueryConsumeQueueRequestHeader.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.common.protocol.header; + +import org.apache.rocketmq.remoting.CommandCustomHeader; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; + +public class QueryConsumeQueueRequestHeader implements CommandCustomHeader { + + private String topic; + private int queueId; + private long index; + private int count; + private String consumerGroup; + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public int getQueueId() { + return queueId; + } + + public void setQueueId(int queueId) { + this.queueId = queueId; + } + + public long getIndex() { + return index; + } + + public void setIndex(long index) { + this.index = index; + } + + public int getCount() { + return count; + } + + public void setCount(int count) { + this.count = count; + } + + public String getConsumerGroup() { + return consumerGroup; + } + + public void setConsumerGroup(String consumerGroup) { + this.consumerGroup = consumerGroup; + } + + @Override + public void checkFields() throws RemotingCommandException { + + } +} diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/heartbeat/SubscriptionData.java b/common/src/main/java/org/apache/rocketmq/common/protocol/heartbeat/SubscriptionData.java index 81f59544..e456b7e6 100644 --- a/common/src/main/java/org/apache/rocketmq/common/protocol/heartbeat/SubscriptionData.java +++ b/common/src/main/java/org/apache/rocketmq/common/protocol/heartbeat/SubscriptionData.java @@ -32,6 +32,7 @@ public class SubscriptionData implements Comparable { private Set tagsSet = new HashSet(); private Set codeSet = new HashSet(); private long subVersion = System.currentTimeMillis(); + private String expressionType; @JSONField(serialize = false) private String filterClassSource; @@ -102,6 +103,14 @@ public class SubscriptionData implements Comparable { this.classFilterMode = classFilterMode; } + public String getExpressionType() { + return expressionType; + } + + public void setExpressionType(String expressionType) { + this.expressionType = expressionType; + } + @Override public int hashCode() { final int prime = 31; @@ -111,6 +120,7 @@ public class SubscriptionData implements Comparable { result = prime * result + ((subString == null) ? 0 : subString.hashCode()); result = prime * result + ((tagsSet == null) ? 0 : tagsSet.hashCode()); result = prime * result + ((topic == null) ? 0 : topic.hashCode()); + result = prime * result + ((expressionType == null) ? 0 : expressionType.hashCode()); return result; } @@ -147,6 +157,11 @@ public class SubscriptionData implements Comparable { return false; } else if (!topic.equals(other.topic)) return false; + if (expressionType == null) { + if (other.expressionType != null) + return false; + } else if (!expressionType.equals(other.expressionType)) + return false; return true; } @@ -154,7 +169,7 @@ public class SubscriptionData implements Comparable { public String toString() { return "SubscriptionData [classFilterMode=" + classFilterMode + ", topic=" + topic + ", subString=" + subString + ", tagsSet=" + tagsSet + ", codeSet=" + codeSet + ", subVersion=" + subVersion - + "]"; + + ", expressionType=" + expressionType + "]"; } @Override diff --git a/common/src/test/java/org/apache/rocketmq/common/filter/FilterAPITest.java b/common/src/test/java/org/apache/rocketmq/common/filter/FilterAPITest.java index 5137f328..c5f8460e 100644 --- a/common/src/test/java/org/apache/rocketmq/common/filter/FilterAPITest.java +++ b/common/src/test/java/org/apache/rocketmq/common/filter/FilterAPITest.java @@ -42,4 +42,53 @@ public class FilterAPITest { } assertThat(subscriptionData.getTagsSet()).isEqualTo(tagSet); } + + @Test + public void testBuildTagSome() { + try { + SubscriptionData subscriptionData = FilterAPI.build( + "TOPIC", "A || B", ExpressionType.TAG + ); + + assertThat(subscriptionData).isNotNull(); + assertThat(subscriptionData.getTopic()).isEqualTo("TOPIC"); + assertThat(subscriptionData.getSubString()).isEqualTo("A || B"); + assertThat(ExpressionType.isTagType(subscriptionData.getExpressionType())).isTrue(); + + assertThat(subscriptionData.getTagsSet()).isNotNull(); + assertThat(subscriptionData.getTagsSet()).containsExactly("A", "B"); + } catch (Exception e) { + e.printStackTrace(); + assertThat(Boolean.FALSE).isTrue(); + } + } + + @Test + public void testBuildSQL() { + try { + SubscriptionData subscriptionData = FilterAPI.build( + "TOPIC", "a is not null", ExpressionType.SQL92 + ); + + assertThat(subscriptionData).isNotNull(); + assertThat(subscriptionData.getTopic()).isEqualTo("TOPIC"); + assertThat(subscriptionData.getExpressionType()).isEqualTo(ExpressionType.SQL92); + } catch (Exception e) { + e.printStackTrace(); + assertThat(Boolean.FALSE).isTrue(); + } + } + + @Test + public void testBuildSQLWithNullSubString() { + try { + FilterAPI.build( + "TOPIC", null, ExpressionType.SQL92 + ); + + assertThat(Boolean.FALSE).isTrue(); + } catch (Exception e) { + e.printStackTrace(); + } + } } diff --git a/common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java b/common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java new file mode 100644 index 00000000..d14d6b06 --- /dev/null +++ b/common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.common.message; + +import org.junit.Test; + +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.UnknownHostException; +import java.nio.ByteBuffer; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +public class MessageDecoderTest { + + @Test + public void testDecodeProperties() { + MessageExt messageExt = new MessageExt(); + + messageExt.setMsgId("645100FA00002A9F000000489A3AA09E"); + messageExt.setTopic("abc"); + messageExt.setBody("hello!q!".getBytes()); + try { + messageExt.setBornHost(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 0)); + } catch (UnknownHostException e) { + e.printStackTrace(); + assertThat(Boolean.FALSE).isTrue(); + } + messageExt.setBornTimestamp(System.currentTimeMillis()); + messageExt.setCommitLogOffset(123456); + messageExt.setPreparedTransactionOffset(0); + messageExt.setQueueId(0); + messageExt.setQueueOffset(123); + messageExt.setReconsumeTimes(0); + try { + messageExt.setStoreHost(new InetSocketAddress(InetAddress.getLocalHost(), 0)); + } catch (UnknownHostException e) { + e.printStackTrace(); + assertThat(Boolean.FALSE).isTrue(); + } + + messageExt.putUserProperty("a", "123"); + messageExt.putUserProperty("b", "hello"); + messageExt.putUserProperty("c", "3.14"); + + byte[] msgBytes = new byte[0]; + try { + msgBytes = MessageDecoder.encode(messageExt, false); + } catch (Exception e) { + e.printStackTrace(); + assertThat(Boolean.FALSE).isTrue(); + } + + ByteBuffer byteBuffer = ByteBuffer.allocate(msgBytes.length); + byteBuffer.put(msgBytes); + + Map properties = MessageDecoder.decodeProperties(byteBuffer); + + assertThat(properties).isNotNull(); + assertThat("123").isEqualTo(properties.get("a")); + assertThat("hello").isEqualTo(properties.get("b")); + assertThat("3.14").isEqualTo(properties.get("c")); + } +} diff --git a/distribution/conf/logback_broker.xml b/distribution/conf/logback_broker.xml index 05c0ee4f..dd5c63f5 100644 --- a/distribution/conf/logback_broker.xml +++ b/distribution/conf/logback_broker.xml @@ -222,6 +222,29 @@ + + ${user.home}/logs/rocketmqlogs/filter.log + true + + ${user.home}/logs/rocketmqlogs/otherdays/filter.%i.log + + 1 + 10 + + + 100MB + + + %d{yyy-MM-dd HH:mm:ss,GMT+8} %p %t - %m%n + UTF-8 + + + + + + ${user.home}/logs/rocketmqlogs/stats.log @@ -321,6 +344,11 @@ + + + + + diff --git a/distribution/release.xml b/distribution/release.xml index 2d3ec1ec..9e4ef2a0 100644 --- a/distribution/release.xml +++ b/distribution/release.xml @@ -67,6 +67,7 @@ org.apache.rocketmq:rocketmq-namesrv org.apache.rocketmq:rocketmq-filtersrv org.apache.rocketmq:rocketmq-example + org.apache.rocketmq:rocketmq-filter lib/ diff --git a/example/src/main/java/org/apache/rocketmq/example/benchmark/Consumer.java b/example/src/main/java/org/apache/rocketmq/example/benchmark/Consumer.java index 473e4c77..3e1b79b0 100644 --- a/example/src/main/java/org/apache/rocketmq/example/benchmark/Consumer.java +++ b/example/src/main/java/org/apache/rocketmq/example/benchmark/Consumer.java @@ -27,10 +27,13 @@ import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.MessageSelector; import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext; import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.filter.ExpressionType; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.srvutil.ServerUtil; @@ -46,12 +49,14 @@ public class Consumer { final String topic = commandLine.hasOption('t') ? commandLine.getOptionValue('t').trim() : "BenchmarkTest"; final String groupPrefix = commandLine.hasOption('g') ? commandLine.getOptionValue('g').trim() : "benchmark_consumer"; final String isPrefixEnable = commandLine.hasOption('p') ? commandLine.getOptionValue('p').trim() : "true"; + final String filterType = commandLine.hasOption('f') ? commandLine.getOptionValue('f').trim() : null; + final String expression = commandLine.hasOption('e') ? commandLine.getOptionValue('e').trim() : null; String group = groupPrefix; if (Boolean.parseBoolean(isPrefixEnable)) { group = groupPrefix + "_" + Long.toString(System.currentTimeMillis() % 100); } - System.out.printf("topic %s group %s prefix %s%n", topic, group, isPrefixEnable); + System.out.printf("topic: %s, group: %s, prefix: %s, filterType: %s, expression: %s%n", topic, group, isPrefixEnable, filterType, expression); final StatsBenchmarkConsumer statsBenchmarkConsumer = new StatsBenchmarkConsumer(); @@ -99,7 +104,21 @@ public class Consumer { DefaultMQPushConsumer consumer = new DefaultMQPushConsumer(group); consumer.setInstanceName(Long.toString(System.currentTimeMillis())); - consumer.subscribe(topic, "*"); + if (filterType == null || expression == null) { + consumer.subscribe(topic, "*"); + } else { + if (ExpressionType.TAG.equals(filterType)) { + String expr = MixAll.file2String(expression); + System.out.printf("Expression: %s%n", expr); + consumer.subscribe(topic, MessageSelector.byTag(expr)); + } else if (ExpressionType.SQL92.equals(filterType)) { + String expr = MixAll.file2String(expression); + System.out.printf("Expression: %s%n", expr); + consumer.subscribe(topic, MessageSelector.bySql(expr)); + } else { + throw new IllegalArgumentException("Not support filter type! " + filterType); + } + } consumer.registerMessageListener(new MessageListenerConcurrently() { @Override @@ -142,6 +161,14 @@ public class Consumer { opt.setRequired(false); options.addOption(opt); + opt = new Option("f", "filterType", true, "TAG, SQL92"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("e", "expression", true, "filter expression content file path.ie: ./test/expr"); + opt.setRequired(false); + options.addOption(opt); + return options; } diff --git a/example/src/main/java/org/apache/rocketmq/example/benchmark/Producer.java b/example/src/main/java/org/apache/rocketmq/example/benchmark/Producer.java index 50d750df..2d8d0f6f 100644 --- a/example/src/main/java/org/apache/rocketmq/example/benchmark/Producer.java +++ b/example/src/main/java/org/apache/rocketmq/example/benchmark/Producer.java @@ -18,11 +18,13 @@ package org.apache.rocketmq.example.benchmark; import java.io.UnsupportedEncodingException; import java.util.LinkedList; +import java.util.Random; import java.util.Timer; import java.util.TimerTask; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicLong; + import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; @@ -50,13 +52,12 @@ public class Producer { final int threadCount = commandLine.hasOption('w') ? Integer.parseInt(commandLine.getOptionValue('w')) : 64; final int messageSize = commandLine.hasOption('s') ? Integer.parseInt(commandLine.getOptionValue('s')) : 128; final boolean keyEnable = commandLine.hasOption('k') && Boolean.parseBoolean(commandLine.getOptionValue('k')); + final int propertySize = commandLine.hasOption('p') ? Integer.parseInt(commandLine.getOptionValue('p')) : 0; System.out.printf("topic %s threadCount %d messageSize %d keyEnable %s%n", topic, threadCount, messageSize, keyEnable); final Logger log = ClientLogger.getLog(); - final Message msg = buildMessage(messageSize, topic); - final ExecutorService sendThreadPool = Executors.newFixedThreadPool(threadCount); final StatsBenchmarkProducer statsBenchmark = new StatsBenchmarkProducer(); @@ -117,10 +118,37 @@ public class Producer { public void run() { while (true) { try { + final Message msg; + try { + msg = buildMessage(messageSize, topic); + } catch (UnsupportedEncodingException e) { + e.printStackTrace(); + return; + } final long beginTimestamp = System.currentTimeMillis(); if (keyEnable) { msg.setKeys(String.valueOf(beginTimestamp / 1000)); } + if (propertySize > 0) { + if (msg.getProperties() != null) { + msg.getProperties().clear(); + } + int i = 0; + int startValue = (new Random(System.currentTimeMillis())).nextInt(100); + int size = 0; + while (true) { + String prop1 = "prop" + i, prop1V = "hello" + startValue; + String prop2 = "prop" + (i + 1), prop2V = String.valueOf(startValue); + msg.putUserProperty(prop1, prop1V); + msg.putUserProperty(prop2, prop2V); + size += prop1.length() + prop2.length() + prop1V.length() + prop2V.length(); + if (size > propertySize) { + break; + } + i += 2; + startValue += 2; + } + } producer.send(msg); statsBenchmark.getSendRequestSuccessCount().incrementAndGet(); statsBenchmark.getReceiveResponseSuccessCount().incrementAndGet(); @@ -214,7 +242,7 @@ class StatsBenchmarkProducer { private final AtomicLong sendMessageMaxRT = new AtomicLong(0L); public Long[] createSnapshot() { - Long[] snap = new Long[] { + Long[] snap = new Long[]{ System.currentTimeMillis(), this.sendRequestSuccessCount.get(), this.sendRequestFailedCount.get(), diff --git a/example/src/main/java/org/apache/rocketmq/example/filter/SqlConsumer.java b/example/src/main/java/org/apache/rocketmq/example/filter/SqlConsumer.java new file mode 100644 index 00000000..9a3b8131 --- /dev/null +++ b/example/src/main/java/org/apache/rocketmq/example/filter/SqlConsumer.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.example.filter; + +import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.MessageSelector; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.common.message.MessageExt; + +import java.util.List; + +public class SqlConsumer { + + public static void main(String[] args) { + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("please_rename_unique_group_name_4"); + + try { + consumer.subscribe("TopicTest", + MessageSelector.bySql("(TAGS is not null and TAGS in ('TagA', 'TagB'))" + + "and (a is not null and a between 0 3)")); + } catch (MQClientException e) { + e.printStackTrace(); + return; + } + + consumer.registerMessageListener(new MessageListenerConcurrently() { + + @Override + public ConsumeConcurrentlyStatus consumeMessage(List msgs, + ConsumeConcurrentlyContext context) { + System.out.printf(Thread.currentThread().getName() + " Receive New Messages: " + msgs + "%n"); + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } + }); + + try { + consumer.start(); + } catch (MQClientException e) { + e.printStackTrace(); + return; + } + System.out.printf("Consumer Started.%n"); + } +} diff --git a/example/src/main/java/org/apache/rocketmq/example/filter/SqlProducer.java b/example/src/main/java/org/apache/rocketmq/example/filter/SqlProducer.java new file mode 100644 index 00000000..3f3a0e65 --- /dev/null +++ b/example/src/main/java/org/apache/rocketmq/example/filter/SqlProducer.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.example.filter; + +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.client.producer.SendResult; +import org.apache.rocketmq.common.message.Message; +import org.apache.rocketmq.remoting.common.RemotingHelper; + +public class SqlProducer { + + public static void main(String[] args) { + DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); + try { + producer.start(); + } catch (MQClientException e) { + e.printStackTrace(); + return; + } + + for (int i = 0; i < 10; i++) { + try { + String tag; + int div = i % 3; + if (div == 0) { + tag = "TagA"; + } else if (div == 1) { + tag = "TagB"; + } else { + tag = "TagC"; + } + Message msg = new Message("TopicTest", + tag, + ("Hello RocketMQ " + i).getBytes(RemotingHelper.DEFAULT_CHARSET) + ); + msg.putUserProperty("a", String.valueOf(i)); + + SendResult sendResult = producer.send(msg); + System.out.printf("%s%n", sendResult); + } catch (Exception e) { + e.printStackTrace(); + try { + Thread.sleep(1000); + } catch (InterruptedException e1) { + e1.printStackTrace(); + } + } + } + producer.shutdown(); + } +} diff --git a/filter/pom.xml b/filter/pom.xml new file mode 100644 index 00000000..7978f055 --- /dev/null +++ b/filter/pom.xml @@ -0,0 +1,43 @@ + + + + + + rocketmq-all + org.apache.rocketmq + 4.1.0-incubating-SNAPSHOT + + + 4.0.0 + jar + rocketmq-filter + rocketmq-filter ${project.version} + + + + ${project.groupId} + rocketmq-common + + + ${project.groupId} + rocketmq-srvutil + + + \ No newline at end of file diff --git a/filter/src/main/java/org/apache/rocketmq/filter/FilterFactory.java b/filter/src/main/java/org/apache/rocketmq/filter/FilterFactory.java new file mode 100644 index 00000000..a3185484 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/FilterFactory.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter; + +import java.util.HashMap; +import java.util.Map; + +/** + * Filter factory: support other filter to register. + */ +public class FilterFactory { + + public static final FilterFactory INSTANCE = new FilterFactory(); + + protected static final Map FILTER_SPI_HOLDER = new HashMap(4); + + static { + FilterFactory.INSTANCE.register(new SqlFilter()); + } + + /** + * Register a filter. + *
    + * Note: + *
  • 1. Filter registered will be used in broker server, so take care of it's reliability and performance.
  • + * + * @param filterSpi + */ + public void register(FilterSpi filterSpi) { + if (FILTER_SPI_HOLDER.containsKey(filterSpi.ofType())) { + throw new IllegalArgumentException(String.format("Filter spi type(%s) already exist!", filterSpi.ofType())); + } + + FILTER_SPI_HOLDER.put(filterSpi.ofType(), filterSpi); + } + + /** + * Un register a filter. + * + * @param type + * @return + */ + public FilterSpi unRegister(String type) { + return FILTER_SPI_HOLDER.remove(type); + } + + /** + * Get a filter registered, null if none exist. + * + * @param type + * @return + */ + public FilterSpi get(String type) { + return FILTER_SPI_HOLDER.get(type); + } + +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/FilterSpi.java b/filter/src/main/java/org/apache/rocketmq/filter/FilterSpi.java new file mode 100644 index 00000000..fcc39fac --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/FilterSpi.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter; + +import org.apache.rocketmq.filter.expression.Expression; +import org.apache.rocketmq.filter.expression.MQFilterException; + +/** + * Filter spi interface. + */ +public interface FilterSpi { + + /** + * Compile. + * + * @param expr + * @return + * @throws org.apache.rocketmq.filter.expression.MQFilterException + */ + Expression compile(final String expr) throws MQFilterException; + + /** + * Which type. + * + * @return + */ + String ofType(); +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/SqlFilter.java b/filter/src/main/java/org/apache/rocketmq/filter/SqlFilter.java new file mode 100644 index 00000000..0c1ffb8d --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/SqlFilter.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter; + +import org.apache.rocketmq.common.filter.ExpressionType; +import org.apache.rocketmq.filter.expression.Expression; +import org.apache.rocketmq.filter.expression.MQFilterException; +import org.apache.rocketmq.filter.parser.SelectorParser; + +/** + * SQL92 Filter, just a wrapper of {@link org.apache.rocketmq.filter.parser.SelectorParser}. + *

    + *

    + * Do not use this filter directly.Use {@link FilterFactory#get} to select a filter. + *

    + */ +public class SqlFilter implements FilterSpi { + + @Override + public Expression compile(final String expr) throws MQFilterException { + return SelectorParser.parse(expr); + } + + @Override + public String ofType() { + return ExpressionType.SQL92; + } +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/constant/UnaryType.java b/filter/src/main/java/org/apache/rocketmq/filter/constant/UnaryType.java new file mode 100644 index 00000000..d2d04cdb --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/constant/UnaryType.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter.constant; + +public enum UnaryType { + NEGATE, + IN, + NOT, + BOOLEANCAST, + LIKE +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/BinaryExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/BinaryExpression.java new file mode 100644 index 00000000..0f172e30 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/BinaryExpression.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter.expression; + +/** + * An expression which performs an operation on two expression values. + *

    + * This class was taken from ActiveMQ org.apache.activemq.filter.BinaryExpression, + *

    + */ +public abstract class BinaryExpression implements Expression { + protected Expression left; + protected Expression right; + + public BinaryExpression(Expression left, Expression right) { + this.left = left; + this.right = right; + } + + public Expression getLeft() { + return left; + } + + public Expression getRight() { + return right; + } + + /** + * @see Object#toString() + */ + public String toString() { + return "(" + left.toString() + " " + getExpressionSymbol() + " " + right.toString() + ")"; + } + + /** + * @see Object#hashCode() + */ + public int hashCode() { + return toString().hashCode(); + } + + /** + * @see Object#equals(Object) + */ + public boolean equals(Object o) { + + if (o == null || !this.getClass().equals(o.getClass())) { + return false; + } + return toString().equals(o.toString()); + + } + + /** + * Returns the symbol that represents this binary expression. For example, addition is + * represented by "+" + * + * @return + */ + public abstract String getExpressionSymbol(); + + /** + * @param expression + */ + public void setRight(Expression expression) { + right = expression; + } + + /** + * @param expression + */ + public void setLeft(Expression expression) { + left = expression; + } + +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/BooleanExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/BooleanExpression.java new file mode 100644 index 00000000..bb54632e --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/BooleanExpression.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter.expression; + +/** + * A BooleanExpression is an expression that always + * produces a Boolean result. + *

    + * This class was taken from ActiveMQ org.apache.activemq.filter.BooleanExpression, + * but the parameter is changed to an interface. + *

    + * + * @see org.apache.rocketmq.filter.expression.EvaluationContext + */ +public interface BooleanExpression extends Expression { + + /** + * @param context + * @return true if the expression evaluates to Boolean.TRUE. + * @throws Exception + */ + boolean matches(EvaluationContext context) throws Exception; + +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/ComparisonExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/ComparisonExpression.java new file mode 100644 index 00000000..8b82e573 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/ComparisonExpression.java @@ -0,0 +1,413 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter.expression; + +import java.util.List; + +/** + * A filter performing a comparison of two objects + *

    + * This class was taken from ActiveMQ org.apache.activemq.filter.ComparisonExpression, + * but: + * 1. Remove LIKE expression, and related methods; + * 2. Extract a new method __compare which has int return value; + * 3. When create between expression, check whether left value is less or equal than right value; + * 4. For string type value(can not convert to number), only equal or unequal comparison are supported. + *

    + */ +public abstract class ComparisonExpression extends BinaryExpression implements BooleanExpression { + + public static final ThreadLocal CONVERT_STRING_EXPRESSIONS = new ThreadLocal(); + + boolean convertStringExpressions = false; + + /** + * @param left + * @param right + */ + public ComparisonExpression(Expression left, Expression right) { + super(left, right); + convertStringExpressions = CONVERT_STRING_EXPRESSIONS.get() != null; + } + + public static BooleanExpression createBetween(Expression value, Expression left, Expression right) { + // check + if (left instanceof ConstantExpression && right instanceof ConstantExpression) { + Object lv = ((ConstantExpression) left).getValue(); + Object rv = ((ConstantExpression) right).getValue(); + if (lv == null || rv == null) { + throw new RuntimeException("Illegal values of between, values can not be null!"); + } + if (lv instanceof Comparable && rv instanceof Comparable) { + int ret = __compare((Comparable) rv, (Comparable) lv, true); + if (ret < 0) + throw new RuntimeException( + String.format("Illegal values of between, left value(%s) must less than or equal to right value(%s)", lv, rv) + ); + } + } + + return LogicExpression.createAND(createGreaterThanEqual(value, left), createLessThanEqual(value, right)); + } + + public static BooleanExpression createNotBetween(Expression value, Expression left, Expression right) { + return LogicExpression.createOR(createLessThan(value, left), createGreaterThan(value, right)); + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + public static BooleanExpression createInFilter(Expression left, List elements) { + + if (!(left instanceof PropertyExpression)) { + throw new RuntimeException("Expected a property for In expression, got: " + left); + } + return UnaryExpression.createInExpression((PropertyExpression) left, elements, false); + + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + public static BooleanExpression createNotInFilter(Expression left, List elements) { + + if (!(left instanceof PropertyExpression)) { + throw new RuntimeException("Expected a property for In expression, got: " + left); + } + return UnaryExpression.createInExpression((PropertyExpression) left, elements, true); + + } + + public static BooleanExpression createIsNull(Expression left) { + return doCreateEqual(left, ConstantExpression.NULL); + } + + public static BooleanExpression createIsNotNull(Expression left) { + return UnaryExpression.createNOT(doCreateEqual(left, ConstantExpression.NULL)); + } + + public static BooleanExpression createNotEqual(Expression left, Expression right) { + return UnaryExpression.createNOT(createEqual(left, right)); + } + + public static BooleanExpression createEqual(Expression left, Expression right) { + checkEqualOperand(left); + checkEqualOperand(right); + checkEqualOperandCompatability(left, right); + return doCreateEqual(left, right); + } + + @SuppressWarnings({"rawtypes"}) + private static BooleanExpression doCreateEqual(Expression left, Expression right) { + return new ComparisonExpression(left, right) { + + public Object evaluate(EvaluationContext context) throws Exception { + Object lv = left.evaluate(context); + Object rv = right.evaluate(context); + + // If one of the values is null + if (lv == null ^ rv == null) { + if (lv == null) { + return null; + } + return Boolean.FALSE; + } + if (lv == rv || lv.equals(rv)) { + return Boolean.TRUE; + } + if (lv instanceof Comparable && rv instanceof Comparable) { + return compare((Comparable) lv, (Comparable) rv); + } + return Boolean.FALSE; + } + + protected boolean asBoolean(int answer) { + return answer == 0; + } + + public String getExpressionSymbol() { + return "=="; + } + }; + } + + public static BooleanExpression createGreaterThan(final Expression left, final Expression right) { + checkLessThanOperand(left); + checkLessThanOperand(right); + return new ComparisonExpression(left, right) { + protected boolean asBoolean(int answer) { + return answer > 0; + } + + public String getExpressionSymbol() { + return ">"; + } + }; + } + + public static BooleanExpression createGreaterThanEqual(final Expression left, final Expression right) { + checkLessThanOperand(left); + checkLessThanOperand(right); + return new ComparisonExpression(left, right) { + protected boolean asBoolean(int answer) { + return answer >= 0; + } + + public String getExpressionSymbol() { + return ">="; + } + }; + } + + public static BooleanExpression createLessThan(final Expression left, final Expression right) { + checkLessThanOperand(left); + checkLessThanOperand(right); + return new ComparisonExpression(left, right) { + + protected boolean asBoolean(int answer) { + return answer < 0; + } + + public String getExpressionSymbol() { + return "<"; + } + + }; + } + + public static BooleanExpression createLessThanEqual(final Expression left, final Expression right) { + checkLessThanOperand(left); + checkLessThanOperand(right); + return new ComparisonExpression(left, right) { + + protected boolean asBoolean(int answer) { + return answer <= 0; + } + + public String getExpressionSymbol() { + return "<="; + } + }; + } + + /** + * Only Numeric expressions can be used in >, >=, < or <= expressions.s + * + * @param expr + */ + public static void checkLessThanOperand(Expression expr) { + if (expr instanceof ConstantExpression) { + Object value = ((ConstantExpression) expr).getValue(); + if (value instanceof Number) { + return; + } + + // Else it's boolean or a String.. + throw new RuntimeException("Value '" + expr + "' cannot be compared."); + } + if (expr instanceof BooleanExpression) { + throw new RuntimeException("Value '" + expr + "' cannot be compared."); + } + } + + /** + * Validates that the expression can be used in == or <> expression. Cannot + * not be NULL TRUE or FALSE litterals. + * + * @param expr + */ + public static void checkEqualOperand(Expression expr) { + if (expr instanceof ConstantExpression) { + Object value = ((ConstantExpression) expr).getValue(); + if (value == null) { + throw new RuntimeException("'" + expr + "' cannot be compared."); + } + } + } + + /** + * @param left + * @param right + */ + private static void checkEqualOperandCompatability(Expression left, Expression right) { + if (left instanceof ConstantExpression && right instanceof ConstantExpression) { + if (left instanceof BooleanExpression && !(right instanceof BooleanExpression)) { + throw new RuntimeException("'" + left + "' cannot be compared with '" + right + "'"); + } + } + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + public Object evaluate(EvaluationContext context) throws Exception { + Comparable lv = (Comparable) left.evaluate(context); + if (lv == null) { + return null; + } + Comparable rv = (Comparable) right.evaluate(context); + if (rv == null) { + return null; + } + if (getExpressionSymbol().equals(">=") || getExpressionSymbol().equals(">") + || getExpressionSymbol().equals("<") || getExpressionSymbol().equals("<=")) { + Class lc = lv.getClass(); + Class rc = rv.getClass(); + if (lc == rc && lc == String.class) { + // Compare String is illegal + // first try to convert to double + try { + Comparable lvC = Double.valueOf((String) (Comparable) lv); + Comparable rvC = Double.valueOf((String) rv); + + return compare(lvC, rvC); + } catch (Exception e) { + throw new RuntimeException("It's illegal to compare string by '>=', '>', '<', '<='. lv=" + lv + ", rv=" + rv, e); + } + } + } + return compare(lv, rv); + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + protected static int __compare(Comparable lv, Comparable rv, boolean convertStringExpressions) { + Class lc = lv.getClass(); + Class rc = rv.getClass(); + // If the the objects are not of the same type, + // try to convert up to allow the comparison. + if (lc != rc) { + try { + if (lc == Boolean.class) { + if (convertStringExpressions && rc == String.class) { + lv = Boolean.valueOf((String) lv).booleanValue(); + } else { + return -1; + } + } else if (lc == Byte.class) { + if (rc == Short.class) { + lv = Short.valueOf(((Number) lv).shortValue()); + } else if (rc == Integer.class) { + lv = Integer.valueOf(((Number) lv).intValue()); + } else if (rc == Long.class) { + lv = Long.valueOf(((Number) lv).longValue()); + } else if (rc == Float.class) { + lv = new Float(((Number) lv).floatValue()); + } else if (rc == Double.class) { + lv = new Double(((Number) lv).doubleValue()); + } else if (convertStringExpressions && rc == String.class) { + rv = Byte.valueOf((String) rv); + } else { + return -1; + } + } else if (lc == Short.class) { + if (rc == Integer.class) { + lv = Integer.valueOf(((Number) lv).intValue()); + } else if (rc == Long.class) { + lv = Long.valueOf(((Number) lv).longValue()); + } else if (rc == Float.class) { + lv = new Float(((Number) lv).floatValue()); + } else if (rc == Double.class) { + lv = new Double(((Number) lv).doubleValue()); + } else if (convertStringExpressions && rc == String.class) { + rv = Short.valueOf((String) rv); + } else { + return -1; + } + } else if (lc == Integer.class) { + if (rc == Long.class) { + lv = Long.valueOf(((Number) lv).longValue()); + } else if (rc == Float.class) { + lv = new Float(((Number) lv).floatValue()); + } else if (rc == Double.class) { + lv = new Double(((Number) lv).doubleValue()); + } else if (convertStringExpressions && rc == String.class) { + rv = Integer.valueOf((String) rv); + } else { + return -1; + } + } else if (lc == Long.class) { + if (rc == Integer.class) { + rv = Long.valueOf(((Number) rv).longValue()); + } else if (rc == Float.class) { + lv = new Float(((Number) lv).floatValue()); + } else if (rc == Double.class) { + lv = new Double(((Number) lv).doubleValue()); + } else if (convertStringExpressions && rc == String.class) { + rv = Long.valueOf((String) rv); + } else { + return -1; + } + } else if (lc == Float.class) { + if (rc == Integer.class) { + rv = new Float(((Number) rv).floatValue()); + } else if (rc == Long.class) { + rv = new Float(((Number) rv).floatValue()); + } else if (rc == Double.class) { + lv = new Double(((Number) lv).doubleValue()); + } else if (convertStringExpressions && rc == String.class) { + rv = Float.valueOf((String) rv); + } else { + return -1; + } + } else if (lc == Double.class) { + if (rc == Integer.class) { + rv = new Double(((Number) rv).doubleValue()); + } else if (rc == Long.class) { + rv = new Double(((Number) rv).doubleValue()); + } else if (rc == Float.class) { + rv = new Float(((Number) rv).doubleValue()); + } else if (convertStringExpressions && rc == String.class) { + rv = Double.valueOf((String) rv); + } else { + return -1; + } + } else if (convertStringExpressions && lc == String.class) { + if (rc == Boolean.class) { + lv = Boolean.valueOf((String) lv); + } else if (rc == Byte.class) { + lv = Byte.valueOf((String) lv); + } else if (rc == Short.class) { + lv = Short.valueOf((String) lv); + } else if (rc == Integer.class) { + lv = Integer.valueOf((String) lv); + } else if (rc == Long.class) { + lv = Long.valueOf((String) lv); + } else if (rc == Float.class) { + lv = Float.valueOf((String) lv); + } else if (rc == Double.class) { + lv = Double.valueOf((String) lv); + } else { + return -1; + } + } else { + return -1; + } + } catch (NumberFormatException e) { + throw new RuntimeException(e); + } + } + return lv.compareTo(rv); + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + protected Boolean compare(Comparable lv, Comparable rv) { + return asBoolean(__compare(lv, rv, convertStringExpressions)) ? Boolean.TRUE : Boolean.FALSE; + } + + protected abstract boolean asBoolean(int answer); + + public boolean matches(EvaluationContext context) throws Exception { + Object object = evaluate(context); + return object != null && object == Boolean.TRUE; + } + +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/ConstantExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/ConstantExpression.java new file mode 100644 index 00000000..ca70f51a --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/ConstantExpression.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter.expression; + +/** + * Represents a constant expression + *

    + * This class was taken from ActiveMQ org.apache.activemq.filter.ConstantExpression, + * but: + * 1. For long type constant, the range bound by java Long type; + * 2. For float type constant, the range bound by java Double type; + * 3. Remove Hex and Octal expression; + * 4. Add now expression to support to get current time. + *

    + */ +public class ConstantExpression implements Expression { + + static class BooleanConstantExpression extends ConstantExpression implements BooleanExpression { + public BooleanConstantExpression(Object value) { + super(value); + } + + public boolean matches(EvaluationContext context) throws Exception { + Object object = evaluate(context); + return object != null && object == Boolean.TRUE; + } + } + + public static final BooleanConstantExpression NULL = new BooleanConstantExpression(null); + public static final BooleanConstantExpression TRUE = new BooleanConstantExpression(Boolean.TRUE); + public static final BooleanConstantExpression FALSE = new BooleanConstantExpression(Boolean.FALSE); + + private Object value; + + public ConstantExpression(Object value) { + this.value = value; + } + + public static ConstantExpression createFromDecimal(String text) { + + // Strip off the 'l' or 'L' if needed. + if (text.endsWith("l") || text.endsWith("L")) { + text = text.substring(0, text.length() - 1); + } + + // only support Long.MIN_VALUE ~ Long.MAX_VALUE + Number value = new Long(text); +// try { +// value = new Long(text); +// } catch (NumberFormatException e) { +// // The number may be too big to fit in a long. +// value = new BigDecimal(text); +// } + + long l = value.longValue(); + if (Integer.MIN_VALUE <= l && l <= Integer.MAX_VALUE) { + value = Integer.valueOf(value.intValue()); + } + return new ConstantExpression(value); + } + + public static ConstantExpression createFloat(String text) { + Double value = new Double(text); + if (value > Double.MAX_VALUE) { + throw new RuntimeException(text + " is greater than " + Double.MAX_VALUE); + } + if (value < Double.MIN_VALUE) { + throw new RuntimeException(text + " is less than " + Double.MIN_VALUE); + } + return new ConstantExpression(value); + } + + public static ConstantExpression createNow() { + return new NowExpression(); + } + + public Object evaluate(EvaluationContext context) throws Exception { + return value; + } + + public Object getValue() { + return value; + } + + /** + * @see Object#toString() + */ + public String toString() { + Object value = getValue(); + if (value == null) { + return "NULL"; + } + if (value instanceof Boolean) { + return ((Boolean) value).booleanValue() ? "TRUE" : "FALSE"; + } + if (value instanceof String) { + return encodeString((String) value); + } + return value.toString(); + } + + /** + * @see Object#hashCode() + */ + public int hashCode() { + return toString().hashCode(); + } + + /** + * @see Object#equals(Object) + */ + public boolean equals(Object o) { + + if (o == null || !this.getClass().equals(o.getClass())) { + return false; + } + return toString().equals(o.toString()); + + } + + /** + * Encodes the value of string so that it looks like it would look like when + * it was provided in a selector. + * + * @return + */ + public static String encodeString(String s) { + StringBuffer b = new StringBuffer(); + b.append('\''); + for (int i = 0; i < s.length(); i++) { + char c = s.charAt(i); + if (c == '\'') { + b.append(c); + } + b.append(c); + } + b.append('\''); + return b.toString(); + } + +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/EmptyEvaluationContext.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/EmptyEvaluationContext.java new file mode 100644 index 00000000..52af2d0a --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/EmptyEvaluationContext.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter.expression; + +import java.util.Map; + +/** + * Empty context. + */ +public class EmptyEvaluationContext implements EvaluationContext { + @Override + public Object get(String name) { + return null; + } + + @Override + public Map keyValues() { + return null; + } +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/EvaluationContext.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/EvaluationContext.java new file mode 100644 index 00000000..094ef53c --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/EvaluationContext.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter.expression; + +import java.util.Map; + +/** + * Context of evaluate expression. + * + * Compare to org.apache.activemq.filter.MessageEvaluationContext, this is just an interface. + */ +public interface EvaluationContext { + + /** + * Get value by name from context + * + * @param name + * @return + */ + Object get(String name); + + /** + * Context variables. + * + * @return + */ + Map keyValues(); +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/Expression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/Expression.java new file mode 100644 index 00000000..3e6d9b33 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/Expression.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter.expression; + +/** + * Interface of expression. + *

    + * This class was taken from ActiveMQ org.apache.activemq.filter.Expression, + * but the parameter is changed to an interface. + *

    + * + * @see org.apache.rocketmq.filter.expression.EvaluationContext + */ +public interface Expression { + + /** + * Calculate express result with context. + * + * @param context context of evaluation + * @return the value of this expression + */ + Object evaluate(EvaluationContext context) throws Exception; +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/LogicExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/LogicExpression.java new file mode 100644 index 00000000..1062bb8f --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/LogicExpression.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter.expression; + +/** + * A filter performing a comparison of two objects + *

    + * This class was taken from ActiveMQ org.apache.activemq.filter.LogicExpression, + *

    + */ +public abstract class LogicExpression extends BinaryExpression implements BooleanExpression { + + /** + * @param left + * @param right + */ + public LogicExpression(BooleanExpression left, BooleanExpression right) { + super(left, right); + } + + public static BooleanExpression createOR(BooleanExpression lvalue, BooleanExpression rvalue) { + return new LogicExpression(lvalue, rvalue) { + + public Object evaluate(EvaluationContext context) throws Exception { + + Boolean lv = (Boolean) left.evaluate(context); + if (lv != null && lv.booleanValue()) { + return Boolean.TRUE; + } + Boolean rv = (Boolean) right.evaluate(context); + if (rv != null && rv.booleanValue()) { + return Boolean.TRUE; + } + if (lv == null || rv == null) { + return null; + } + return Boolean.FALSE; + } + + public String getExpressionSymbol() { + return "||"; + } + }; + } + + public static BooleanExpression createAND(BooleanExpression lvalue, BooleanExpression rvalue) { + return new LogicExpression(lvalue, rvalue) { + + public Object evaluate(EvaluationContext context) throws Exception { + + Boolean lv = (Boolean) left.evaluate(context); + + if (lv != null && !lv.booleanValue()) { + return Boolean.FALSE; + } + Boolean rv = (Boolean) right.evaluate(context); + if (rv != null && !rv.booleanValue()) { + return Boolean.FALSE; + } + if (lv == null || rv == null) { + return null; + } + return Boolean.TRUE; + } + + public String getExpressionSymbol() { + return "&&"; + } + }; + } + + public abstract Object evaluate(EvaluationContext context) throws Exception; + + public boolean matches(EvaluationContext context) throws Exception { + Object object = evaluate(context); + return object != null && object == Boolean.TRUE; + } + +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/MQFilterException.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/MQFilterException.java new file mode 100644 index 00000000..676a17ba --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/MQFilterException.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter.expression; + +/** + * Exception. + */ +public class MQFilterException extends Exception { + private static final long serialVersionUID = 1L; + private final int responseCode; + private final String errorMessage; + + public MQFilterException(String errorMessage, Throwable cause) { + super(cause); + this.responseCode = -1; + this.errorMessage = errorMessage; + } + + public MQFilterException(int responseCode, String errorMessage) { + this.responseCode = responseCode; + this.errorMessage = errorMessage; + } + + public int getResponseCode() { + return responseCode; + } + + public String getErrorMessage() { + return errorMessage; + } +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/NowExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/NowExpression.java new file mode 100644 index 00000000..d76caca9 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/NowExpression.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter.expression; + +/** + * Current time expression.Just for test. + */ +public class NowExpression extends ConstantExpression { + public NowExpression() { + super("now"); + } + + @Override + public Object evaluate(EvaluationContext context) throws Exception { + return new Long(System.currentTimeMillis()); + } + + public Object getValue() { + return new Long(System.currentTimeMillis()); + } +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/PropertyExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/PropertyExpression.java new file mode 100644 index 00000000..b9657b04 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/PropertyExpression.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter.expression; + +/** + * Represents a property expression + *

    + * This class was taken from ActiveMQ org.apache.activemq.filter.PropertyExpression, + * but more simple and no transfer between expression and message property. + *

    + */ +public class PropertyExpression implements Expression { + private final String name; + + public PropertyExpression(String name) { + this.name = name; + } + + @Override + public Object evaluate(EvaluationContext context) throws Exception { + return context.get(name); + } + + public String getName() { + return name; + } + + /** + * @see Object#toString() + */ + @Override + public String toString() { + return name; + } + + /** + * @see Object#hashCode() + */ + @Override + public int hashCode() { + return name.hashCode(); + } + + /** + * @see Object#equals(Object) + */ + @Override + public boolean equals(Object o) { + + if (o == null || !this.getClass().equals(o.getClass())) { + return false; + } + return name.equals(((PropertyExpression) o).name); + } +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/UnaryExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/UnaryExpression.java new file mode 100644 index 00000000..0519f4d5 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/UnaryExpression.java @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter.expression; + +import org.apache.rocketmq.filter.constant.UnaryType; + +import java.math.BigDecimal; +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; + +/** + * An expression which performs an operation on two expression values + *

    + * This class was taken from ActiveMQ org.apache.activemq.filter.UnaryExpression, + * but: + * 1. remove XPath and XQuery expression; + * 2. Add constant UnaryType to distinguish different unary expression; + * 3. Extract UnaryInExpression to an independent class. + *

    + */ +public abstract class UnaryExpression implements Expression { + + private static final BigDecimal BD_LONG_MIN_VALUE = BigDecimal.valueOf(Long.MIN_VALUE); + protected Expression right; + + public UnaryType unaryType; + + public UnaryExpression(Expression left) { + this.right = left; + } + + public UnaryExpression(Expression left, UnaryType unaryType) { + this.setUnaryType(unaryType); + this.right = left; + } + + public static Expression createNegate(Expression left) { + return new UnaryExpression(left, UnaryType.NEGATE) { + public Object evaluate(EvaluationContext context) throws Exception { + Object rvalue = right.evaluate(context); + if (rvalue == null) { + return null; + } + if (rvalue instanceof Number) { + return negate((Number) rvalue); + } + return null; + } + + public String getExpressionSymbol() { + return "-"; + } + }; + } + + public static BooleanExpression createInExpression(PropertyExpression right, List elements, + final boolean not) { + + // Use a HashSet if there are many elements. + Collection t; + if (elements.size() == 0) { + t = null; + } else if (elements.size() < 5) { + t = elements; + } else { + t = new HashSet(elements); + } + final Collection inList = t; + + return new UnaryInExpression(right, UnaryType.IN, inList, not) { + public Object evaluate(EvaluationContext context) throws Exception { + + Object rvalue = right.evaluate(context); + if (rvalue == null) { + return null; + } + if (rvalue.getClass() != String.class) { + return null; + } + + if ((inList != null && inList.contains(rvalue)) ^ not) { + return Boolean.TRUE; + } else { + return Boolean.FALSE; + } + + } + + public String toString() { + StringBuffer answer = new StringBuffer(); + answer.append(right); + answer.append(" "); + answer.append(getExpressionSymbol()); + answer.append(" ( "); + + int count = 0; + for (Iterator i = inList.iterator(); i.hasNext(); ) { + Object o = (Object) i.next(); + if (count != 0) { + answer.append(", "); + } + answer.append(o); + count++; + } + + answer.append(" )"); + return answer.toString(); + } + + public String getExpressionSymbol() { + if (not) { + return "NOT IN"; + } else { + return "IN"; + } + } + }; + } + + abstract static class BooleanUnaryExpression extends UnaryExpression implements BooleanExpression { + public BooleanUnaryExpression(Expression left, UnaryType unaryType) { + super(left, unaryType); + } + + public boolean matches(EvaluationContext context) throws Exception { + Object object = evaluate(context); + return object != null && object == Boolean.TRUE; + } + } + + public static BooleanExpression createNOT(BooleanExpression left) { + return new BooleanUnaryExpression(left, UnaryType.NOT) { + public Object evaluate(EvaluationContext context) throws Exception { + Boolean lvalue = (Boolean) right.evaluate(context); + if (lvalue == null) { + return null; + } + return lvalue.booleanValue() ? Boolean.FALSE : Boolean.TRUE; + } + + public String getExpressionSymbol() { + return "NOT"; + } + }; + } + + public static BooleanExpression createBooleanCast(Expression left) { + return new BooleanUnaryExpression(left, UnaryType.BOOLEANCAST) { + public Object evaluate(EvaluationContext context) throws Exception { + Object rvalue = right.evaluate(context); + if (rvalue == null) { + return null; + } + if (!rvalue.getClass().equals(Boolean.class)) { + return Boolean.FALSE; + } + return ((Boolean) rvalue).booleanValue() ? Boolean.TRUE : Boolean.FALSE; + } + + public String toString() { + return right.toString(); + } + + public String getExpressionSymbol() { + return ""; + } + }; + } + + private static Number negate(Number left) { + Class clazz = left.getClass(); + if (clazz == Integer.class) { + return new Integer(-left.intValue()); + } else if (clazz == Long.class) { + return new Long(-left.longValue()); + } else if (clazz == Float.class) { + return new Float(-left.floatValue()); + } else if (clazz == Double.class) { + return new Double(-left.doubleValue()); + } else if (clazz == BigDecimal.class) { + // We ussually get a big deciamal when we have Long.MIN_VALUE + // constant in the + // Selector. Long.MIN_VALUE is too big to store in a Long as a + // positive so we store it + // as a Big decimal. But it gets Negated right away.. to here we try + // to covert it back + // to a Long. + BigDecimal bd = (BigDecimal) left; + bd = bd.negate(); + + if (BD_LONG_MIN_VALUE.compareTo(bd) == 0) { + return Long.valueOf(Long.MIN_VALUE); + } + return bd; + } else { + throw new RuntimeException("Don't know how to negate: " + left); + } + } + + public Expression getRight() { + return right; + } + + public void setRight(Expression expression) { + right = expression; + } + + public UnaryType getUnaryType() { + return unaryType; + } + + public void setUnaryType(UnaryType unaryType) { + this.unaryType = unaryType; + } + + /** + * @see Object#toString() + */ + public String toString() { + return "(" + 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(); + +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/expression/UnaryInExpression.java b/filter/src/main/java/org/apache/rocketmq/filter/expression/UnaryInExpression.java new file mode 100644 index 00000000..7d9083c0 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/expression/UnaryInExpression.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.filter.expression; + +import org.apache.rocketmq.filter.constant.UnaryType; + +import java.util.Collection; + +/** + * In expression. + */ +abstract public class UnaryInExpression extends UnaryExpression implements BooleanExpression { + + private boolean not; + + private Collection inList; + + public UnaryInExpression(Expression left, UnaryType unaryType, + Collection inList, boolean not) { + super(left, unaryType); + this.setInList(inList); + this.setNot(not); + + } + + public boolean matches(EvaluationContext context) throws Exception { + Object object = evaluate(context); + return object != null && object == Boolean.TRUE; + } + + public boolean isNot() { + return not; + } + + public void setNot(boolean not) { + this.not = not; + } + + public Collection getInList() { + return inList; + } + + public void setInList(Collection inList) { + this.inList = inList; + } +}; diff --git a/filter/src/main/java/org/apache/rocketmq/filter/parser/ParseException.java b/filter/src/main/java/org/apache/rocketmq/filter/parser/ParseException.java new file mode 100644 index 00000000..2ccccaf5 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/parser/ParseException.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* Generated By:JavaCC: Do not edit this line. ParseException.java Version 5.0 */ +/* JavaCCOptions:KEEP_LINE_COL=null */ +package org.apache.rocketmq.filter.parser; + +/** + * This exception is thrown when parse errors are encountered. + * You can explicitly create objects of this exception type by + * calling the method generateParseException in the generated + * parser. + *

    + * You can modify this class to customize your error reporting + * mechanisms so long as you retain the public fields. + */ +public class ParseException extends Exception { + + /** + * The version identifier for this Serializable class. + * Increment only if the serialized form of the + * class changes. + */ + private static final long serialVersionUID = 1L; + + /** + * This constructor is used by the method "generateParseException" + * in the generated parser. Calling this constructor generates + * a new object of this type with the fields "currentToken", + * "expectedTokenSequences", and "TOKEN_IMAGE" set. + */ + public ParseException(Token currentTokenVal, + int[][] expectedTokenSequencesVal, + String[] tokenImageVal + ) { + super(initialise(currentTokenVal, expectedTokenSequencesVal, tokenImageVal)); + currentToken = currentTokenVal; + expectedTokenSequences = expectedTokenSequencesVal; + tokenImage = tokenImageVal; + } + + /** + * The following constructors are for use by you for whatever + * purpose you can think of. Constructing the exception in this + * manner makes the exception behave in the normal way - i.e., as + * documented in the class "Throwable". The fields "errorToken", + * "expectedTokenSequences", and "TOKEN_IMAGE" do not contain + * relevant information. The JavaCC generated code does not use + * these constructors. + */ + + public ParseException() { + super(); + } + + /** + * Constructor with message. + */ + public ParseException(String message) { + super(message); + } + + /** + * This is the last token that has been consumed successfully. If + * this object has been created due to a parse error, the token + * followng this token will (therefore) be the first error token. + */ + public Token currentToken; + + /** + * Each entry in this array is an array of integers. Each array + * of integers represents a sequence of tokens (by their ordinal + * values) that is expected at this point of the parse. + */ + public int[][] expectedTokenSequences; + + /** + * This is a reference to the "TOKEN_IMAGE" array of the generated + * parser within which the parse error occurred. This array is + * defined in the generated ...Constants interface. + */ + public String[] tokenImage; + + /** + * It uses "currentToken" and "expectedTokenSequences" to generate a parse + * error message and returns it. If this object has been created + * due to a parse error, and you do not catch it (it gets thrown + * from the parser) the correct error message + * gets displayed. + */ + private static String initialise(Token currentToken, + int[][] expectedTokenSequences, + String[] tokenImage) { + String eol = System.getProperty("line.separator", "\n"); + StringBuffer expected = new StringBuffer(); + int maxSize = 0; + for (int i = 0; i < expectedTokenSequences.length; i++) { + if (maxSize < expectedTokenSequences[i].length) { + maxSize = expectedTokenSequences[i].length; + } + for (int j = 0; j < expectedTokenSequences[i].length; j++) { + expected.append(tokenImage[expectedTokenSequences[i][j]]).append(' '); + } + if (expectedTokenSequences[i][expectedTokenSequences[i].length - 1] != 0) { + expected.append("..."); + } + expected.append(eol).append(" "); + } + String retval = "Encountered \""; + Token tok = currentToken.next; + for (int i = 0; i < maxSize; i++) { + if (i != 0) + retval += " "; + if (tok.kind == 0) { + retval += tokenImage[0]; + break; + } + retval += " " + tokenImage[tok.kind]; + retval += " \""; + retval += add_escapes(tok.image); + retval += " \""; + tok = tok.next; + } + retval += "\" at line " + currentToken.next.beginLine + ", column " + currentToken.next.beginColumn; + retval += "." + eol; + if (expectedTokenSequences.length == 1) { + retval += "Was expecting:" + eol + " "; + } else { + retval += "Was expecting one of:" + eol + " "; + } + retval += expected.toString(); + return retval; + } + + /** + * The end of line string for this machine. + */ + protected String eol = System.getProperty("line.separator", "\n"); + + /** + * Used to convert raw characters to their escaped version + * when these raw version cannot be used as part of an ASCII + * string literal. + */ + static String add_escapes(String str) { + StringBuffer retval = new StringBuffer(); + char ch; + for (int i = 0; i < str.length(); i++) { + switch (str.charAt(i)) { + case 0: + continue; + case '\b': + retval.append("\\b"); + continue; + case '\t': + retval.append("\\t"); + continue; + case '\n': + retval.append("\\n"); + continue; + case '\f': + retval.append("\\f"); + continue; + case '\r': + retval.append("\\r"); + continue; + case '\"': + retval.append("\\\""); + continue; + case '\'': + retval.append("\\\'"); + continue; + case '\\': + retval.append("\\\\"); + continue; + default: + if ((ch = str.charAt(i)) < 0x20 || ch > 0x7e) { + String s = "0000" + Integer.toString(ch, 16); + retval.append("\\u" + s.substring(s.length() - 4, s.length())); + } else { + retval.append(ch); + } + continue; + } + } + return retval.toString(); + } + +} +/* JavaCC - OriginalChecksum=4c829b0daa2c9af00ddafe2441eb9097 (do not edit this line) */ diff --git a/filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParser.java b/filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParser.java new file mode 100644 index 00000000..74e5501b --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParser.java @@ -0,0 +1,1354 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* Generated By:JavaCC: Do not edit this line. SelectorParser.java */ +package org.apache.rocketmq.filter.parser; + +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import org.apache.rocketmq.filter.expression.BooleanExpression; +import org.apache.rocketmq.filter.expression.ComparisonExpression; +import org.apache.rocketmq.filter.expression.ConstantExpression; +import org.apache.rocketmq.filter.expression.Expression; +import org.apache.rocketmq.filter.expression.LogicExpression; +import org.apache.rocketmq.filter.expression.MQFilterException; +import org.apache.rocketmq.filter.expression.PropertyExpression; +import org.apache.rocketmq.filter.expression.UnaryExpression; + +import java.io.StringReader; +import java.util.ArrayList; + +/** + * JMS Selector Parser generated by JavaCC + *

    + * Do not edit this .java file directly - it is autogenerated from SelectorParser.jj + */ +public class SelectorParser implements SelectorParserConstants { + + private static final Cache PARSE_CACHE = CacheBuilder.newBuilder().maximumSize(100).build(); + // private static final String CONVERT_STRING_EXPRESSIONS_PREFIX = "convert_string_expressions:"; + + public static BooleanExpression parse(String sql) throws MQFilterException { + // sql = "("+sql+")"; + Object result = PARSE_CACHE.getIfPresent(sql); + if (result instanceof MQFilterException) { + throw (MQFilterException) result; + } else if (result instanceof BooleanExpression) { + return (BooleanExpression) result; + } else { + + // boolean convertStringExpressions = false; + // if( sql.startsWith(CONVERT_STRING_EXPRESSIONS_PREFIX)) { + // convertStringExpressions = true; + // sql = sql.substring(CONVERT_STRING_EXPRESSIONS_PREFIX.length()); + // } + // + // if( convertStringExpressions ) { + // ComparisonExpression.CONVERT_STRING_EXPRESSIONS.set(true); + // } + ComparisonExpression.CONVERT_STRING_EXPRESSIONS.set(true); + try { + + BooleanExpression e = new SelectorParser(sql).parse(); + PARSE_CACHE.put(sql, e); + return e; + } catch (MQFilterException t) { + PARSE_CACHE.put(sql, t); + throw t; + } finally { + ComparisonExpression.CONVERT_STRING_EXPRESSIONS.remove(); + // if( convertStringExpressions ) { + // ComparisonExpression.CONVERT_STRING_EXPRESSIONS.remove(); + // } + } + } + } + + public static void clearCache() { + PARSE_CACHE.cleanUp(); + } + + private String sql; + + protected SelectorParser(String sql) { + this(new StringReader(sql)); + this.sql = sql; + } + + protected BooleanExpression parse() throws MQFilterException { + try { + return this.JmsSelector(); + } catch (Throwable e) { + throw new MQFilterException("Invalid MessageSelector. ", e); + } + } + + private BooleanExpression asBooleanExpression(Expression value) throws ParseException { + if (value instanceof BooleanExpression) { + return (BooleanExpression) value; + } + if (value instanceof PropertyExpression) { + return UnaryExpression.createBooleanCast(value); + } + throw new ParseException("Expression will not result in a boolean value: " + value); + } + + // ---------------------------------------------------------------------------- + // Grammer + // ---------------------------------------------------------------------------- + final public BooleanExpression JmsSelector() throws ParseException { + Expression left = null; + left = orExpression(); + { + if (true) + return asBooleanExpression(left); + } + throw new Error("Missing return statement in function"); + } + + final public Expression orExpression() throws ParseException { + Expression left; + Expression right; + left = andExpression(); + label_1: + while (true) { + switch ((jjNtk == -1) ? jj_ntk() : jjNtk) { + case OR: + break; + default: + jjLa1[0] = jjGen; + break label_1; + } + jj_consume_token(OR); + right = andExpression(); + left = LogicExpression.createOR(asBooleanExpression(left), asBooleanExpression(right)); + } + { + if (true) + return left; + } + throw new Error("Missing return statement in function"); + } + + final public Expression andExpression() throws ParseException { + Expression left; + Expression right; + left = equalityExpression(); + label_2: + while (true) { + switch ((jjNtk == -1) ? jj_ntk() : jjNtk) { + case AND: + break; + default: + jjLa1[1] = jjGen; + break label_2; + } + jj_consume_token(AND); + right = equalityExpression(); + left = LogicExpression.createAND(asBooleanExpression(left), asBooleanExpression(right)); + } + { + if (true) + return left; + } + throw new Error("Missing return statement in function"); + } + + final public Expression equalityExpression() throws ParseException { + Expression left; + Expression right; + left = comparisonExpression(); + label_3: + while (true) { + switch ((jjNtk == -1) ? jj_ntk() : jjNtk) { + case IS: + case 22: + case 23: + break; + default: + jjLa1[2] = jjGen; + break label_3; + } + switch ((jjNtk == -1) ? jj_ntk() : jjNtk) { + case 22: + jj_consume_token(22); + right = comparisonExpression(); + left = ComparisonExpression.createEqual(left, right); + break; + case 23: + jj_consume_token(23); + right = comparisonExpression(); + left = ComparisonExpression.createNotEqual(left, right); + break; + default: + jjLa1[3] = jjGen; + if (jj_2_1(2)) { + jj_consume_token(IS); + jj_consume_token(NULL); + left = ComparisonExpression.createIsNull(left); + } else { + switch ((jjNtk == -1) ? jj_ntk() : jjNtk) { + case IS: + jj_consume_token(IS); + jj_consume_token(NOT); + jj_consume_token(NULL); + left = ComparisonExpression.createIsNotNull(left); + break; + default: + jjLa1[4] = jjGen; + jj_consume_token(-1); + throw new ParseException(); + } + } + } + } + { + if (true) + return left; + } + throw new Error("Missing return statement in function"); + } + + final public Expression comparisonExpression() throws ParseException { + Expression left; + Expression right; + Expression low; + Expression high; + String t, u; + boolean not; + ArrayList list; + left = unaryExpr(); + label_4: + while (true) { + switch ((jjNtk == -1) ? jj_ntk() : jjNtk) { + case NOT: + case BETWEEN: + case IN: + case 24: + case 25: + case 26: + case 27: + break; + default: + jjLa1[5] = jjGen; + break label_4; + } + switch ((jjNtk == -1) ? jj_ntk() : jjNtk) { + case 24: + jj_consume_token(24); + right = unaryExpr(); + left = ComparisonExpression.createGreaterThan(left, right); + break; + case 25: + jj_consume_token(25); + right = unaryExpr(); + left = ComparisonExpression.createGreaterThanEqual(left, right); + break; + case 26: + jj_consume_token(26); + right = unaryExpr(); + left = ComparisonExpression.createLessThan(left, right); + break; + case 27: + jj_consume_token(27); + right = unaryExpr(); + left = ComparisonExpression.createLessThanEqual(left, right); + break; + case BETWEEN: + jj_consume_token(BETWEEN); + low = unaryExpr(); + jj_consume_token(AND); + high = unaryExpr(); + left = ComparisonExpression.createBetween(left, low, high); + break; + default: + jjLa1[8] = jjGen; + if (jj_2_2(2)) { + jj_consume_token(NOT); + jj_consume_token(BETWEEN); + low = unaryExpr(); + jj_consume_token(AND); + high = unaryExpr(); + left = ComparisonExpression.createNotBetween(left, low, high); + } else { + switch ((jjNtk == -1) ? jj_ntk() : jjNtk) { + case IN: + jj_consume_token(IN); + jj_consume_token(28); + t = stringLitteral(); + list = new ArrayList(); + list.add(t); + label_5: + while (true) { + switch ((jjNtk == -1) ? jj_ntk() : jjNtk) { + case 29: + break; + default: + jjLa1[6] = jjGen; + break label_5; + } + jj_consume_token(29); + t = stringLitteral(); + list.add(t); + } + jj_consume_token(30); + left = ComparisonExpression.createInFilter(left, list); + break; + default: + jjLa1[9] = jjGen; + if (jj_2_3(2)) { + jj_consume_token(NOT); + jj_consume_token(IN); + jj_consume_token(28); + t = stringLitteral(); + list = new ArrayList(); + list.add(t); + label_6: + while (true) { + switch ((jjNtk == -1) ? jj_ntk() : jjNtk) { + case 29: + break; + default: + jjLa1[7] = jjGen; + break label_6; + } + jj_consume_token(29); + t = stringLitteral(); + list.add(t); + } + jj_consume_token(30); + left = ComparisonExpression.createNotInFilter(left, list); + } else { + jj_consume_token(-1); + throw new ParseException(); + } + } + } + } + } + { + if (true) + return left; + } + throw new Error("Missing return statement in function"); + } + + final public Expression unaryExpr() throws ParseException { + String s = null; + Expression left = null; + if (jj_2_4(2147483647)) { + jj_consume_token(31); + left = unaryExpr(); + } else { + switch ((jjNtk == -1) ? jj_ntk() : jjNtk) { + case 32: + jj_consume_token(32); + left = unaryExpr(); + left = UnaryExpression.createNegate(left); + break; + case NOT: + jj_consume_token(NOT); + left = unaryExpr(); + left = UnaryExpression.createNOT(asBooleanExpression(left)); + break; + case TRUE: + case FALSE: + case NULL: + case DECIMAL_LITERAL: + case FLOATING_POINT_LITERAL: + case STRING_LITERAL: + case ID: + case 28: + left = primaryExpr(); + break; + default: + jjLa1[10] = jjGen; + jj_consume_token(-1); + throw new ParseException(); + } + } + { + if (true) + return left; + } + throw new Error("Missing return statement in function"); + } + + final public Expression primaryExpr() throws ParseException { + Expression left = null; + switch ((jjNtk == -1) ? jj_ntk() : jjNtk) { + case TRUE: + case FALSE: + case NULL: + case DECIMAL_LITERAL: + case FLOATING_POINT_LITERAL: + case STRING_LITERAL: + left = literal(); + break; + case ID: + left = variable(); + break; + case 28: + jj_consume_token(28); + left = orExpression(); + jj_consume_token(30); + break; + default: + jjLa1[11] = jjGen; + jj_consume_token(-1); + throw new ParseException(); + } + { + if (true) + return left; + } + throw new Error("Missing return statement in function"); + } + + final public ConstantExpression literal() throws ParseException { + Token t; + String s; + ConstantExpression left = null; + switch ((jjNtk == -1) ? jj_ntk() : jjNtk) { + case STRING_LITERAL: + s = stringLitteral(); + left = new ConstantExpression(s); + break; + case DECIMAL_LITERAL: + t = jj_consume_token(DECIMAL_LITERAL); + left = ConstantExpression.createFromDecimal(t.image); + break; + case FLOATING_POINT_LITERAL: + t = jj_consume_token(FLOATING_POINT_LITERAL); + left = ConstantExpression.createFloat(t.image); + break; + case TRUE: + jj_consume_token(TRUE); + left = ConstantExpression.TRUE; + break; + case FALSE: + jj_consume_token(FALSE); + left = ConstantExpression.FALSE; + break; + case NULL: + jj_consume_token(NULL); + left = ConstantExpression.NULL; + break; + default: + jjLa1[12] = jjGen; + jj_consume_token(-1); + throw new ParseException(); + } + { + if (true) + return left; + } + throw new Error("Missing return statement in function"); + } + + final public String stringLitteral() throws ParseException { + Token t; + StringBuffer rc = new StringBuffer(); + boolean first = true; + t = jj_consume_token(STRING_LITERAL); + // Decode the sting value. + String image = t.image; + for (int i = 1; i < image.length() - 1; i++) { + char c = image.charAt(i); + if (c == '\'') + i++; + rc.append(c); + } + { + if (true) + return rc.toString(); + } + throw new Error("Missing return statement in function"); + } + + final public PropertyExpression variable() throws ParseException { + Token t; + PropertyExpression left = null; + t = jj_consume_token(ID); + left = new PropertyExpression(t.image); + { + if (true) + return left; + } + throw new Error("Missing return statement in function"); + } + + private boolean jj_2_1(int xla) { + jjLa = xla; + jjLastpos = jjScanpos = token; + try { + return !jj_3_1(); + } catch (LookaheadSuccess ls) { + return true; + } finally { + jj_save(0, xla); + } + } + + private boolean jj_2_2(int xla) { + jjLa = xla; + jjLastpos = jjScanpos = token; + try { + return !jj_3_2(); + } catch (LookaheadSuccess ls) { + return true; + } finally { + jj_save(1, xla); + } + } + + private boolean jj_2_3(int xla) { + jjLa = xla; + jjLastpos = jjScanpos = token; + try { + return !jj_3_3(); + } catch (LookaheadSuccess ls) { + return true; + } finally { + jj_save(2, xla); + } + } + + private boolean jj_2_4(int xla) { + jjLa = xla; + jjLastpos = jjScanpos = token; + try { + return !jj_3_4(); + } catch (LookaheadSuccess ls) { + return true; + } finally { + jj_save(3, xla); + } + } + + private boolean jj_3R_7() { + Token xsp; + xsp = jjScanpos; + if (jj_3R_8()) { + jjScanpos = xsp; + if (jj_3R_9()) { + jjScanpos = xsp; + if (jj_3R_10()) { + jjScanpos = xsp; + if (jj_3R_11()) + return true; + } + } + } + return false; + } + + private boolean jj_3R_43() { + if (jj_scan_token(29)) + return true; + if (jj_3R_27()) + return true; + return false; + } + + private boolean jj_3R_24() { + if (jj_scan_token(NULL)) + return true; + return false; + } + + private boolean jj_3R_35() { + if (jj_scan_token(IS)) + return true; + if (jj_scan_token(NOT)) + return true; + if (jj_scan_token(NULL)) + return true; + return false; + } + + private boolean jj_3_1() { + if (jj_scan_token(IS)) + return true; + if (jj_scan_token(NULL)) + return true; + return false; + } + + private boolean jj_3R_23() { + if (jj_scan_token(FALSE)) + return true; + return false; + } + + private boolean jj_3R_34() { + if (jj_scan_token(23)) + return true; + if (jj_3R_30()) + return true; + return false; + } + + private boolean jj_3R_22() { + if (jj_scan_token(TRUE)) + return true; + return false; + } + + private boolean jj_3_3() { + if (jj_scan_token(NOT)) + return true; + if (jj_scan_token(IN)) + return true; + if (jj_scan_token(28)) + return true; + if (jj_3R_27()) + return true; + Token xsp; + while (true) { + xsp = jjScanpos; + if (jj_3R_43()) { + jjScanpos = xsp; + break; + } + } + if (jj_scan_token(30)) + return true; + return false; + } + + private boolean jj_3R_31() { + Token xsp; + xsp = jjScanpos; + if (jj_3R_33()) { + jjScanpos = xsp; + if (jj_3R_34()) { + jjScanpos = xsp; + if (jj_3_1()) { + jjScanpos = xsp; + if (jj_3R_35()) + return true; + } + } + } + return false; + } + + private boolean jj_3R_33() { + if (jj_scan_token(22)) + return true; + if (jj_3R_30()) + return true; + return false; + } + + private boolean jj_3R_42() { + if (jj_scan_token(29)) + return true; + if (jj_3R_27()) + return true; + return false; + } + + private boolean jj_3R_21() { + if (jj_scan_token(FLOATING_POINT_LITERAL)) + return true; + return false; + } + + private boolean jj_3R_20() { + if (jj_scan_token(DECIMAL_LITERAL)) + return true; + return false; + } + + private boolean jj_3R_28() { + if (jj_3R_30()) + return true; + Token xsp; + while (true) { + xsp = jjScanpos; + if (jj_3R_31()) { + jjScanpos = xsp; + break; + } + } + return false; + } + + private boolean jj_3R_41() { + if (jj_scan_token(IN)) + return true; + if (jj_scan_token(28)) + return true; + if (jj_3R_27()) + return true; + Token xsp; + while (true) { + xsp = jjScanpos; + if (jj_3R_42()) { + jjScanpos = xsp; + break; + } + } + if (jj_scan_token(30)) + return true; + return false; + } + + private boolean jj_3R_19() { + if (jj_3R_27()) + return true; + return false; + } + + private boolean jj_3R_29() { + if (jj_scan_token(AND)) + return true; + if (jj_3R_28()) + return true; + return false; + } + + private boolean jj_3R_16() { + Token xsp; + xsp = jjScanpos; + if (jj_3R_19()) { + jjScanpos = xsp; + if (jj_3R_20()) { + jjScanpos = xsp; + if (jj_3R_21()) { + jjScanpos = xsp; + if (jj_3R_22()) { + jjScanpos = xsp; + if (jj_3R_23()) { + jjScanpos = xsp; + if (jj_3R_24()) + return true; + } + } + } + } + } + return false; + } + + private boolean jj_3_2() { + if (jj_scan_token(NOT)) + return true; + if (jj_scan_token(BETWEEN)) + return true; + if (jj_3R_7()) + return true; + if (jj_scan_token(AND)) + return true; + if (jj_3R_7()) + return true; + return false; + } + + private boolean jj_3R_40() { + if (jj_scan_token(BETWEEN)) + return true; + if (jj_3R_7()) + return true; + if (jj_scan_token(AND)) + return true; + if (jj_3R_7()) + return true; + return false; + } + + private boolean jj_3R_25() { + if (jj_3R_28()) + return true; + Token xsp; + while (true) { + xsp = jjScanpos; + if (jj_3R_29()) { + jjScanpos = xsp; + break; + } + } + return false; + } + + private boolean jj_3R_39() { + if (jj_scan_token(27)) + return true; + if (jj_3R_7()) + return true; + return false; + } + + private boolean jj_3R_15() { + if (jj_scan_token(28)) + return true; + if (jj_3R_18()) + return true; + if (jj_scan_token(30)) + return true; + return false; + } + + private boolean jj_3R_14() { + if (jj_3R_17()) + return true; + return false; + } + + private boolean jj_3R_38() { + if (jj_scan_token(26)) + return true; + if (jj_3R_7()) + return true; + return false; + } + + private boolean jj_3R_13() { + if (jj_3R_16()) + return true; + return false; + } + + private boolean jj_3R_26() { + if (jj_scan_token(OR)) + return true; + if (jj_3R_25()) + return true; + return false; + } + + private boolean jj_3R_17() { + if (jj_scan_token(ID)) + return true; + return false; + } + + private boolean jj_3R_37() { + if (jj_scan_token(25)) + return true; + if (jj_3R_7()) + return true; + return false; + } + + private boolean jj_3R_12() { + Token xsp; + xsp = jjScanpos; + if (jj_3R_13()) { + jjScanpos = xsp; + if (jj_3R_14()) { + jjScanpos = xsp; + if (jj_3R_15()) + return true; + } + } + return false; + } + + private boolean jj_3R_32() { + Token xsp; + xsp = jjScanpos; + if (jj_3R_36()) { + jjScanpos = xsp; + if (jj_3R_37()) { + jjScanpos = xsp; + if (jj_3R_38()) { + jjScanpos = xsp; + if (jj_3R_39()) { + jjScanpos = xsp; + if (jj_3R_40()) { + jjScanpos = xsp; + if (jj_3_2()) { + jjScanpos = xsp; + if (jj_3R_41()) { + jjScanpos = xsp; + if (jj_3_3()) + return true; + } + } + } + } + } + } + } + return false; + } + + private boolean jj_3R_36() { + if (jj_scan_token(24)) + return true; + if (jj_3R_7()) + return true; + return false; + } + + private boolean jj_3R_11() { + if (jj_3R_12()) + return true; + return false; + } + + private boolean jj_3R_18() { + if (jj_3R_25()) + return true; + Token xsp; + while (true) { + xsp = jjScanpos; + if (jj_3R_26()) { + jjScanpos = xsp; + break; + } + } + return false; + } + + private boolean jj_3_4() { + if (jj_scan_token(31)) + return true; + if (jj_3R_7()) + return true; + return false; + } + + private boolean jj_3R_10() { + if (jj_scan_token(NOT)) + return true; + if (jj_3R_7()) + return true; + return false; + } + + private boolean jj_3R_9() { + if (jj_scan_token(32)) + return true; + if (jj_3R_7()) + return true; + return false; + } + + private boolean jj_3R_27() { + if (jj_scan_token(STRING_LITERAL)) + return true; + return false; + } + + private boolean jj_3R_30() { + if (jj_3R_7()) + return true; + Token xsp; + while (true) { + xsp = jjScanpos; + if (jj_3R_32()) { + jjScanpos = xsp; + break; + } + } + return false; + } + + private boolean jj_3R_8() { + if (jj_scan_token(31)) + return true; + if (jj_3R_7()) + return true; + return false; + } + + /** + * Generated Token Manager. + */ + public SelectorParserTokenManager tokenSource; + SimpleCharStream jjInputStream; + /** + * Current token. + */ + public Token token; + /** + * Next token. + */ + public Token jjNt; + private int jjNtk; + private Token jjScanpos, jjLastpos; + private int jjLa; + private int jjGen; + final private int[] jjLa1 = new int[13]; + static private int[] jjLa10; + static private int[] jjLa11; + + static { + jj_la1_init_0(); + jj_la1_init_1(); + } + + private static void jj_la1_init_0() { + jjLa10 = new int[]{0x400, 0x200, 0xc10000, 0xc00000, 0x10000, 0xf001900, 0x20000000, 0x20000000, 0xf000800, + 0x1000, 0x1036e100, 0x1036e000, 0x16e000}; + } + + private static void jj_la1_init_1() { + jjLa11 = new int[]{0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x0, 0x0}; + } + + final private JJCalls[] jj2Rtns = new JJCalls[4]; + private boolean jjRescan = false; + private int jjGc = 0; + + /** + * Constructor with InputStream. + */ + public SelectorParser(java.io.InputStream stream) { + this(stream, null); + } + + /** + * Constructor with InputStream and supplied encoding + */ + public SelectorParser(java.io.InputStream stream, String encoding) { + try { + jjInputStream = new SimpleCharStream(stream, encoding, 1, 1); + } catch (java.io.UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + tokenSource = new SelectorParserTokenManager(jjInputStream); + token = new Token(); + jjNtk = -1; + jjGen = 0; + for (int i = 0; i < 13; i++) + jjLa1[i] = -1; + for (int i = 0; i < jj2Rtns.length; i++) + jj2Rtns[i] = new JJCalls(); + } + + /** + * Reinitialise. + */ + public void ReInit(java.io.InputStream stream) { + ReInit(stream, null); + } + + /** + * Reinitialise. + */ + public void ReInit(java.io.InputStream stream, String encoding) { + try { + jjInputStream.ReInit(stream, encoding, 1, 1); + } catch (java.io.UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + tokenSource.ReInit(jjInputStream); + token = new Token(); + jjNtk = -1; + jjGen = 0; + for (int i = 0; i < 13; i++) + jjLa1[i] = -1; + for (int i = 0; i < jj2Rtns.length; i++) + jj2Rtns[i] = new JJCalls(); + } + + /** + * Constructor. + */ + public SelectorParser(java.io.Reader stream) { + jjInputStream = new SimpleCharStream(stream, 1, 1); + tokenSource = new SelectorParserTokenManager(jjInputStream); + token = new Token(); + jjNtk = -1; + jjGen = 0; + for (int i = 0; i < 13; i++) + jjLa1[i] = -1; + for (int i = 0; i < jj2Rtns.length; i++) + jj2Rtns[i] = new JJCalls(); + } + + /** + * Reinitialise. + */ + public void ReInit(java.io.Reader stream) { + jjInputStream.ReInit(stream, 1, 1); + tokenSource.ReInit(jjInputStream); + token = new Token(); + jjNtk = -1; + jjGen = 0; + for (int i = 0; i < 13; i++) + jjLa1[i] = -1; + for (int i = 0; i < jj2Rtns.length; i++) + jj2Rtns[i] = new JJCalls(); + } + + /** + * Constructor with generated Token Manager. + */ + public SelectorParser(SelectorParserTokenManager tm) { + tokenSource = tm; + token = new Token(); + jjNtk = -1; + jjGen = 0; + for (int i = 0; i < 13; i++) + jjLa1[i] = -1; + for (int i = 0; i < jj2Rtns.length; i++) + jj2Rtns[i] = new JJCalls(); + } + + /** + * Reinitialise. + */ + public void ReInit(SelectorParserTokenManager tm) { + tokenSource = tm; + token = new Token(); + jjNtk = -1; + jjGen = 0; + for (int i = 0; i < 13; i++) + jjLa1[i] = -1; + for (int i = 0; i < jj2Rtns.length; i++) + jj2Rtns[i] = new JJCalls(); + } + + private Token jj_consume_token(int kind) throws ParseException { + Token oldToken; + if ((oldToken = token).next != null) + token = token.next; + else + token = token.next = tokenSource.getNextToken(); + jjNtk = -1; + if (token.kind == kind) { + jjGen++; + if (++jjGc > 100) { + jjGc = 0; + for (int i = 0; i < jj2Rtns.length; i++) { + JJCalls c = jj2Rtns[i]; + while (c != null) { + if (c.gen < jjGen) + c.first = null; + c = c.next; + } + } + } + return token; + } + token = oldToken; + jjKind = kind; + throw generateParseException(); + } + + static private final class LookaheadSuccess extends java.lang.Error { + } + + final private LookaheadSuccess jjLs = new LookaheadSuccess(); + + private boolean jj_scan_token(int kind) { + if (jjScanpos == jjLastpos) { + jjLa--; + if (jjScanpos.next == null) { + jjLastpos = jjScanpos = jjScanpos.next = tokenSource.getNextToken(); + } else { + jjLastpos = jjScanpos = jjScanpos.next; + } + } else { + jjScanpos = jjScanpos.next; + } + if (jjRescan) { + int i = 0; + Token tok = token; + while (tok != null && tok != jjScanpos) { + i++; + tok = tok.next; + } + if (tok != null) + jj_add_error_token(kind, i); + } + if (jjScanpos.kind != kind) + return true; + if (jjLa == 0 && jjScanpos == jjLastpos) + throw jjLs; + return false; + } + + /** + * Get the next Token. + */ + final public Token getNextToken() { + if (token.next != null) + token = token.next; + else + token = token.next = tokenSource.getNextToken(); + jjNtk = -1; + jjGen++; + return token; + } + + /** + * Get the specific Token. + */ + final public Token getToken(int index) { + Token t = token; + for (int i = 0; i < index; i++) { + if (t.next != null) + t = t.next; + else + t = t.next = tokenSource.getNextToken(); + } + return t; + } + + private int jj_ntk() { + if ((jjNt = token.next) == null) + return jjNtk = (token.next = tokenSource.getNextToken()).kind; + else + return jjNtk = jjNt.kind; + } + + private java.util.List jjExpentries = new java.util.ArrayList(); + private int[] jjExpentry; + private int jjKind = -1; + private int[] jjLasttokens = new int[100]; + private int jjEndpos; + + private void jj_add_error_token(int kind, int pos) { + if (pos >= 100) + return; + if (pos == jjEndpos + 1) { + jjLasttokens[jjEndpos++] = kind; + } else if (jjEndpos != 0) { + jjExpentry = new int[jjEndpos]; + for (int i = 0; i < jjEndpos; i++) { + jjExpentry[i] = jjLasttokens[i]; + } + jj_entries_loop: + for (java.util.Iterator it = jjExpentries.iterator(); it.hasNext(); ) { + int[] oldentry = (int[]) (it.next()); + if (oldentry.length == jjExpentry.length) { + for (int i = 0; i < jjExpentry.length; i++) { + if (oldentry[i] != jjExpentry[i]) { + continue jj_entries_loop; + } + } + jjExpentries.add(jjExpentry); + break jj_entries_loop; + } + } + if (pos != 0) + jjLasttokens[(jjEndpos = pos) - 1] = kind; + } + } + + /** + * Generate ParseException. + */ + public ParseException generateParseException() { + jjExpentries.clear(); + boolean[] la1tokens = new boolean[33]; + if (jjKind >= 0) { + la1tokens[jjKind] = true; + jjKind = -1; + } + for (int i = 0; i < 13; i++) { + if (jjLa1[i] == jjGen) { + for (int j = 0; j < 32; j++) { + if ((jjLa10[i] & (1 << j)) != 0) { + la1tokens[j] = true; + } + if ((jjLa11[i] & (1 << j)) != 0) { + la1tokens[32 + j] = true; + } + } + } + } + for (int i = 0; i < 33; i++) { + if (la1tokens[i]) { + jjExpentry = new int[1]; + jjExpentry[0] = i; + jjExpentries.add(jjExpentry); + } + } + jjEndpos = 0; + jj_rescan_token(); + jj_add_error_token(0, 0); + int[][] exptokseq = new int[jjExpentries.size()][]; + for (int i = 0; i < jjExpentries.size(); i++) { + exptokseq[i] = jjExpentries.get(i); + } + return new ParseException(token, exptokseq, TOKEN_IMAGE); + } + + /** + * Enable tracing. + */ + final public void enable_tracing() { + } + + /** + * Disable tracing. + */ + final public void disable_tracing() { + } + + private void jj_rescan_token() { + jjRescan = true; + for (int i = 0; i < 4; i++) { + try { + JJCalls p = jj2Rtns[i]; + do { + if (p.gen > jjGen) { + jjLa = p.arg; + jjLastpos = jjScanpos = p.first; + switch (i) { + case 0: + jj_3_1(); + break; + case 1: + jj_3_2(); + break; + case 2: + jj_3_3(); + break; + case 3: + jj_3_4(); + break; + } + } + p = p.next; + } while (p != null); + } catch (LookaheadSuccess ls) { + } + } + jjRescan = false; + } + + private void jj_save(int index, int xla) { + JJCalls p = jj2Rtns[index]; + while (p.gen > jjGen) { + if (p.next == null) { + p = p.next = new JJCalls(); + break; + } + p = p.next; + } + p.gen = jjGen + xla - jjLa; + p.first = token; + p.arg = xla; + } + + static final class JJCalls { + int gen; + Token first; + int arg; + JJCalls next; + } + +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParser.jj b/filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParser.jj new file mode 100644 index 00000000..5d1a4a7d --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParser.jj @@ -0,0 +1,524 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * This file was taken from ActiveMQ activemq-client/src/main/grammar/SelectorParser.jj. + * + * There are some modifications: + * 1. Convert string expressions default; + * 2. HEX_LITERAL and OCTAL_LITERAL were removed; + * 3. LIKE, ESCAPE, XPATH and XQUERY were removed; + * 4. Computation expressions were removed; + */ + +// ---------------------------------------------------------------------------- +// OPTIONS +// ---------------------------------------------------------------------------- +options { + STATIC = false; + UNICODE_INPUT = true; + + //ERROR_REPORTING = false; +} + +// ---------------------------------------------------------------------------- +// PARSER +// ---------------------------------------------------------------------------- + +PARSER_BEGIN(SelectorParser) + +package org.apache.rocketmq.filter.parser; + +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import org.apache.rocketmq.filter.expression.BooleanExpression; +import org.apache.rocketmq.filter.expression.ComparisonExpression; +import org.apache.rocketmq.filter.expression.ConstantExpression; +import org.apache.rocketmq.filter.expression.Expression; +import org.apache.rocketmq.filter.expression.LogicExpression; +import org.apache.rocketmq.filter.expression.MQFilterException; +import org.apache.rocketmq.filter.expression.PropertyExpression; +import org.apache.rocketmq.filter.expression.UnaryExpression; +import org.apache.rocketmq.filter.util.LRUCache; + +import java.io.StringReader; +import java.util.ArrayList; + +/** + * JMS Selector Parser generated by JavaCC + * + * Do not edit this .java file directly - it is autogenerated from SelectorParser.jj + */ +public class SelectorParser { + + private static final Cache PARSE_CACHE = CacheBuilder.newBuilder().maximumSize(100).build(); +// private static final String CONVERT_STRING_EXPRESSIONS_PREFIX = "convert_string_expressions:"; + + public static BooleanExpression parse(String sql) throws MQFilterException { +// sql = "("+sql+")"; + Object result = PARSE_CACHE.getIfPresent(sql); + if (result instanceof MQFilterException) { + throw (MQFilterException) result; + } else if (result instanceof BooleanExpression) { + return (BooleanExpression) result; + } else { + +// boolean convertStringExpressions = false; +// if( sql.startsWith(CONVERT_STRING_EXPRESSIONS_PREFIX)) { +// convertStringExpressions = true; +// sql = sql.substring(CONVERT_STRING_EXPRESSIONS_PREFIX.length()); +// } +// +// if( convertStringExpressions ) { +// ComparisonExpression.CONVERT_STRING_EXPRESSIONS.set(true); +// } + ComparisonExpression.CONVERT_STRING_EXPRESSIONS.set(true); + try { + + BooleanExpression e = new SelectorParser(sql).parse(); + PARSE_CACHE.put(sql, e); + return e; + } catch (MQFilterException t) { + PARSE_CACHE.put(sql, t); + throw t; + } finally { + ComparisonExpression.CONVERT_STRING_EXPRESSIONS.remove(); +// if( convertStringExpressions ) { +// ComparisonExpression.CONVERT_STRING_EXPRESSIONS.remove(); +// } + } + } + } + + public static void clearCache() { + PARSE_CACHE.cleanUp(); + } + + private String sql; + + protected SelectorParser(String sql) { + this(new StringReader(sql)); + this.sql = sql; + } + + protected BooleanExpression parse() throws MQFilterException { + try { + return this.JmsSelector(); + } + catch (Throwable e) { + throw new MQFilterException("Invalid MessageSelector. ", e); + } + } + + private BooleanExpression asBooleanExpression(Expression value) throws ParseException { + if (value instanceof BooleanExpression) { + return (BooleanExpression) value; + } + if (value instanceof PropertyExpression) { + return UnaryExpression.createBooleanCast( value ); + } + throw new ParseException("Expression will not result in a boolean value: " + value); + } + +} + +PARSER_END(SelectorParser) + +// ---------------------------------------------------------------------------- +// Tokens +// ---------------------------------------------------------------------------- + +/* White Space */ +SPECIAL_TOKEN : +{ + " " | "\t" | "\n" | "\r" | "\f" +} + +/* Comments */ +SKIP: +{ + +} + +SKIP: +{ + +} + +/* Reserved Words */ +TOKEN [IGNORE_CASE] : +{ + < NOT : "NOT"> + | < AND : "AND"> + | < OR : "OR"> + | < BETWEEN : "BETWEEN"> + | < IN : "IN"> + | < TRUE : "TRUE" > + | < FALSE : "FALSE" > + | < NULL : "NULL" > + | < IS : "IS" > +} + +/* Literals */ +TOKEN [IGNORE_CASE] : + +{ + + < DECIMAL_LITERAL: "0" | ["1"-"9"] (["0"-"9"])* (["l","L"])? > + | < FLOATING_POINT_LITERAL: + (["0"-"9"])+ "." (["0"-"9"])* ()? // matches: 5.5 or 5. or 5.5E10 or 5.E10 + | "." (["0"-"9"])+ ()? // matches: .5 or .5E10 + | (["0"-"9"])+ // matches: 5E10 + > + | < #EXPONENT: "E" (["+","-"])? (["0"-"9"])+ > + | < STRING_LITERAL: "'" ( ("''") | ~["'"] )* "'" > +} + +TOKEN [IGNORE_CASE] : +{ + < ID : ["a"-"z", "_", "$"] (["a"-"z","0"-"9","_", "$"])* > +} + +// ---------------------------------------------------------------------------- +// Grammer +// ---------------------------------------------------------------------------- +BooleanExpression JmsSelector() : +{ + Expression left=null; +} +{ + ( + left = orExpression() + ) + { + return asBooleanExpression(left); + } + +} + +Expression orExpression() : +{ + Expression left; + Expression right; +} +{ + ( + left = andExpression() + ( + right = andExpression() + { + left = LogicExpression.createOR(asBooleanExpression(left), asBooleanExpression(right)); + } + )* + ) + { + return left; + } + +} + + +Expression andExpression() : +{ + Expression left; + Expression right; +} +{ + ( + left = equalityExpression() + ( + right = equalityExpression() + { + left = LogicExpression.createAND(asBooleanExpression(left), asBooleanExpression(right)); + } + )* + ) + { + return left; + } +} + +Expression equalityExpression() : +{ + Expression left; + Expression right; +} +{ + ( + left = comparisonExpression() + ( + + "=" right = comparisonExpression() + { + left = ComparisonExpression.createEqual(left, right); + } + | + "<>" right = comparisonExpression() + { + left = ComparisonExpression.createNotEqual(left, right); + } + | + LOOKAHEAD(2) + + { + left = ComparisonExpression.createIsNull(left); + } + | + + { + left = ComparisonExpression.createIsNotNull(left); + } + )* + ) + { + return left; + } +} + +Expression comparisonExpression() : +{ + Expression left; + Expression right; + Expression low; + Expression high; + String t, u; + boolean not; + ArrayList list; +} +{ + ( + left = unaryExpr() + ( + + ">" right = unaryExpr() + { + left = ComparisonExpression.createGreaterThan(left, right); + } + | + ">=" right = unaryExpr() + { + left = ComparisonExpression.createGreaterThanEqual(left, right); + } + | + "<" right = unaryExpr() + { + left = ComparisonExpression.createLessThan(left, right); + } + | + "<=" right = unaryExpr() + { + left = ComparisonExpression.createLessThanEqual(left, right); + } + | + low = unaryExpr() high = unaryExpr() + { + left = ComparisonExpression.createBetween(left, low, high); + } + | + LOOKAHEAD(2) + low = unaryExpr() high = unaryExpr() + { + left = ComparisonExpression.createNotBetween(left, low, high); + } + | + + "(" + t = stringLitteral() + { + list = new ArrayList(); + list.add( t ); + } + ( + "," + t = stringLitteral() + { + list.add( t ); + } + + )* + ")" + { + left = ComparisonExpression.createInFilter(left, list); + } + | + LOOKAHEAD(2) + + "(" + t = stringLitteral() + { + list = new ArrayList(); + list.add( t ); + } + ( + "," + t = stringLitteral() + { + list.add( t ); + } + + )* + ")" + { + left = ComparisonExpression.createNotInFilter(left, list); + } + + )* + ) + { + return left; + } +} + +Expression unaryExpr() : +{ + String s=null; + Expression left=null; +} +{ + ( + LOOKAHEAD( "+" unaryExpr() ) + "+" left=unaryExpr() + | + "-" left=unaryExpr() + { + left = UnaryExpression.createNegate(left); + } + | + left=unaryExpr() + { + left = UnaryExpression.createNOT( asBooleanExpression(left) ); + } + | + left = primaryExpr() + ) + { + return left; + } + +} + +Expression primaryExpr() : +{ + Expression left=null; +} +{ + ( + left = literal() + | + left = variable() + | + "(" left = orExpression() ")" + ) + { + return left; + } +} + + + +ConstantExpression literal() : +{ + Token t; + String s; + ConstantExpression left=null; +} +{ + ( + ( + s = stringLitteral() + { + left = new ConstantExpression(s); + } + ) + | + ( + t = + { + left = ConstantExpression.createFromDecimal(t.image); + } + ) + | + ( + t = + { + left = ConstantExpression.createFloat(t.image); + } + ) + | + ( + + { + left = ConstantExpression.TRUE; + } + ) + | + ( + + { + left = ConstantExpression.FALSE; + } + ) + | + ( + + { + left = ConstantExpression.NULL; + } + ) + ) + { + return left; + } +} + +String stringLitteral() : +{ + Token t; + StringBuffer rc = new StringBuffer(); + boolean first=true; +} +{ + t = + { + // Decode the sting value. + String image = t.image; + for( int i=1; i < image.length()-1; i++ ) { + char c = image.charAt(i); + if( c == '\'' ) + i++; + rc.append(c); + } + return rc.toString(); + } +} + +PropertyExpression variable() : +{ + Token t; + PropertyExpression left=null; +} +{ + ( + t = + { + left = new PropertyExpression(t.image); + } + ) + { + return left; + } +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParserConstants.java b/filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParserConstants.java new file mode 100644 index 00000000..915658ca --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParserConstants.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* Generated By:JavaCC: Do not edit this line. SelectorParserConstants.java */ +package org.apache.rocketmq.filter.parser; + +/** + * Token literal values and constants. + * Generated by org.javacc.parser.OtherFilesGen#start() + */ +public interface SelectorParserConstants { + + /** + * End of File. + */ + int EOF = 0; + /** + * RegularExpression Id. + */ + int LINE_COMMENT = 6; + /** + * RegularExpression Id. + */ + int BLOCK_COMMENT = 7; + /** + * RegularExpression Id. + */ + int NOT = 8; + /** + * RegularExpression Id. + */ + int AND = 9; + /** + * RegularExpression Id. + */ + int OR = 10; + /** + * RegularExpression Id. + */ + int BETWEEN = 11; + /** + * RegularExpression Id. + */ + int IN = 12; + /** + * RegularExpression Id. + */ + int TRUE = 13; + /** + * RegularExpression Id. + */ + int FALSE = 14; + /** + * RegularExpression Id. + */ + int NULL = 15; + /** + * RegularExpression Id. + */ + int IS = 16; + /** + * RegularExpression Id. + */ + int DECIMAL_LITERAL = 17; + /** + * RegularExpression Id. + */ + int FLOATING_POINT_LITERAL = 18; + /** + * RegularExpression Id. + */ + int EXPONENT = 19; + /** + * RegularExpression Id. + */ + int STRING_LITERAL = 20; + /** + * RegularExpression Id. + */ + int ID = 21; + + /** + * Lexical state. + */ + int DEFAULT = 0; + + /** + * Literal token values. + */ + String[] TOKEN_IMAGE = { + "", + "\" \"", + "\"\\t\"", + "\"\\n\"", + "\"\\r\"", + "\"\\f\"", + "", + "", + "\"NOT\"", + "\"AND\"", + "\"OR\"", + "\"BETWEEN\"", + "\"IN\"", + "\"TRUE\"", + "\"FALSE\"", + "\"NULL\"", + "\"IS\"", + "", + "", + "", + "", + "", + "\"=\"", + "\"<>\"", + "\">\"", + "\">=\"", + "\"<\"", + "\"<=\"", + "\"(\"", + "\",\"", + "\")\"", + "\"+\"", + "\"-\"", + }; + +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParserTokenManager.java b/filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParserTokenManager.java new file mode 100644 index 00000000..354f5ba2 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/parser/SelectorParserTokenManager.java @@ -0,0 +1,919 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* Generated By:JavaCC: Do not edit this line. SelectorParserTokenManager.java */ +package org.apache.rocketmq.filter.parser; + +/** + * Token Manager. + */ +public class SelectorParserTokenManager implements SelectorParserConstants { + + /** + * Debug output. + */ + public java.io.PrintStream debugStream = System.out; + + /** + * Set debug output. + */ + public void setDebugStream(java.io.PrintStream ds) { + debugStream = ds; + } + + private int jjStopAtPos(int pos, int kind) { + jjmatchedKind = kind; + jjmatchedPos = pos; + return pos + 1; + } + + private int jjMoveStringLiteralDfa0_0() { + switch (curChar) { + case 9: + jjmatchedKind = 2; + return jjMoveNfa_0(5, 0); + case 10: + jjmatchedKind = 3; + return jjMoveNfa_0(5, 0); + case 12: + jjmatchedKind = 5; + return jjMoveNfa_0(5, 0); + case 13: + jjmatchedKind = 4; + return jjMoveNfa_0(5, 0); + case 32: + jjmatchedKind = 1; + return jjMoveNfa_0(5, 0); + case 40: + jjmatchedKind = 28; + return jjMoveNfa_0(5, 0); + case 41: + jjmatchedKind = 30; + return jjMoveNfa_0(5, 0); + case 43: + jjmatchedKind = 31; + return jjMoveNfa_0(5, 0); + case 44: + jjmatchedKind = 29; + return jjMoveNfa_0(5, 0); + case 45: + jjmatchedKind = 32; + return jjMoveNfa_0(5, 0); + case 60: + jjmatchedKind = 26; + return jjMoveStringLiteralDfa1_0(0x8800000L); + case 61: + jjmatchedKind = 22; + return jjMoveNfa_0(5, 0); + case 62: + jjmatchedKind = 24; + return jjMoveStringLiteralDfa1_0(0x2000000L); + case 65: + return jjMoveStringLiteralDfa1_0(0x200L); + case 66: + return jjMoveStringLiteralDfa1_0(0x800L); + case 70: + return jjMoveStringLiteralDfa1_0(0x4000L); + case 73: + return jjMoveStringLiteralDfa1_0(0x11000L); + case 78: + return jjMoveStringLiteralDfa1_0(0x8100L); + case 79: + return jjMoveStringLiteralDfa1_0(0x400L); + case 84: + return jjMoveStringLiteralDfa1_0(0x2000L); + case 97: + return jjMoveStringLiteralDfa1_0(0x200L); + case 98: + return jjMoveStringLiteralDfa1_0(0x800L); + case 102: + return jjMoveStringLiteralDfa1_0(0x4000L); + case 105: + return jjMoveStringLiteralDfa1_0(0x11000L); + case 110: + return jjMoveStringLiteralDfa1_0(0x8100L); + case 111: + return jjMoveStringLiteralDfa1_0(0x400L); + case 116: + return jjMoveStringLiteralDfa1_0(0x2000L); + default: + return jjMoveNfa_0(5, 0); + } + } + + private int jjMoveStringLiteralDfa1_0(long active0) { + try { + curChar = inputStream.readChar(); + } catch (java.io.IOException e) { + return jjMoveNfa_0(5, 0); + } + switch (curChar) { + case 61: + if ((active0 & 0x2000000L) != 0L) { + jjmatchedKind = 25; + jjmatchedPos = 1; + } else if ((active0 & 0x8000000L) != 0L) { + jjmatchedKind = 27; + jjmatchedPos = 1; + } + break; + case 62: + if ((active0 & 0x800000L) != 0L) { + jjmatchedKind = 23; + jjmatchedPos = 1; + } + break; + case 65: + return jjMoveStringLiteralDfa2_0(active0, 0x4000L); + case 69: + return jjMoveStringLiteralDfa2_0(active0, 0x800L); + case 78: + if ((active0 & 0x1000L) != 0L) { + jjmatchedKind = 12; + jjmatchedPos = 1; + } + return jjMoveStringLiteralDfa2_0(active0, 0x200L); + case 79: + return jjMoveStringLiteralDfa2_0(active0, 0x100L); + case 82: + if ((active0 & 0x400L) != 0L) { + jjmatchedKind = 10; + jjmatchedPos = 1; + } + return jjMoveStringLiteralDfa2_0(active0, 0x2000L); + case 83: + if ((active0 & 0x10000L) != 0L) { + jjmatchedKind = 16; + jjmatchedPos = 1; + } + break; + case 85: + return jjMoveStringLiteralDfa2_0(active0, 0x8000L); + case 97: + return jjMoveStringLiteralDfa2_0(active0, 0x4000L); + case 101: + return jjMoveStringLiteralDfa2_0(active0, 0x800L); + case 110: + if ((active0 & 0x1000L) != 0L) { + jjmatchedKind = 12; + jjmatchedPos = 1; + } + return jjMoveStringLiteralDfa2_0(active0, 0x200L); + case 111: + return jjMoveStringLiteralDfa2_0(active0, 0x100L); + case 114: + if ((active0 & 0x400L) != 0L) { + jjmatchedKind = 10; + jjmatchedPos = 1; + } + return jjMoveStringLiteralDfa2_0(active0, 0x2000L); + case 115: + if ((active0 & 0x10000L) != 0L) { + jjmatchedKind = 16; + jjmatchedPos = 1; + } + break; + case 117: + return jjMoveStringLiteralDfa2_0(active0, 0x8000L); + default: + break; + } + return jjMoveNfa_0(5, 1); + } + + private int jjMoveStringLiteralDfa2_0(long old0, long active0) { + if (((active0 &= old0)) == 0L) + return jjMoveNfa_0(5, 1); + try { + curChar = inputStream.readChar(); + } catch (java.io.IOException e) { + return jjMoveNfa_0(5, 1); + } + switch (curChar) { + case 68: + if ((active0 & 0x200L) != 0L) { + jjmatchedKind = 9; + jjmatchedPos = 2; + } + break; + case 76: + return jjMoveStringLiteralDfa3_0(active0, 0xc000L); + case 84: + if ((active0 & 0x100L) != 0L) { + jjmatchedKind = 8; + jjmatchedPos = 2; + } + return jjMoveStringLiteralDfa3_0(active0, 0x800L); + case 85: + return jjMoveStringLiteralDfa3_0(active0, 0x2000L); + case 100: + if ((active0 & 0x200L) != 0L) { + jjmatchedKind = 9; + jjmatchedPos = 2; + } + break; + case 108: + return jjMoveStringLiteralDfa3_0(active0, 0xc000L); + case 116: + if ((active0 & 0x100L) != 0L) { + jjmatchedKind = 8; + jjmatchedPos = 2; + } + return jjMoveStringLiteralDfa3_0(active0, 0x800L); + case 117: + return jjMoveStringLiteralDfa3_0(active0, 0x2000L); + default: + break; + } + return jjMoveNfa_0(5, 2); + } + + private int jjMoveStringLiteralDfa3_0(long old0, long active0) { + if (((active0 &= old0)) == 0L) + return jjMoveNfa_0(5, 2); + try { + curChar = inputStream.readChar(); + } catch (java.io.IOException e) { + return jjMoveNfa_0(5, 2); + } + switch (curChar) { + case 69: + if ((active0 & 0x2000L) != 0L) { + jjmatchedKind = 13; + jjmatchedPos = 3; + } + break; + case 76: + if ((active0 & 0x8000L) != 0L) { + jjmatchedKind = 15; + jjmatchedPos = 3; + } + break; + case 83: + return jjMoveStringLiteralDfa4_0(active0, 0x4000L); + case 87: + return jjMoveStringLiteralDfa4_0(active0, 0x800L); + case 101: + if ((active0 & 0x2000L) != 0L) { + jjmatchedKind = 13; + jjmatchedPos = 3; + } + break; + case 108: + if ((active0 & 0x8000L) != 0L) { + jjmatchedKind = 15; + jjmatchedPos = 3; + } + break; + case 115: + return jjMoveStringLiteralDfa4_0(active0, 0x4000L); + case 119: + return jjMoveStringLiteralDfa4_0(active0, 0x800L); + default: + break; + } + return jjMoveNfa_0(5, 3); + } + + private int jjMoveStringLiteralDfa4_0(long old0, long active0) { + if (((active0 &= old0)) == 0L) + return jjMoveNfa_0(5, 3); + try { + curChar = inputStream.readChar(); + } catch (java.io.IOException e) { + return jjMoveNfa_0(5, 3); + } + switch (curChar) { + case 69: + if ((active0 & 0x4000L) != 0L) { + jjmatchedKind = 14; + jjmatchedPos = 4; + } + return jjMoveStringLiteralDfa5_0(active0, 0x800L); + case 101: + if ((active0 & 0x4000L) != 0L) { + jjmatchedKind = 14; + jjmatchedPos = 4; + } + return jjMoveStringLiteralDfa5_0(active0, 0x800L); + default: + break; + } + return jjMoveNfa_0(5, 4); + } + + private int jjMoveStringLiteralDfa5_0(long old0, long active0) { + if (((active0 &= old0)) == 0L) + return jjMoveNfa_0(5, 4); + try { + curChar = inputStream.readChar(); + } catch (java.io.IOException e) { + return jjMoveNfa_0(5, 4); + } + switch (curChar) { + case 69: + return jjMoveStringLiteralDfa6_0(active0, 0x800L); + case 101: + return jjMoveStringLiteralDfa6_0(active0, 0x800L); + default: + break; + } + return jjMoveNfa_0(5, 5); + } + + private int jjMoveStringLiteralDfa6_0(long old0, long active0) { + if (((active0 &= old0)) == 0L) + return jjMoveNfa_0(5, 5); + try { + curChar = inputStream.readChar(); + } catch (java.io.IOException e) { + return jjMoveNfa_0(5, 5); + } + switch (curChar) { + case 78: + if ((active0 & 0x800L) != 0L) { + jjmatchedKind = 11; + jjmatchedPos = 6; + } + break; + case 110: + if ((active0 & 0x800L) != 0L) { + jjmatchedKind = 11; + jjmatchedPos = 6; + } + break; + default: + break; + } + return jjMoveNfa_0(5, 6); + } + + static final long[] JJ_BIT_VEC_0 = { + 0xfffffffffffffffeL, 0xffffffffffffffffL, 0xffffffffffffffffL, 0xffffffffffffffffL + }; + static final long[] JJ_BIT_VEC_2 = { + 0x0L, 0x0L, 0xffffffffffffffffL, 0xffffffffffffffffL + }; + + private int jjMoveNfa_0(int startState, int curPos) { + int strKind = jjmatchedKind; + int strPos = jjmatchedPos; + int seenUpto; + inputStream.backup(seenUpto = curPos + 1); + try { + curChar = inputStream.readChar(); + } catch (java.io.IOException e) { + throw new Error("Internal Error"); + } + curPos = 0; + int startsAt = 0; + jjnewStateCnt = 40; + int i = 1; + jjstateSet[0] = startState; + int kind = 0x7fffffff; + for (;;) { + if (++jjround == 0x7fffffff) + ReInitRounds(); + if (curChar < 64) { + long l = 1L << curChar; + do { + switch (jjstateSet[--i]) { + case 5: + if ((0x3ff000000000000L & l) != 0L) + jjCheckNAddStates(0, 3); + else if (curChar == 36) { + if (kind > 21) + kind = 21; + jjCheckNAdd(28); + } else if (curChar == 39) + jjCheckNAddStates(4, 6); + else if (curChar == 46) + jjCheckNAdd(18); + else if (curChar == 47) + jjstateSet[jjnewStateCnt++] = 6; + else if (curChar == 45) + jjstateSet[jjnewStateCnt++] = 0; + if ((0x3fe000000000000L & l) != 0L) { + if (kind > 17) + kind = 17; + jjCheckNAddTwoStates(15, 16); + } else if (curChar == 48) { + if (kind > 17) + kind = 17; + } + break; + case 0: + if (curChar == 45) + jjCheckNAddStates(7, 9); + break; + case 1: + if ((0xffffffffffffdbffL & l) != 0L) + jjCheckNAddStates(7, 9); + break; + case 2: + if ((0x2400L & l) != 0L && kind > 6) + kind = 6; + break; + case 3: + if (curChar == 10 && kind > 6) + kind = 6; + break; + case 4: + if (curChar == 13) + jjstateSet[jjnewStateCnt++] = 3; + break; + case 6: + if (curChar == 42) + jjCheckNAddTwoStates(7, 8); + break; + case 7: + if ((0xfffffbffffffffffL & l) != 0L) + jjCheckNAddTwoStates(7, 8); + break; + case 8: + if (curChar == 42) + jjCheckNAddStates(10, 12); + break; + case 9: + if ((0xffff7bffffffffffL & l) != 0L) + jjCheckNAddTwoStates(10, 8); + break; + case 10: + if ((0xfffffbffffffffffL & l) != 0L) + jjCheckNAddTwoStates(10, 8); + break; + case 11: + if (curChar == 47 && kind > 7) + kind = 7; + break; + case 12: + if (curChar == 47) + jjstateSet[jjnewStateCnt++] = 6; + break; + case 13: + if (curChar == 48 && kind > 17) + kind = 17; + break; + case 14: + if ((0x3fe000000000000L & l) == 0L) + break; + if (kind > 17) + kind = 17; + jjCheckNAddTwoStates(15, 16); + break; + case 15: + if ((0x3ff000000000000L & l) == 0L) + break; + if (kind > 17) + kind = 17; + jjCheckNAddTwoStates(15, 16); + break; + case 17: + if (curChar == 46) + jjCheckNAdd(18); + break; + case 18: + if ((0x3ff000000000000L & l) == 0L) + break; + if (kind > 18) + kind = 18; + jjCheckNAddTwoStates(18, 19); + break; + case 20: + if ((0x280000000000L & l) != 0L) + jjCheckNAdd(21); + break; + case 21: + if ((0x3ff000000000000L & l) == 0L) + break; + if (kind > 18) + kind = 18; + jjCheckNAdd(21); + break; + case 22: + case 23: + if (curChar == 39) + jjCheckNAddStates(4, 6); + break; + case 24: + if (curChar == 39) + jjstateSet[jjnewStateCnt++] = 23; + break; + case 25: + if ((0xffffff7fffffffffL & l) != 0L) + jjCheckNAddStates(4, 6); + break; + case 26: + if (curChar == 39 && kind > 20) + kind = 20; + break; + case 27: + if (curChar != 36) + break; + if (kind > 21) + kind = 21; + jjCheckNAdd(28); + break; + case 28: + if ((0x3ff001000000000L & l) == 0L) + break; + if (kind > 21) + kind = 21; + jjCheckNAdd(28); + break; + case 29: + if ((0x3ff000000000000L & l) != 0L) + jjCheckNAddStates(0, 3); + break; + case 30: + if ((0x3ff000000000000L & l) != 0L) + jjCheckNAddTwoStates(30, 31); + break; + case 31: + if (curChar != 46) + break; + if (kind > 18) + kind = 18; + jjCheckNAddTwoStates(32, 33); + break; + case 32: + if ((0x3ff000000000000L & l) == 0L) + break; + if (kind > 18) + kind = 18; + jjCheckNAddTwoStates(32, 33); + break; + case 34: + if ((0x280000000000L & l) != 0L) + jjCheckNAdd(35); + break; + case 35: + if ((0x3ff000000000000L & l) == 0L) + break; + if (kind > 18) + kind = 18; + jjCheckNAdd(35); + break; + case 36: + if ((0x3ff000000000000L & l) != 0L) + jjCheckNAddTwoStates(36, 37); + break; + case 38: + if ((0x280000000000L & l) != 0L) + jjCheckNAdd(39); + break; + case 39: + if ((0x3ff000000000000L & l) == 0L) + break; + if (kind > 18) + kind = 18; + jjCheckNAdd(39); + break; + default: + break; + } + } while (i != startsAt); + } else if (curChar < 128) { + long l = 1L << (curChar & 077); + do { + switch (jjstateSet[--i]) { + case 5: + case 28: + if ((0x7fffffe87fffffeL & l) == 0L) + break; + if (kind > 21) + kind = 21; + jjCheckNAdd(28); + break; + case 1: + jjAddStates(7, 9); + break; + case 7: + jjCheckNAddTwoStates(7, 8); + break; + case 9: + case 10: + jjCheckNAddTwoStates(10, 8); + break; + case 16: + if ((0x100000001000L & l) != 0L && kind > 17) + kind = 17; + break; + case 19: + if ((0x2000000020L & l) != 0L) + jjAddStates(13, 14); + break; + case 25: + jjAddStates(4, 6); + break; + case 33: + if ((0x2000000020L & l) != 0L) + jjAddStates(15, 16); + break; + case 37: + if ((0x2000000020L & l) != 0L) + jjAddStates(17, 18); + break; + default: + break; + } + } while (i != startsAt); + } else { + int hiByte = (int) (curChar >> 8); + int i1 = hiByte >> 6; + long l1 = 1L << (hiByte & 077); + int i2 = (curChar & 0xff) >> 6; + long l2 = 1L << (curChar & 077); + do { + switch (jjstateSet[--i]) { + case 1: + if (jjCanMove_0(hiByte, i1, i2, l1, l2)) + jjAddStates(7, 9); + break; + case 7: + if (jjCanMove_0(hiByte, i1, i2, l1, l2)) + jjCheckNAddTwoStates(7, 8); + break; + case 9: + case 10: + if (jjCanMove_0(hiByte, i1, i2, l1, l2)) + jjCheckNAddTwoStates(10, 8); + break; + case 25: + if (jjCanMove_0(hiByte, i1, i2, l1, l2)) + jjAddStates(4, 6); + break; + default: + break; + } + } while (i != startsAt); + } + if (kind != 0x7fffffff) { + jjmatchedKind = kind; + jjmatchedPos = curPos; + kind = 0x7fffffff; + } + ++curPos; + if ((i = jjnewStateCnt) == (startsAt = 40 - (jjnewStateCnt = startsAt))) + break; + try { + curChar = inputStream.readChar(); + } catch (java.io.IOException e) { + break; + } + } + if (jjmatchedPos > strPos) + return curPos; + + int toRet = Math.max(curPos, seenUpto); + + if (curPos < toRet) + for (i = toRet - Math.min(curPos, seenUpto); i-- > 0; ) + try { + curChar = inputStream.readChar(); + } catch (java.io.IOException e) { + throw new Error("Internal Error : Please send a bug report."); + } + + if (jjmatchedPos < strPos) { + jjmatchedKind = strKind; + jjmatchedPos = strPos; + } else if (jjmatchedPos == strPos && jjmatchedKind > strKind) + jjmatchedKind = strKind; + + return toRet; + } + + static final int[] JJ_NEXT_STATES = { + 30, 31, 36, 37, 24, 25, 26, 1, 2, 4, 8, 9, 11, 20, 21, 34, + 35, 38, 39, + }; + + private static final boolean jjCanMove_0(int hiByte, int i1, int i2, long l1, long l2) { + switch (hiByte) { + case 0: + return (JJ_BIT_VEC_2[i2] & l2) != 0L; + default: + if ((JJ_BIT_VEC_0[i1] & l1) != 0L) + return true; + return false; + } + } + + /** + * Token literal values. + */ + public static final String[] JJ_STR_LITERAL_IMAGES = { + "", null, null, null, null, null, null, null, null, null, null, null, null, + null, null, null, null, null, null, null, null, null, "\75", "\74\76", "\76", + "\76\75", "\74", "\74\75", "\50", "\54", "\51", "\53", "\55"}; + + /** + * Lexer state names. + */ + public static final String[] LEX_STATE_NAMES = { + "DEFAULT", + }; + static final long[] JJ_TO_TOKEN = { + 0x1fff7ff01L, + }; + static final long[] JJ_TO_SKIP = { + 0xfeL, + }; + static final long[] JJ_TO_SPECIAL = { + 0x3eL, + }; + protected SimpleCharStream inputStream; + private final int[] jjrounds = new int[40]; + private final int[] jjstateSet = new int[80]; + protected char curChar; + + /** + * Constructor. + */ + public SelectorParserTokenManager(SimpleCharStream stream) { + if (SimpleCharStream.STATIC_FLAG) + throw new Error("ERROR: Cannot use a static CharStream class with a non-static lexical analyzer."); + inputStream = stream; + } + + /** + * Constructor. + */ + public SelectorParserTokenManager(SimpleCharStream stream, int lexState) { + this(stream); + SwitchTo(lexState); + } + + /** + * Reinitialise parser. + */ + public void ReInit(SimpleCharStream stream) { + jjmatchedPos = jjnewStateCnt = 0; + curLexState = defaultLexState; + inputStream = stream; + ReInitRounds(); + } + + private void ReInitRounds() { + int i; + jjround = 0x80000001; + for (i = 40; i-- > 0; ) + jjrounds[i] = 0x80000000; + } + + /** + * Reinitialise parser. + */ + public void ReInit(SimpleCharStream stream, int lexState) { + ReInit(stream); + SwitchTo(lexState); + } + + /** + * Switch to specified lex state. + */ + public void SwitchTo(int lexState) { + if (lexState >= 1 || lexState < 0) + throw new TokenMgrError("Error: Ignoring invalid lexical state : " + lexState + ". State unchanged.", + TokenMgrError.INVALID_LEXICAL_STATE); + else + curLexState = lexState; + } + + protected Token jjFillToken() { + final Token t; + final String curTokenImage; + final int beginLine; + final int endLine; + final int beginColumn; + final int endColumn; + String im = JJ_STR_LITERAL_IMAGES[jjmatchedKind]; + curTokenImage = (im == null) ? inputStream.GetImage() : im; + beginLine = inputStream.getBeginLine(); + beginColumn = inputStream.getBeginColumn(); + endLine = inputStream.getEndLine(); + endColumn = inputStream.getEndColumn(); + t = Token.newToken(jjmatchedKind, curTokenImage); + + t.beginLine = beginLine; + t.endLine = endLine; + t.beginColumn = beginColumn; + t.endColumn = endColumn; + + return t; + } + + int curLexState = 0; + int defaultLexState = 0; + int jjnewStateCnt; + int jjround; + int jjmatchedPos; + int jjmatchedKind; + + /** + * Get the next Token. + */ + public Token getNextToken() { + Token specialToken = null; + Token matchedToken; + int curPos = 0; + + EOFLoop: + for (;;) { + try { + curChar = inputStream.BeginToken(); + } catch (java.io.IOException e) { + jjmatchedKind = 0; + matchedToken = jjFillToken(); + matchedToken.specialToken = specialToken; + return matchedToken; + } + + jjmatchedKind = 0x7fffffff; + jjmatchedPos = 0; + curPos = jjMoveStringLiteralDfa0_0(); + if (jjmatchedKind != 0x7fffffff) { + if (jjmatchedPos + 1 < curPos) + inputStream.backup(curPos - jjmatchedPos - 1); + if ((JJ_TO_TOKEN[jjmatchedKind >> 6] & (1L << (jjmatchedKind & 077))) != 0L) { + matchedToken = jjFillToken(); + matchedToken.specialToken = specialToken; + return matchedToken; + } else { + if ((JJ_TO_SPECIAL[jjmatchedKind >> 6] & (1L << (jjmatchedKind & 077))) != 0L) { + matchedToken = jjFillToken(); + if (specialToken == null) + specialToken = matchedToken; + else { + matchedToken.specialToken = specialToken; + specialToken = specialToken.next = matchedToken; + } + } + continue EOFLoop; + } + } + int errorLine = inputStream.getEndLine(); + int errorColumn = inputStream.getEndColumn(); + String errorAfter = null; + boolean eofSeen = false; + try { + inputStream.readChar(); + inputStream.backup(1); + } catch (java.io.IOException e1) { + eofSeen = true; + errorAfter = curPos <= 1 ? "" : inputStream.GetImage(); + if (curChar == '\n' || curChar == '\r') { + errorLine++; + errorColumn = 0; + } else + errorColumn++; + } + if (!eofSeen) { + inputStream.backup(1); + errorAfter = curPos <= 1 ? "" : inputStream.GetImage(); + } + throw new TokenMgrError(eofSeen, curLexState, errorLine, errorColumn, errorAfter, curChar, + TokenMgrError.LEXICAL_ERROR); + } + } + + private void jjCheckNAdd(int state) { + if (jjrounds[state] != jjround) { + jjstateSet[jjnewStateCnt++] = state; + jjrounds[state] = jjround; + } + } + + private void jjAddStates(int start, int end) { + do { + jjstateSet[jjnewStateCnt++] = JJ_NEXT_STATES[start]; + } while (start++ != end); + } + + private void jjCheckNAddTwoStates(int state1, int state2) { + jjCheckNAdd(state1); + jjCheckNAdd(state2); + } + + private void jjCheckNAddStates(int start, int end) { + do { + jjCheckNAdd(JJ_NEXT_STATES[start]); + } while (start++ != end); + } + +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/parser/SimpleCharStream.java b/filter/src/main/java/org/apache/rocketmq/filter/parser/SimpleCharStream.java new file mode 100644 index 00000000..94a54b41 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/parser/SimpleCharStream.java @@ -0,0 +1,502 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* Generated By:JavaCC: Do not edit this line. SimpleCharStream.java Version 5.0 */ +/* JavaCCOptions:STATIC=false,SUPPORT_CLASS_VISIBILITY_PUBLIC=true */ +package org.apache.rocketmq.filter.parser; + +/** + * An implementation of interface CharStream, where the stream is assumed to + * contain only ASCII characters (without unicode processing). + */ + +public class SimpleCharStream { + /** + * Whether parser is static. + */ + public static final boolean STATIC_FLAG = false; + int bufsize; + int available; + int tokenBegin; + /** + * Position in buffer. + */ + public int bufpos = -1; + protected int bufline[]; + protected int bufcolumn[]; + + protected int column = 0; + protected int line = 1; + + protected boolean prevCharIsCR = false; + protected boolean prevCharIsLF = false; + + protected java.io.Reader inputStream; + + protected char[] buffer; + protected int maxNextCharInd = 0; + protected int inBuf = 0; + protected int tabSize = 8; + + protected void setTabSize(int i) { + tabSize = i; + } + + protected int getTabSize(int i) { + return tabSize; + } + + protected void ExpandBuff(boolean wrapAround) { + char[] newbuffer = new char[bufsize + 2048]; + int newbufline[] = new int[bufsize + 2048]; + int newbufcolumn[] = new int[bufsize + 2048]; + + try { + if (wrapAround) { + System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin); + System.arraycopy(buffer, 0, newbuffer, bufsize - tokenBegin, bufpos); + buffer = newbuffer; + + System.arraycopy(bufline, tokenBegin, newbufline, 0, bufsize - tokenBegin); + System.arraycopy(bufline, 0, newbufline, bufsize - tokenBegin, bufpos); + bufline = newbufline; + + System.arraycopy(bufcolumn, tokenBegin, newbufcolumn, 0, bufsize - tokenBegin); + System.arraycopy(bufcolumn, 0, newbufcolumn, bufsize - tokenBegin, bufpos); + bufcolumn = newbufcolumn; + + maxNextCharInd = bufpos += bufsize - tokenBegin; + } else { + System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin); + buffer = newbuffer; + + System.arraycopy(bufline, tokenBegin, newbufline, 0, bufsize - tokenBegin); + bufline = newbufline; + + System.arraycopy(bufcolumn, tokenBegin, newbufcolumn, 0, bufsize - tokenBegin); + bufcolumn = newbufcolumn; + + maxNextCharInd = bufpos -= tokenBegin; + } + } catch (Throwable t) { + throw new Error(t.getMessage()); + } + + bufsize += 2048; + available = bufsize; + tokenBegin = 0; + } + + protected void FillBuff() throws java.io.IOException { + if (maxNextCharInd == available) { + if (available == bufsize) { + if (tokenBegin > 2048) { + bufpos = maxNextCharInd = 0; + available = tokenBegin; + } else if (tokenBegin < 0) + bufpos = maxNextCharInd = 0; + else + ExpandBuff(false); + } else if (available > tokenBegin) + available = bufsize; + else if ((tokenBegin - available) < 2048) + ExpandBuff(true); + else + available = tokenBegin; + } + + int i; + try { + if ((i = inputStream.read(buffer, maxNextCharInd, available - maxNextCharInd)) == -1) { + inputStream.close(); + throw new java.io.IOException(); + } else + maxNextCharInd += i; + return; + } catch (java.io.IOException e) { + --bufpos; + backup(0); + if (tokenBegin == -1) + tokenBegin = bufpos; + throw e; + } + } + + /** + * Start. + */ + public char BeginToken() throws java.io.IOException { + tokenBegin = -1; + char c = readChar(); + tokenBegin = bufpos; + + return c; + } + + protected void UpdateLineColumn(char c) { + column++; + + if (prevCharIsLF) { + prevCharIsLF = false; + line += column = 1; + } else if (prevCharIsCR) { + prevCharIsCR = false; + if (c == '\n') { + prevCharIsLF = true; + } else + line += column = 1; + } + + switch (c) { + case '\r': + prevCharIsCR = true; + break; + case '\n': + prevCharIsLF = true; + break; + case '\t': + column--; + column += tabSize - (column % tabSize); + break; + default: + break; + } + + bufline[bufpos] = line; + bufcolumn[bufpos] = column; + } + + /** + * Read a character. + */ + public char readChar() throws java.io.IOException { + if (inBuf > 0) { + --inBuf; + + if (++bufpos == bufsize) + bufpos = 0; + + return buffer[bufpos]; + } + + if (++bufpos >= maxNextCharInd) + FillBuff(); + + char c = buffer[bufpos]; + + UpdateLineColumn(c); + return c; + } + + @Deprecated + /** + * @deprecated + * @see #getEndColumn + */ + + public int getColumn() { + return bufcolumn[bufpos]; + } + + @Deprecated + /** + * @deprecated + * @see #getEndLine + */ + + public int getLine() { + return bufline[bufpos]; + } + + /** + * Get token end column number. + */ + public int getEndColumn() { + return bufcolumn[bufpos]; + } + + /** + * Get token end line number. + */ + public int getEndLine() { + return bufline[bufpos]; + } + + /** + * Get token beginning column number. + */ + public int getBeginColumn() { + return bufcolumn[tokenBegin]; + } + + /** + * Get token beginning line number. + */ + public int getBeginLine() { + return bufline[tokenBegin]; + } + + /** + * Backup a number of characters. + */ + public void backup(int amount) { + + inBuf += amount; + if ((bufpos -= amount) < 0) + bufpos += bufsize; + } + + /** + * Constructor. + */ + public SimpleCharStream(java.io.Reader dstream, int startline, + int startcolumn, int buffersize) { + inputStream = dstream; + line = startline; + column = startcolumn - 1; + + available = bufsize = buffersize; + buffer = new char[buffersize]; + bufline = new int[buffersize]; + bufcolumn = new int[buffersize]; + } + + /** + * Constructor. + */ + public SimpleCharStream(java.io.Reader dstream, int startline, + int startcolumn) { + this(dstream, startline, startcolumn, 4096); + } + + /** + * Constructor. + */ + public SimpleCharStream(java.io.Reader dstream) { + this(dstream, 1, 1, 4096); + } + + /** + * Reinitialise. + */ + public void ReInit(java.io.Reader dstream, int startline, + int startcolumn, int buffersize) { + inputStream = dstream; + line = startline; + column = startcolumn - 1; + + if (buffer == null || buffersize != buffer.length) { + available = bufsize = buffersize; + buffer = new char[buffersize]; + bufline = new int[buffersize]; + bufcolumn = new int[buffersize]; + } + prevCharIsLF = prevCharIsCR = false; + tokenBegin = inBuf = maxNextCharInd = 0; + bufpos = -1; + } + + /** + * Reinitialise. + */ + public void ReInit(java.io.Reader dstream, int startline, + int startcolumn) { + ReInit(dstream, startline, startcolumn, 4096); + } + + /** + * Reinitialise. + */ + public void ReInit(java.io.Reader dstream) { + ReInit(dstream, 1, 1, 4096); + } + + /** + * Constructor. + */ + public SimpleCharStream(java.io.InputStream dstream, String encoding, int startline, + int startcolumn, int buffersize) throws java.io.UnsupportedEncodingException { + this(encoding == null ? + new java.io.InputStreamReader(dstream) : + new java.io.InputStreamReader(dstream, encoding), startline, startcolumn, buffersize); + } + + /** + * Constructor. + */ + public SimpleCharStream(java.io.InputStream dstream, int startline, + int startcolumn, int buffersize) { + this(new java.io.InputStreamReader(dstream), startline, startcolumn, buffersize); + } + + /** + * Constructor. + */ + public SimpleCharStream(java.io.InputStream dstream, String encoding, int startline, + int startcolumn) throws java.io.UnsupportedEncodingException { + this(dstream, encoding, startline, startcolumn, 4096); + } + + /** + * Constructor. + */ + public SimpleCharStream(java.io.InputStream dstream, int startline, + int startcolumn) { + this(dstream, startline, startcolumn, 4096); + } + + /** + * Constructor. + */ + public SimpleCharStream(java.io.InputStream dstream, String encoding) throws java.io.UnsupportedEncodingException { + this(dstream, encoding, 1, 1, 4096); + } + + /** + * Constructor. + */ + public SimpleCharStream(java.io.InputStream dstream) { + this(dstream, 1, 1, 4096); + } + + /** + * Reinitialise. + */ + public void ReInit(java.io.InputStream dstream, String encoding, int startline, + int startcolumn, int buffersize) throws java.io.UnsupportedEncodingException { + ReInit(encoding == null ? + new java.io.InputStreamReader(dstream) : + new java.io.InputStreamReader(dstream, encoding), startline, startcolumn, buffersize); + } + + /** + * Reinitialise. + */ + public void ReInit(java.io.InputStream dstream, int startline, + int startcolumn, int buffersize) { + ReInit(new java.io.InputStreamReader(dstream), startline, startcolumn, buffersize); + } + + /** + * Reinitialise. + */ + public void ReInit(java.io.InputStream dstream, String encoding) throws java.io.UnsupportedEncodingException { + ReInit(dstream, encoding, 1, 1, 4096); + } + + /** + * Reinitialise. + */ + public void ReInit(java.io.InputStream dstream) { + ReInit(dstream, 1, 1, 4096); + } + + /** + * Reinitialise. + */ + public void ReInit(java.io.InputStream dstream, String encoding, int startline, + int startcolumn) throws java.io.UnsupportedEncodingException { + ReInit(dstream, encoding, startline, startcolumn, 4096); + } + + /** + * Reinitialise. + */ + public void ReInit(java.io.InputStream dstream, int startline, + int startcolumn) { + ReInit(dstream, startline, startcolumn, 4096); + } + + /** + * Get token literal value. + */ + public String GetImage() { + if (bufpos >= tokenBegin) + return new String(buffer, tokenBegin, bufpos - tokenBegin + 1); + else + return new String(buffer, tokenBegin, bufsize - tokenBegin) + + new String(buffer, 0, bufpos + 1); + } + + /** + * Get the suffix. + */ + public char[] GetSuffix(int len) { + char[] ret = new char[len]; + + if ((bufpos + 1) >= len) + System.arraycopy(buffer, bufpos - len + 1, ret, 0, len); + else { + System.arraycopy(buffer, bufsize - (len - bufpos - 1), ret, 0, + len - bufpos - 1); + System.arraycopy(buffer, 0, ret, len - bufpos - 1, bufpos + 1); + } + + return ret; + } + + /** + * Reset buffer when finished. + */ + public void Done() { + buffer = null; + bufline = null; + bufcolumn = null; + } + + /** + * Method to adjust line and column numbers for the start of a token. + */ + public void adjustBeginLineColumn(int newLine, int newCol) { + int start = tokenBegin; + int len; + + if (bufpos >= tokenBegin) { + len = bufpos - tokenBegin + inBuf + 1; + } else { + len = bufsize - tokenBegin + bufpos + 1 + inBuf; + } + + int i = 0, j = 0, k = 0; + int nextColDiff = 0, columnDiff = 0; + + while (i < len && bufline[j = start % bufsize] == bufline[k = ++start % bufsize]) { + bufline[j] = newLine; + nextColDiff = columnDiff + bufcolumn[k] - bufcolumn[j]; + bufcolumn[j] = newCol + columnDiff; + columnDiff = nextColDiff; + i++; + } + + if (i < len) { + bufline[j] = newLine++; + bufcolumn[j] = newCol + columnDiff; + + while (i++ < len) { + if (bufline[j = start % bufsize] != bufline[++start % bufsize]) + bufline[j] = newLine++; + else + bufline[j] = newLine; + } + } + + line = bufline[j]; + column = bufcolumn[j]; + } + +} +/* JavaCC - OriginalChecksum=af79bfe4b18b4b4ea9720ffeb7e52fc5 (do not edit this line) */ diff --git a/filter/src/main/java/org/apache/rocketmq/filter/parser/Token.java b/filter/src/main/java/org/apache/rocketmq/filter/parser/Token.java new file mode 100644 index 00000000..8e6a48a0 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/parser/Token.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* Generated By:JavaCC: Do not edit this line. Token.java Version 5.0 */ +/* JavaCCOptions:TOKEN_EXTENDS=,KEEP_LINE_COL=null,SUPPORT_CLASS_VISIBILITY_PUBLIC=true */ +package org.apache.rocketmq.filter.parser; + +/** + * Describes the input token stream. + */ + +public class Token implements java.io.Serializable { + + /** + * The version identifier for this Serializable class. + * Increment only if the serialized form of the + * class changes. + */ + private static final long serialVersionUID = 1L; + + /** + * An integer that describes the kind of this token. This numbering + * system is determined by JavaCCParser, and a table of these numbers is + * stored in the file ...Constants.java. + */ + public int kind; + + /** + * The line number of the first character of this Token. + */ + public int beginLine; + /** + * The column number of the first character of this Token. + */ + public int beginColumn; + /** + * The line number of the last character of this Token. + */ + public int endLine; + /** + * The column number of the last character of this Token. + */ + public int endColumn; + + /** + * The string image of the token. + */ + public String image; + + /** + * A reference to the next regular (non-special) token from the input + * stream. If this is the last token from the input stream, or if the + * token manager has not read tokens beyond this one, this field is + * set to null. This is true only if this token is also a regular + * token. Otherwise, see below for a description of the contents of + * this field. + */ + public Token next; + + /** + * This field is used to access special tokens that occur prior to this + * token, but after the immediately preceding regular (non-special) token. + * If there are no such special tokens, this field is set to null. + * When there are more than one such special token, this field refers + * to the last of these special tokens, which in turn refers to the next + * previous special token through its specialToken field, and so on + * until the first special token (whose specialToken field is null). + * The next fields of special tokens refer to other special tokens that + * immediately follow it (without an intervening regular token). If there + * is no such token, this field is null. + */ + public Token specialToken; + + /** + * An optional attribute value of the Token. + * Tokens which are not used as syntactic sugar will often contain + * meaningful values that will be used later on by the compiler or + * interpreter. This attribute value is often different from the image. + * Any subclass of Token that actually wants to return a non-null value can + * override this method as appropriate. + */ + public Object getValue() { + return null; + } + + /** + * No-argument constructor + */ + public Token() { + } + + /** + * Constructs a new token for the specified Image. + */ + public Token(int kind) { + this(kind, null); + } + + /** + * Constructs a new token for the specified Image and Kind. + */ + public Token(int kind, String image) { + this.kind = kind; + this.image = image; + } + + /** + * Returns the image. + */ + public String toString() { + return image; + } + + /** + * Returns a new Token object, by default. However, if you want, you + * can create and return subclass objects based on the value of ofKind. + * Simply add the cases to the switch for all those special cases. + * For example, if you have a subclass of Token called IDToken that + * you want to create if ofKind is ID, simply add something like : + *

    + * case MyParserConstants.ID : return new IDToken(ofKind, image); + *

    + * to the following switch statement. Then you can cast matchedToken + * variable to the appropriate type and use sit in your lexical actions. + */ + public static Token newToken(int ofKind, String image) { + switch (ofKind) { + default: + return new Token(ofKind, image); + } + } + + public static Token newToken(int ofKind) { + return newToken(ofKind, null); + } + +} +/* JavaCC - OriginalChecksum=6b0af88eb45a551d929d3cdd9582f827 (do not edit this line) */ diff --git a/filter/src/main/java/org/apache/rocketmq/filter/parser/TokenMgrError.java b/filter/src/main/java/org/apache/rocketmq/filter/parser/TokenMgrError.java new file mode 100644 index 00000000..75d83e5f --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/parser/TokenMgrError.java @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* Generated By:JavaCC: Do not edit this line. TokenMgrError.java Version 5.0 */ +/* JavaCCOptions: */ +package org.apache.rocketmq.filter.parser; + +/** + * Token Manager Error. + */ +public class TokenMgrError extends Error { + + /** + * The version identifier for this Serializable class. + * Increment only if the serialized form of the + * class changes. + */ + private static final long serialVersionUID = 1L; + + /* + * Ordinals for various reasons why an Error of this type can be thrown. + */ + + /** + * Lexical error occurred. + */ + static final int LEXICAL_ERROR = 0; + + /** + * An attempt was made to create a second instance of a static token manager. + */ + static final int STATIC_LEXER_ERROR = 1; + + /** + * Tried to change to an invalid lexical state. + */ + static final int INVALID_LEXICAL_STATE = 2; + + /** + * Detected (and bailed out of) an infinite loop in the token manager. + */ + static final int LOOP_DETECTED = 3; + + /** + * Indicates the reason why the exception is thrown. It will have + * one of the above 4 values. + */ + int errorCode; + + /** + * Replaces unprintable characters by their escaped (or unicode escaped) + * equivalents in the given string + */ + protected static final String addEscapes(String str) { + StringBuffer retval = new StringBuffer(); + char ch; + for (int i = 0; i < str.length(); i++) { + switch (str.charAt(i)) { + case 0: + continue; + case '\b': + retval.append("\\b"); + continue; + case '\t': + retval.append("\\t"); + continue; + case '\n': + retval.append("\\n"); + continue; + case '\f': + retval.append("\\f"); + continue; + case '\r': + retval.append("\\r"); + continue; + case '\"': + retval.append("\\\""); + continue; + case '\'': + retval.append("\\\'"); + continue; + case '\\': + retval.append("\\\\"); + continue; + default: + if ((ch = str.charAt(i)) < 0x20 || ch > 0x7e) { + String s = "0000" + Integer.toString(ch, 16); + retval.append("\\u" + s.substring(s.length() - 4, s.length())); + } else { + retval.append(ch); + } + continue; + } + } + return retval.toString(); + } + + /** + * Returns a detailed message for the Error when it is thrown by the + * token manager to indicate a lexical error. + * Parameters : + * eofSeen : indicates if EOF caused the lexical error + * curLexState : lexical state in which this error occurred + * errorLine : line number when the error occurred + * errorColumn : column number when the error occurred + * errorAfter : prefix that was seen before this error occurred + * curchar : the offending character + * Note: You can customize the lexical error message by modifying this method. + */ + protected static String LexicalError(boolean eofSeen, int lexState, int errorLine, int errorColumn, + String errorAfter, char curChar) { + return "Lexical error at line " + + errorLine + ", column " + + errorColumn + ". Encountered: " + + (eofSeen ? + " " : + ("\"" + addEscapes(String.valueOf(curChar)) + "\"") + " (" + (int) curChar + "), ") + + "after : \"" + addEscapes(errorAfter) + "\""; + } + + /** + * You can also modify the body of this method to customize your error messages. + * For example, cases like LOOP_DETECTED and INVALID_LEXICAL_STATE are not + * of end-users concern, so you can return something like : + *

    + * "Internal Error : Please file a bug report .... " + *

    + * from this method for such cases in the release version of your parser. + */ + public String getMessage() { + return super.getMessage(); + } + + /* + * Constructors of various flavors follow. + */ + + /** + * No arg constructor. + */ + public TokenMgrError() { + } + + /** + * Constructor with message and reason. + */ + public TokenMgrError(String message, int reason) { + super(message); + errorCode = reason; + } + + /** + * Full Constructor. + */ + public TokenMgrError(boolean eofSeen, int lexState, int errorLine, int errorColumn, String errorAfter, char curChar, + int reason) { + this(LexicalError(eofSeen, lexState, errorLine, errorColumn, errorAfter, curChar), reason); + } +} +/* JavaCC - OriginalChecksum=e960778c8dcd73e167ed5bfddd59f288 (do not edit this line) */ diff --git a/filter/src/main/java/org/apache/rocketmq/filter/util/BitsArray.java b/filter/src/main/java/org/apache/rocketmq/filter/util/BitsArray.java new file mode 100644 index 00000000..98668549 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/util/BitsArray.java @@ -0,0 +1,260 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.util; + +/** + * Wrapper of bytes array, in order to operate single bit easily. + */ +public class BitsArray implements Cloneable { + + private byte[] bytes; + private int bitLength; + + public static BitsArray create(int bitLength) { + return new BitsArray(bitLength); + } + + public static BitsArray create(byte[] bytes, int bitLength) { + return new BitsArray(bytes, bitLength); + } + + public static BitsArray create(byte[] bytes) { + return new BitsArray(bytes); + } + + private BitsArray(int bitLength) { + this.bitLength = bitLength; + // init bytes + int temp = bitLength / Byte.SIZE; + if (bitLength % Byte.SIZE > 0) { + temp++; + } + bytes = new byte[temp]; + for (int i = 0; i < bytes.length; i++) { + bytes[i] = (byte) 0x00; + } + } + + private BitsArray(byte[] bytes, int bitLength) { + if (bytes == null || bytes.length < 1) { + throw new IllegalArgumentException("Bytes is empty!"); + } + + if (bitLength < 1) { + throw new IllegalArgumentException("Bit is less than 1."); + } + + if (bitLength < bytes.length * Byte.SIZE) { + throw new IllegalArgumentException("BitLength is less than bytes.length() * " + Byte.SIZE); + } + + this.bytes = new byte[bytes.length]; + System.arraycopy(bytes, 0, this.bytes, 0, this.bytes.length); + this.bitLength = bitLength; + } + + private BitsArray(byte[] bytes) { + if (bytes == null || bytes.length < 1) { + throw new IllegalArgumentException("Bytes is empty!"); + } + + this.bitLength = bytes.length * Byte.SIZE; + this.bytes = new byte[bytes.length]; + System.arraycopy(bytes, 0, this.bytes, 0, this.bytes.length); + } + + public int bitLength() { + return this.bitLength; + } + + public int byteLength() { + return this.bytes.length; + } + + public byte[] bytes() { + return this.bytes; + } + + public void xor(final BitsArray other) { + checkInitialized(this); + checkInitialized(other); + + int minByteLength = Math.min(this.byteLength(), other.byteLength()); + + for (int i = 0; i < minByteLength; i++) { + this.bytes[i] = (byte) (this.bytes[i] ^ other.getByte(i)); + } + } + + public void xor(int bitPos, boolean set) { + checkBitPosition(bitPos, this); + + boolean value = getBit(bitPos); + if (value ^ set) { + setBit(bitPos, true); + } else { + setBit(bitPos, false); + } + } + + public void or(final BitsArray other) { + checkInitialized(this); + checkInitialized(other); + + int minByteLength = Math.min(this.byteLength(), other.byteLength()); + + for (int i = 0; i < minByteLength; i++) { + this.bytes[i] = (byte) (this.bytes[i] | other.getByte(i)); + } + } + + public void or(int bitPos, boolean set) { + checkBitPosition(bitPos, this); + + if (set) { + setBit(bitPos, true); + } + } + + public void and(final BitsArray other) { + checkInitialized(this); + checkInitialized(other); + + int minByteLength = Math.min(this.byteLength(), other.byteLength()); + + for (int i = 0; i < minByteLength; i++) { + this.bytes[i] = (byte) (this.bytes[i] & other.getByte(i)); + } + } + + public void and(int bitPos, boolean set) { + checkBitPosition(bitPos, this); + + if (!set) { + setBit(bitPos, false); + } + } + + public void not(int bitPos) { + checkBitPosition(bitPos, this); + + setBit(bitPos, !getBit(bitPos)); + } + + public void setBit(int bitPos, boolean set) { + checkBitPosition(bitPos, this); + int sub = subscript(bitPos); + int pos = position(bitPos); + if (set) { + this.bytes[sub] = (byte) (this.bytes[sub] | pos); + } else { + this.bytes[sub] = (byte) (this.bytes[sub] & ~pos); + } + } + + public void setByte(int bytePos, byte set) { + checkBytePosition(bytePos, this); + + this.bytes[bytePos] = set; + } + + public boolean getBit(int bitPos) { + checkBitPosition(bitPos, this); + + return (this.bytes[subscript(bitPos)] & position(bitPos)) != 0; + } + + public byte getByte(int bytePos) { + checkBytePosition(bytePos, this); + + return this.bytes[bytePos]; + } + + protected int subscript(int bitPos) { + return bitPos / Byte.SIZE; + } + + protected int position(int bitPos) { + return 1 << bitPos % Byte.SIZE; + } + + protected void checkBytePosition(int bytePos, BitsArray bitsArray) { + checkInitialized(bitsArray); + if (bytePos > bitsArray.byteLength()) { + throw new IllegalArgumentException("BytePos is greater than " + bytes.length); + } + if (bytePos < 0) { + throw new IllegalArgumentException("BytePos is less than 0"); + } + } + + protected void checkBitPosition(int bitPos, BitsArray bitsArray) { + checkInitialized(bitsArray); + if (bitPos > bitsArray.bitLength()) { + throw new IllegalArgumentException("BitPos is greater than " + bitLength); + } + if (bitPos < 0) { + throw new IllegalArgumentException("BitPos is less than 0"); + } + } + + protected void checkInitialized(BitsArray bitsArray) { + if (bitsArray.bytes() == null) { + throw new RuntimeException("Not initialized!"); + } + } + + public BitsArray clone() { + byte[] clone = new byte[this.byteLength()]; + + System.arraycopy(this.bytes, 0, clone, 0, this.byteLength()); + + return create(clone, bitLength()); + } + + @Override + public String toString() { + if (this.bytes == null) { + return "null"; + } + StringBuilder stringBuilder = new StringBuilder(this.bytes.length * Byte.SIZE); + for (int i = this.bytes.length - 1; i >= 0; i--) { + + int j = Byte.SIZE - 1; + if (i == this.bytes.length - 1 && this.bitLength % Byte.SIZE > 0) { + // not full byte + j = this.bitLength % Byte.SIZE; + } + + for (; j >= 0; j--) { + + byte mask = (byte) (1 << j); + if ((this.bytes[i] & mask) == mask) { + stringBuilder.append("1"); + } else { + stringBuilder.append("0"); + } + } + if (i % 8 == 0) { + stringBuilder.append("\n"); + } + } + + return stringBuilder.toString(); + } +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/util/BloomFilter.java b/filter/src/main/java/org/apache/rocketmq/filter/util/BloomFilter.java new file mode 100644 index 00000000..f6109062 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/util/BloomFilter.java @@ -0,0 +1,338 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.util; + +import com.google.common.hash.Hashing; + +import java.nio.charset.Charset; + +/** + * Simple implement of bloom filter. + */ +public class BloomFilter { + + public static final Charset UTF_8 = Charset.forName("UTF-8"); + + // as error rate, 10/100 = 0.1 + private int f = 10; + private int n = 128; + + // hash function num, by calculation. + private int k; + // bit count, by calculation. + private int m; + + /** + * Create bloom filter by error rate and mapping num. + * + * @param f error rate + * @param n num will mapping to bit + * @return + */ + public static BloomFilter createByFn(int f, int n) { + return new BloomFilter(f, n); + } + + /** + * Constructor. + * + * @param f error rate + * @param n num will mapping to bit + */ + private BloomFilter(int f, int n) { + if (f < 1 || f >= 100) { + throw new IllegalArgumentException("f must be greater or equal than 1 and less than 100"); + } + if (n < 1) { + throw new IllegalArgumentException("n must be greater than 0"); + } + + this.f = f; + this.n = n; + + // set p = e^(-kn/m) + // f = (1 - p)^k = e^(kln(1-p)) + // when p = 0.5, k = ln2 * (m/n), f = (1/2)^k = (0.618)^(m/n) + double errorRate = f / 100.0; + this.k = (int) Math.ceil(logMN(0.5, errorRate)); + + if (this.k < 1) { + throw new IllegalArgumentException("Hash function num is less than 1, maybe you should change the value of error rate or bit num!"); + } + + // m >= n*log2(1/f)*log2(e) + this.m = (int) Math.ceil(this.n * logMN(2, 1 / errorRate) * logMN(2, Math.E)); + // m%8 = 0 + this.m = (int) (Byte.SIZE * Math.ceil(this.m / (Byte.SIZE * 1.0))); + } + + /** + * Calculate bit positions of {@code str}. + *

    + * See "Less Hashing, Same Performance: Building a Better Bloom Filter" by Adam Kirsch and Michael + * Mitzenmacher. + *

    + * + * @param str + * @return + */ + public int[] calcBitPositions(String str) { + int[] bitPositions = new int[this.k]; + + long hash64 = Hashing.murmur3_128().hashString(str, UTF_8).asLong(); + + int hash1 = (int) hash64; + int hash2 = (int) (hash64 >>> 32); + + for (int i = 1; i <= this.k; i++) { + int combinedHash = hash1 + (i * hash2); + // Flip all the bits if it's negative (guaranteed positive number) + if (combinedHash < 0) { + combinedHash = ~combinedHash; + } + bitPositions[i - 1] = combinedHash % this.m; + } + + return bitPositions; + } + + /** + * Calculate bit positions of {@code str} to construct {@code BloomFilterData} + * + * @param str + * @return + */ + public BloomFilterData generate(String str) { + int[] bitPositions = calcBitPositions(str); + + return new BloomFilterData(bitPositions, this.m); + } + + /** + * Calculate bit positions of {@code str}, then set the related {@code bits} positions to 1. + * + * @param str + * @param bits + */ + public void hashTo(String str, BitsArray bits) { + hashTo(calcBitPositions(str), bits); + } + + /** + * Set the related {@code bits} positions to 1. + * + * @param bitPositions + * @param bits + */ + public void hashTo(int[] bitPositions, BitsArray bits) { + check(bits); + + for (int i : bitPositions) { + bits.setBit(i, true); + } + } + + /** + * Extra check: + *
  • 1. check {@code filterData} belong to this bloom filter.
  • + *

    + * Then set the related {@code bits} positions to 1. + *

    + * + * @param filterData + * @param bits + */ + public void hashTo(BloomFilterData filterData, BitsArray bits) { + if (!isValid(filterData)) { + throw new IllegalArgumentException( + String.format("Bloom filter data may not belong to this filter! %s, %s", + filterData, this.toString()) + ); + } + hashTo(filterData.getBitPos(), bits); + } + + /** + * Calculate bit positions of {@code str}, then check all the related {@code bits} positions is 1. + * + * @param str + * @param bits + * @return true: all the related {@code bits} positions is 1 + */ + public boolean isHit(String str, BitsArray bits) { + return isHit(calcBitPositions(str), bits); + } + + /** + * Check all the related {@code bits} positions is 1. + * + * @param bitPositions + * @param bits + * @return true: all the related {@code bits} positions is 1 + */ + public boolean isHit(int[] bitPositions, BitsArray bits) { + check(bits); + boolean ret = bits.getBit(bitPositions[0]); + for (int i = 1; i < bitPositions.length; i++) { + ret &= bits.getBit(bitPositions[i]); + } + return ret; + } + + /** + * Check all the related {@code bits} positions is 1. + * + * @param filterData + * @param bits + * @return true: all the related {@code bits} positions is 1 + */ + public boolean isHit(BloomFilterData filterData, BitsArray bits) { + if (!isValid(filterData)) { + throw new IllegalArgumentException( + String.format("Bloom filter data may not belong to this filter! %s, %s", + filterData, this.toString()) + ); + } + return isHit(filterData.getBitPos(), bits); + } + + /** + * Check whether one of {@code bitPositions} has been occupied. + * + * @param bitPositions + * @param bits + * @return true: if all positions have been occupied. + */ + public boolean checkFalseHit(int[] bitPositions, BitsArray bits) { + for (int j = 0; j < bitPositions.length; j++) { + int pos = bitPositions[j]; + + // check position of bits has been set. + // that mean no one occupy the position. + if (!bits.getBit(pos)) { + return false; + } + } + + return true; + } + + protected void check(BitsArray bits) { + if (bits.bitLength() != this.m) { + throw new IllegalArgumentException( + String.format("Length(%d) of bits in BitsArray is not equal to %d!", bits.bitLength(), this.m) + ); + } + } + + /** + * Check {@code BloomFilterData} is valid, and belong to this bloom filter. + *
  • 1. not null
  • + *
  • 2. {@link org.apache.rocketmq.filter.util.BloomFilterData#getBitNum} must be equal to {@code m}
  • + *
  • 3. {@link org.apache.rocketmq.filter.util.BloomFilterData#getBitPos} is not null
  • + *
  • 4. {@link org.apache.rocketmq.filter.util.BloomFilterData#getBitPos}'s length is equal to {@code k}
  • + * + * @param filterData + * @return + */ + public boolean isValid(BloomFilterData filterData) { + if (filterData == null + || filterData.getBitNum() != this.m + || filterData.getBitPos() == null + || filterData.getBitPos().length != this.k) { + return false; + } + + return true; + } + + /** + * error rate. + * + * @return + */ + public int getF() { + return f; + } + + /** + * expect mapping num. + * + * @return + */ + public int getN() { + return n; + } + + /** + * hash function num. + * + * @return + */ + public int getK() { + return k; + } + + /** + * total bit num. + * + * @return + */ + public int getM() { + return m; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (!(o instanceof BloomFilter)) + return false; + + BloomFilter that = (BloomFilter) o; + + if (f != that.f) + return false; + if (k != that.k) + return false; + if (m != that.m) + return false; + if (n != that.n) + return false; + + return true; + } + + @Override + public int hashCode() { + int result = f; + result = 31 * result + n; + result = 31 * result + k; + result = 31 * result + m; + return result; + } + + @Override + public String toString() { + return String.format("f: %d, n: %d, k: %d, m: %d", f, n, k, m); + } + + protected double logMN(double m, double n) { + return Math.log(n) / Math.log(m); + } +} diff --git a/filter/src/main/java/org/apache/rocketmq/filter/util/BloomFilterData.java b/filter/src/main/java/org/apache/rocketmq/filter/util/BloomFilterData.java new file mode 100644 index 00000000..de02d927 --- /dev/null +++ b/filter/src/main/java/org/apache/rocketmq/filter/util/BloomFilterData.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.util; + +import java.util.Arrays; + +/** + * Data generated by bloom filter, include: + *
  • 1. Bit positions allocated to requester;
  • + *
  • 2. Total bit num when allocating;
  • + */ +public class BloomFilterData { + + private int[] bitPos; + private int bitNum; + + public BloomFilterData() { + } + + public BloomFilterData(int[] bitPos, int bitNum) { + this.bitPos = bitPos; + this.bitNum = bitNum; + } + + public int[] getBitPos() { + return bitPos; + } + + public int getBitNum() { + return bitNum; + } + + public void setBitPos(final int[] bitPos) { + this.bitPos = bitPos; + } + + public void setBitNum(final int bitNum) { + this.bitNum = bitNum; + } + + @Override + public boolean equals(final Object o) { + if (this == o) return true; + if (!(o instanceof BloomFilterData)) return false; + + final BloomFilterData that = (BloomFilterData) o; + + if (bitNum != that.bitNum) return false; + if (!Arrays.equals(bitPos, that.bitPos)) return false; + + return true; + } + + @Override + public int hashCode() { + int result = bitPos != null ? Arrays.hashCode(bitPos) : 0; + result = 31 * result + bitNum; + return result; + } + + @Override + public String toString() { + return "BloomFilterData{" + + "bitPos=" + Arrays.toString(bitPos) + + ", bitNum=" + bitNum + + '}'; + } +} diff --git a/filter/src/test/java/org/apache/rocketmq/filter/BitsArrayTest.java b/filter/src/test/java/org/apache/rocketmq/filter/BitsArrayTest.java new file mode 100644 index 00000000..ef81b29b --- /dev/null +++ b/filter/src/test/java/org/apache/rocketmq/filter/BitsArrayTest.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.BitsArray; +import org.junit.Test; + +import java.util.Random; + +import static org.assertj.core.api.Assertions.assertThat; + +public class BitsArrayTest { + + BitsArray gen(int bitCount) { + BitsArray bitsArray = BitsArray.create(bitCount); + + for (int i = 0; i < bitCount / Byte.SIZE; i++) { + bitsArray.setByte(i, (byte) (new Random(System.currentTimeMillis())).nextInt(0xff)); + try { + Thread.sleep(2); + } catch (InterruptedException e) { + } + } + + return bitsArray; + } + + int bitLength = Byte.SIZE; + + @Test + public void testConstructor() { + BitsArray bitsArray = BitsArray.create(8); + + assertThat(bitsArray.byteLength() == 1 && bitsArray.bitLength() == 8).isTrue(); + + bitsArray = BitsArray.create(9); + + assertThat(bitsArray.byteLength() == 2 && bitsArray.bitLength() == 9).isTrue(); + + bitsArray = BitsArray.create(7); + + assertThat(bitsArray.byteLength() == 1 && bitsArray.bitLength() == 7).isTrue(); + } + + @Test + public void testSet() { + BitsArray bitsArray = gen(bitLength); + BitsArray backUp = bitsArray.clone(); + + boolean val = bitsArray.getBit(2); + + bitsArray.setBit(2, !val); + + bitsArray.xor(backUp); + + assertThat(bitsArray.getBit(2)).isTrue(); + } + + @Test + public void testAndOr() { + BitsArray bitsArray = gen(bitLength); + + boolean val = bitsArray.getBit(2); + + if (val) { + bitsArray.and(2, false); + assertThat(!bitsArray.getBit(2)).isTrue(); + } else { + bitsArray.or(2, true); + assertThat(bitsArray.getBit(2)).isTrue(); + } + } + + @Test + public void testXor() { + BitsArray bitsArray = gen(bitLength); + + boolean val = bitsArray.getBit(2); + + bitsArray.xor(2, !val); + + assertThat(bitsArray.getBit(2)).isTrue(); + } + + @Test + public void testNot() { + BitsArray bitsArray = gen(bitLength); + BitsArray backUp = bitsArray.clone(); + + bitsArray.not(2); + + bitsArray.xor(backUp); + + assertThat(bitsArray.getBit(2)).isTrue(); + } + + @Test + public void testOr() { + BitsArray b1 = BitsArray.create(new byte[]{(byte) 0xff, 0x00}); + BitsArray b2 = BitsArray.create(new byte[]{0x00, (byte) 0xff}); + + b1.or(b2); + + for (int i = 0; i < b1.bitLength(); i++) { + assertThat(b1.getBit(i)).isTrue(); + } + } +} diff --git a/filter/src/test/java/org/apache/rocketmq/filter/BloomFilterTest.java b/filter/src/test/java/org/apache/rocketmq/filter/BloomFilterTest.java new file mode 100644 index 00000000..c6097ee8 --- /dev/null +++ b/filter/src/test/java/org/apache/rocketmq/filter/BloomFilterTest.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.BitsArray; +import org.apache.rocketmq.filter.util.BloomFilter; +import org.apache.rocketmq.filter.util.BloomFilterData; +import org.junit.Test; + +import java.util.Random; + +import static org.assertj.core.api.Assertions.assertThat; + +public class BloomFilterTest { + + @Test + public void testEquals() { + BloomFilter a = BloomFilter.createByFn(10, 20); + + BloomFilter b = BloomFilter.createByFn(10, 20); + + BloomFilter c = BloomFilter.createByFn(12, 20); + + BloomFilter d = BloomFilter.createByFn(10, 30); + + assertThat(a).isEqualTo(b); + assertThat(a).isNotEqualTo(c); + assertThat(a).isNotEqualTo(d); + assertThat(d).isNotEqualTo(c); + + assertThat(a.hashCode()).isEqualTo(b.hashCode()); + assertThat(a.hashCode()).isNotEqualTo(c.hashCode()); + assertThat(a.hashCode()).isNotEqualTo(d.hashCode()); + assertThat(c.hashCode()).isNotEqualTo(d.hashCode()); + } + + @Test + public void testHashTo() { + String cid = "CID_abc_efg"; + + BloomFilter bloomFilter = BloomFilter.createByFn(10, 20); + + BitsArray bits = BitsArray.create(bloomFilter.getM()); + + int[] bitPos = bloomFilter.calcBitPositions(cid); + + bloomFilter.hashTo(cid, bits); + + for (int bit : bitPos) { + assertThat(bits.getBit(bit)).isTrue(); + } + } + + @Test + public void testCalcBitPositions() { + String cid = "CID_abc_efg"; + + BloomFilter bloomFilter = BloomFilter.createByFn(10, 20); + + int[] bitPos = bloomFilter.calcBitPositions(cid); + + assertThat(bitPos).isNotNull(); + assertThat(bitPos.length).isEqualTo(bloomFilter.getK()); + + int[] bitPos2 = bloomFilter.calcBitPositions(cid); + + assertThat(bitPos2).isNotNull(); + assertThat(bitPos2.length).isEqualTo(bloomFilter.getK()); + + assertThat(bitPos).isEqualTo(bitPos2); + } + + @Test + public void testIsHit() { + String cid = "CID_abc_efg"; + String cid2 = "CID_abc_123"; + + BloomFilter bloomFilter = BloomFilter.createByFn(10, 20); + + BitsArray bits = BitsArray.create(bloomFilter.getM()); + + bloomFilter.hashTo(cid, bits); + + assertThat(bloomFilter.isHit(cid, bits)).isTrue(); + assertThat(!bloomFilter.isHit(cid2, bits)).isTrue(); + + bloomFilter.hashTo(cid2, bits); + + assertThat(bloomFilter.isHit(cid, bits)).isTrue(); + assertThat(bloomFilter.isHit(cid2, bits)).isTrue(); + } + + @Test + public void testBloomFilterData() { + BloomFilterData bloomFilterData = new BloomFilterData(new int[]{1, 2, 3}, 128); + BloomFilterData bloomFilterData1 = new BloomFilterData(new int[]{1, 2, 3}, 128); + BloomFilterData bloomFilterData2 = new BloomFilterData(new int[]{1, 2, 3}, 129); + + assertThat(bloomFilterData).isEqualTo(bloomFilterData1); + assertThat(bloomFilterData2).isNotEqualTo(bloomFilterData); + assertThat(bloomFilterData2).isNotEqualTo(bloomFilterData1); + + assertThat(bloomFilterData.hashCode()).isEqualTo(bloomFilterData1.hashCode()); + assertThat(bloomFilterData2.hashCode()).isNotEqualTo(bloomFilterData.hashCode()); + assertThat(bloomFilterData2.hashCode()).isNotEqualTo(bloomFilterData1.hashCode()); + + assertThat(bloomFilterData.getBitPos()).isEqualTo(bloomFilterData2.getBitPos()); + assertThat(bloomFilterData.getBitNum()).isEqualTo(bloomFilterData1.getBitNum()); + assertThat(bloomFilterData.getBitNum()).isNotEqualTo(bloomFilterData2.getBitNum()); + + bloomFilterData2.setBitNum(128); + + assertThat(bloomFilterData).isEqualTo(bloomFilterData2); + + bloomFilterData2.setBitPos(new int[]{1, 2, 3, 4}); + + assertThat(bloomFilterData).isNotEqualTo(bloomFilterData2); + + BloomFilterData nullData = new BloomFilterData(); + + assertThat(nullData.getBitNum()).isEqualTo(0); + assertThat(nullData.getBitPos()).isNull(); + + BloomFilter bloomFilter = BloomFilter.createByFn(1, 300); + + assertThat(bloomFilter).isNotNull(); + assertThat(bloomFilter.isValid(bloomFilterData)).isFalse(); + } + + @Test + public void testCheckFalseHit() { + BloomFilter bloomFilter = BloomFilter.createByFn(1, 300); + BitsArray bits = BitsArray.create(bloomFilter.getM()); + int falseHit = 0; + for (int i = 0; i < bloomFilter.getN(); i++) { + String str = randomString((new Random(System.nanoTime())).nextInt(127) + 10); + int[] bitPos = bloomFilter.calcBitPositions(str); + + if (bloomFilter.checkFalseHit(bitPos, bits)) { + falseHit++; + } + + bloomFilter.hashTo(bitPos, bits); + } + + assertThat(falseHit).isLessThanOrEqualTo(bloomFilter.getF() * bloomFilter.getN() / 100); + } + + private String randomString(int length) { + StringBuilder stringBuilder = new StringBuilder(length); + for (int i = 0; i < length; i++) { + stringBuilder.append((char) ((new Random(System.nanoTime())).nextInt(123 - 97) + 97)); + } + + return stringBuilder.toString(); + } +} diff --git a/filter/src/test/java/org/apache/rocketmq/filter/ExpressionTest.java b/filter/src/test/java/org/apache/rocketmq/filter/ExpressionTest.java new file mode 100644 index 00000000..0ee81c98 --- /dev/null +++ b/filter/src/test/java/org/apache/rocketmq/filter/ExpressionTest.java @@ -0,0 +1,594 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.ComparisonExpression; +import org.apache.rocketmq.filter.expression.ConstantExpression; +import org.apache.rocketmq.filter.expression.EvaluationContext; +import org.apache.rocketmq.filter.expression.Expression; +import org.apache.rocketmq.filter.expression.MQFilterException; +import org.apache.rocketmq.filter.expression.PropertyExpression; +import org.apache.rocketmq.filter.parser.SelectorParser; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +public class ExpressionTest { + + private static String andExpression = "a=3 and b<>4 And c>5 AND d<=4"; + private static String orExpression = "a=3 or b<>4 Or c>5 OR d<=4"; + private static String inExpression = "a in ('3', '4', '5')"; + private static String notInExpression = "a not in ('3', '4', '5')"; + private static String betweenExpression = "a between 2 and 10"; + private static String notBetweenExpression = "a not between 2 and 10"; + private static String isNullExpression = "a is null"; + private static String isNotNullExpression = "a is not null"; + private static String equalExpression = "a is not null and a='hello'"; + private static String booleanExpression = "a=TRUE OR b=FALSE"; + private static String nullOrExpression = "a is null OR a='hello'"; + private static String stringHasString = "TAGS is not null and TAGS='''''tag'''''"; + + @Test + public void testEvaluate_stringHasString() { + Expression expr = genExp(stringHasString); + + EvaluationContext context = genContext( + KeyValue.c("TAGS", "''tag''") + ); + + eval(expr, context, Boolean.TRUE); + } + + @Test + public void testEvaluate_now() { + EvaluationContext context = genContext( + KeyValue.c("a", System.currentTimeMillis()) + ); + + Expression nowExpression = ConstantExpression.createNow(); + Expression propertyExpression = new PropertyExpression("a"); + + Expression expression = ComparisonExpression.createLessThanEqual(propertyExpression, + nowExpression); + + eval(expression, context, Boolean.TRUE); + } + + @Test + public void testEvaluate_stringCompare() { + Expression expression = genExp("a between up and low"); + + EvaluationContext context = genContext( + KeyValue.c("a", "3.14") + ); + + eval(expression, context, Boolean.FALSE); + + { + context = genContext( + KeyValue.c("a", "3.14"), + KeyValue.c("up", "up"), + KeyValue.c("low", "low") + ); + + eval(expression, context, Boolean.FALSE); + } + + { + expression = genExp("key is not null and key between 0 and 100"); + + context = genContext( + KeyValue.c("key", "con") + ); + + eval(expression, context, Boolean.FALSE); + } + + { + expression = genExp("a between 0 and 100"); + + context = genContext( + KeyValue.c("a", "abc") + ); + + eval(expression, context, Boolean.FALSE); + } + + { + expression = genExp("a=b"); + + context = genContext( + KeyValue.c("a", "3.14"), + KeyValue.c("b", "3.14") + ); + + eval(expression, context, Boolean.TRUE); + } + + { + expression = genExp("a<>b"); + + context = genContext( + KeyValue.c("a", "3.14"), + KeyValue.c("b", "3.14") + ); + + eval(expression, context, Boolean.FALSE); + } + + { + expression = genExp("a<>b"); + + context = genContext( + KeyValue.c("a", "3.14"), + KeyValue.c("b", "3.141") + ); + + eval(expression, context, Boolean.TRUE); + } + } + + @Test + public void testEvaluate_exponent() { + Expression expression = genExp("a > 3.1E10"); + + EvaluationContext context = genContext( + KeyValue.c("a", String.valueOf(3.1415 * Math.pow(10, 10))) + ); + + eval(expression, context, Boolean.TRUE); + } + + @Test + public void testEvaluate_floatNumber() { + Expression expression = genExp("a > 3.14"); + + EvaluationContext context = genContext( + KeyValue.c("a", String.valueOf(3.1415)) + ); + + eval(expression, context, Boolean.TRUE); + } + + @Test + public void testEvaluate_twoVariable() { + Expression expression = genExp("a > b"); + + EvaluationContext context = genContext( + KeyValue.c("a", String.valueOf(10)), + KeyValue.c("b", String.valueOf(20)) + ); + + eval(expression, context, Boolean.FALSE); + + context = genContext( + KeyValue.c("b", String.valueOf(10)), + KeyValue.c("a", String.valueOf(20)) + ); + + eval(expression, context, Boolean.TRUE); + } + + @Test + public void testEvaluate_nullOr() { + Expression expression = genExp(nullOrExpression); + + EvaluationContext context = genContext( + ); + + eval(expression, context, Boolean.TRUE); + + context = genContext( + KeyValue.c("a", "hello") + ); + + eval(expression, context, Boolean.TRUE); + + context = genContext( + KeyValue.c("a", "abc") + ); + + eval(expression, context, Boolean.FALSE); + } + + @Test + public void testEvaluate_boolean() { + Expression expression = genExp(booleanExpression); + + EvaluationContext context = genContext( + KeyValue.c("a", "true"), + KeyValue.c("b", "false") + ); + + eval(expression, context, Boolean.TRUE); + + context = genContext( + KeyValue.c("a", "false"), + KeyValue.c("b", "true") + ); + + eval(expression, context, Boolean.FALSE); + } + + @Test + public void testEvaluate_equal() { + Expression expression = genExp(equalExpression); + + EvaluationContext context = genContext( + KeyValue.c("a", "hello") + ); + + eval(expression, context, Boolean.TRUE); + + context = genContext( + ); + + eval(expression, context, Boolean.FALSE); + } + + @Test + public void testEvaluate_andTrue() { + Expression expression = genExp(andExpression); + + EvaluationContext context = genContext( + KeyValue.c("a", 3), + KeyValue.c("b", 5), + KeyValue.c("c", 6), + KeyValue.c("d", 1) + ); + + for (int i = 0; i < 500; i++) { + eval(expression, context, Boolean.TRUE); + } + + long start = System.currentTimeMillis(); + for (int j = 0; j < 100; j++) { + for (int i = 0; i < 1000; i++) { + eval(expression, context, Boolean.TRUE); + } + } + + // use string + context = genContext( + KeyValue.c("a", "3"), + KeyValue.c("b", "5"), + KeyValue.c("c", "6"), + KeyValue.c("d", "1") + ); + + eval(expression, context, Boolean.TRUE); + } + + @Test + public void testEvaluate_andFalse() { + Expression expression = genExp(andExpression); + + EvaluationContext context = genContext( + KeyValue.c("a", 4), + KeyValue.c("b", 5), + KeyValue.c("c", 6), + KeyValue.c("d", 1) + ); + + eval(expression, context, Boolean.FALSE); + + // use string + context = genContext( + KeyValue.c("a", "4"), + KeyValue.c("b", "5"), + KeyValue.c("c", "6"), + KeyValue.c("d", "1") + ); + + eval(expression, context, Boolean.FALSE); + } + + @Test + public void testEvaluate_orTrue() { + Expression expression = genExp(orExpression); + + // first + EvaluationContext context = genContext( + KeyValue.c("a", 3) + ); + eval(expression, context, Boolean.TRUE); + + // second + context = genContext( + KeyValue.c("a", 4), + KeyValue.c("b", 5) + ); + eval(expression, context, Boolean.TRUE); + + // third + context = genContext( + KeyValue.c("a", 4), + KeyValue.c("b", 4), + KeyValue.c("c", 6) + ); + eval(expression, context, Boolean.TRUE); + + // forth + context = genContext( + KeyValue.c("a", 4), + KeyValue.c("b", 4), + KeyValue.c("c", 3), + KeyValue.c("d", 2) + ); + eval(expression, context, Boolean.TRUE); + } + + @Test + public void testEvaluate_orFalse() { + Expression expression = genExp(orExpression); + // forth + EvaluationContext context = genContext( + KeyValue.c("a", 4), + KeyValue.c("b", 4), + KeyValue.c("c", 3), + KeyValue.c("d", 10) + ); + eval(expression, context, Boolean.FALSE); + } + + @Test + public void testEvaluate_inTrue() { + Expression expression = genExp(inExpression); + + EvaluationContext context = genContext( + KeyValue.c("a", "3") + ); + eval(expression, context, Boolean.TRUE); + + context = genContext( + KeyValue.c("a", "4") + ); + eval(expression, context, Boolean.TRUE); + + context = genContext( + KeyValue.c("a", "5") + ); + eval(expression, context, Boolean.TRUE); + } + + @Test + public void testEvaluate_inFalse() { + Expression expression = genExp(inExpression); + + EvaluationContext context = genContext( + KeyValue.c("a", "8") + ); + eval(expression, context, Boolean.FALSE); + } + + @Test + public void testEvaluate_notInTrue() { + Expression expression = genExp(notInExpression); + + EvaluationContext context = genContext( + KeyValue.c("a", "8") + ); + eval(expression, context, Boolean.TRUE); + } + + @Test + public void testEvaluate_notInFalse() { + Expression expression = genExp(notInExpression); + + EvaluationContext context = genContext( + KeyValue.c("a", "3") + ); + eval(expression, context, Boolean.FALSE); + + context = genContext( + KeyValue.c("a", "4") + ); + eval(expression, context, Boolean.FALSE); + + context = genContext( + KeyValue.c("a", "5") + ); + eval(expression, context, Boolean.FALSE); + } + + @Test + public void testEvaluate_betweenTrue() { + Expression expression = genExp(betweenExpression); + + EvaluationContext context = genContext( + KeyValue.c("a", "2") + ); + eval(expression, context, Boolean.TRUE); + + context = genContext( + KeyValue.c("a", "10") + ); + eval(expression, context, Boolean.TRUE); + + context = genContext( + KeyValue.c("a", "3") + ); + eval(expression, context, Boolean.TRUE); + } + + @Test + public void testEvaluate_betweenFalse() { + Expression expression = genExp(betweenExpression); + + EvaluationContext context = genContext( + KeyValue.c("a", "1") + ); + eval(expression, context, Boolean.FALSE); + + context = genContext( + KeyValue.c("a", "11") + ); + eval(expression, context, Boolean.FALSE); + } + + @Test + public void testEvaluate_notBetweenTrue() { + Expression expression = genExp(notBetweenExpression); + + EvaluationContext context = genContext( + KeyValue.c("a", "1") + ); + eval(expression, context, Boolean.TRUE); + + context = genContext( + KeyValue.c("a", "11") + ); + eval(expression, context, Boolean.TRUE); + } + + @Test + public void testEvaluate_notBetweenFalse() { + Expression expression = genExp(notBetweenExpression); + + EvaluationContext context = genContext( + KeyValue.c("a", "2") + ); + eval(expression, context, Boolean.FALSE); + + context = genContext( + KeyValue.c("a", "10") + ); + eval(expression, context, Boolean.FALSE); + + context = genContext( + KeyValue.c("a", "3") + ); + eval(expression, context, Boolean.FALSE); + } + + @Test + public void testEvaluate_isNullTrue() { + Expression expression = genExp(isNullExpression); + + EvaluationContext context = genContext( + KeyValue.c("abc", "2") + ); + eval(expression, context, Boolean.TRUE); + } + + @Test + public void testEvaluate_isNullFalse() { + Expression expression = genExp(isNullExpression); + + EvaluationContext context = genContext( + KeyValue.c("a", "2") + ); + eval(expression, context, Boolean.FALSE); + } + + @Test + public void testEvaluate_isNotNullTrue() { + Expression expression = genExp(isNotNullExpression); + + EvaluationContext context = genContext( + KeyValue.c("a", "2") + ); + eval(expression, context, Boolean.TRUE); + } + + @Test + public void testEvaluate_isNotNullFalse() { + Expression expression = genExp(isNotNullExpression); + + EvaluationContext context = genContext( + KeyValue.c("abc", "2") + ); + eval(expression, context, Boolean.FALSE); + } + + protected void eval(Expression expression, EvaluationContext context, Boolean result) { + Object ret = null; + try { + ret = expression.evaluate(context); + } catch (Throwable e) { + e.printStackTrace(); + } + + if (ret == null || !(ret instanceof Boolean)) { + assertThat(result).isFalse(); + } else { + assertThat(result).isEqualTo(ret); + } + } + + protected EvaluationContext genContext(KeyValue... keyValues) { + if (keyValues == null || keyValues.length < 1) { + return new PropertyContext(); + } + + PropertyContext context = new PropertyContext(); + for (KeyValue keyValue : keyValues) { + context.properties.put(keyValue.key, keyValue.value); + } + + return context; + } + + protected Expression genExp(String exp) { + Expression expression = null; + + try { + expression = SelectorParser.parse(exp); + + assertThat(expression).isNotNull(); + } catch (MQFilterException e) { + e.printStackTrace(); + assertThat(Boolean.FALSE).isTrue(); + } + + return expression; + } + + static class KeyValue { + public static KeyValue c(String key, Object value) { + return new KeyValue(key, value); + } + + public KeyValue(String key, Object value) { + this.key = key; + this.value = value; + } + + public String key; + public Object value; + } + + class PropertyContext implements EvaluationContext { + + public Map properties = new HashMap(8); + + @Override + public Object get(final String name) { + return properties.get(name); + } + + @Override + public Map keyValues() { + return properties; + } + + } +} diff --git a/filter/src/test/java/org/apache/rocketmq/filter/FilterSpiTest.java b/filter/src/test/java/org/apache/rocketmq/filter/FilterSpiTest.java new file mode 100644 index 00000000..22eeb860 --- /dev/null +++ b/filter/src/test/java/org/apache/rocketmq/filter/FilterSpiTest.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.EmptyEvaluationContext; +import org.apache.rocketmq.filter.expression.EvaluationContext; +import org.apache.rocketmq.filter.expression.Expression; +import org.apache.rocketmq.filter.expression.MQFilterException; +import org.junit.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +public class FilterSpiTest { + + static class NothingExpression implements Expression { + + @Override + public Object evaluate(final EvaluationContext context) throws Exception { + return Boolean.TRUE; + } + } + + static class NothingFilter implements FilterSpi { + @Override + public Expression compile(final String expr) throws MQFilterException { + return new NothingExpression(); + } + + @Override + public String ofType() { + return "Nothing"; + } + } + + + @Test + public void testRegister() { + FilterFactory.INSTANCE.register(new NothingFilter()); + + Expression expr = null; + try { + expr = FilterFactory.INSTANCE.get("Nothing").compile("abc"); + } catch (MQFilterException e) { + e.printStackTrace(); + assertThat(Boolean.FALSE).isTrue(); + } + + assertThat(expr).isNotNull(); + + try { + assertThat((Boolean) expr.evaluate(new EmptyEvaluationContext())).isTrue(); + } catch (Exception e) { + e.printStackTrace(); + assertThat(Boolean.FALSE).isTrue(); + } + } + + @Test + public void testGet() { + try { + assertThat((Boolean) FilterFactory.INSTANCE.get(ExpressionType.SQL92).compile("a is not null and a > 0") + .evaluate(new EmptyEvaluationContext())).isFalse(); + } catch (Exception e) { + e.printStackTrace(); + assertThat(Boolean.FALSE).isTrue(); + } + } +} diff --git a/filter/src/test/java/org/apache/rocketmq/filter/ParserTest.java b/filter/src/test/java/org/apache/rocketmq/filter/ParserTest.java new file mode 100644 index 00000000..36ef2714 --- /dev/null +++ b/filter/src/test/java/org/apache/rocketmq/filter/ParserTest.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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; +import org.apache.rocketmq.filter.parser.SelectorParser; +import org.junit.Test; + +import java.util.Arrays; + +import static org.assertj.core.api.Assertions.assertThat; + +public class ParserTest { + + private static String andExpression = "a=3 and b<>4 And c>5 AND d<=4"; + private static String andExpressionHasBlank = "a=3 and b<>4 And c>5 AND d<=4"; + private static String orExpression = "a=3 or b<>4 Or c>5 OR d<=4"; + private static String inExpression = "a in ('3', '4', '5')"; + private static String notInExpression = "(a not in ('6', '4', '5')) or (b in ('3', '4', '5'))"; + private static String betweenExpression = "(a between 2 and 10) AND (b not between 6 and 9)"; + private static String equalNullExpression = "a is null"; + private static String notEqualNullExpression = "a is not null"; + private static String nowExpression = "a <= now"; + + private static String invalidExpression = "a and between 2 and 10"; + private static String illegalBetween = " a between 10 and 0"; + + @Test + public void testParse_valid() { + for (String expr : Arrays.asList( + andExpression, orExpression, inExpression, notInExpression, betweenExpression, + equalNullExpression, notEqualNullExpression, nowExpression + )) { + + try { + Expression expression = SelectorParser.parse(expr); + assertThat(expression).isNotNull(); + } catch (MQFilterException e) { + e.printStackTrace(); + assertThat(Boolean.FALSE).isTrue(); + } + + } + } + + @Test + public void testParse_invalid() { + try { + SelectorParser.parse(invalidExpression); + + assertThat(Boolean.TRUE).isFalse(); + } catch (MQFilterException e) { + } + } + + @Test + public void testParse_decimalOverFlow() { + try { + String str = "100000000000000000000000"; + + SelectorParser.parse("a > " + str); + + assertThat(Boolean.TRUE).isFalse(); + } catch (Exception e) { + } + } + + @Test + public void testParse_floatOverFlow() { + try { + String str = "1"; + for (int i = 0; i < 2048; i++) { + str += "111111111111111111111111111111111111111111111111111"; + } + str += "."; + for (int i = 0; i < 2048; i++) { + str += "111111111111111111111111111111111111111111111111111"; + } + + SelectorParser.parse("a > " + str); + + assertThat(Boolean.TRUE).isFalse(); + } catch (Exception e) { + } + } + + @Test + public void testParse_illegalBetween() { + try { + SelectorParser.parse(illegalBetween); + + assertThat(Boolean.TRUE).isFalse(); + } catch (Exception e) { + } + } + + @Test + public void testEquals() { + try { + Expression expr1 = SelectorParser.parse(andExpression); + + Expression expr2 = SelectorParser.parse(andExpressionHasBlank); + + Expression expr3 = SelectorParser.parse(orExpression); + + assertThat(expr1).isEqualTo(expr2); + assertThat(expr1).isNotEqualTo(expr3); + } catch (MQFilterException e) { + e.printStackTrace(); + assertThat(Boolean.TRUE).isFalse(); + } + } +} diff --git a/pom.xml b/pom.xml index 47df84d4..feb8b148 100644 --- a/pom.xml +++ b/pom.xml @@ -178,6 +178,7 @@ example filtersrv srvutil + filter test distribution @@ -553,6 +554,11 @@ rocketmq-srvutil ${project.version} + + org.apache.rocketmq + rocketmq-filter + ${project.version} + ${project.groupId} rocketmq-example @@ -603,6 +609,11 @@ commons-lang3 3.4 + + com.google.guava + guava + 19.0 + diff --git a/srvutil/pom.xml b/srvutil/pom.xml index 32699035..6dc0377f 100644 --- a/srvutil/pom.xml +++ b/srvutil/pom.xml @@ -41,5 +41,9 @@ commons-cli commons-cli + + com.google.guava + guava + diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java index 5be8258e..7841febf 100644 --- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java @@ -314,10 +314,11 @@ public class CommitLog { // 17 properties short propertiesLength = byteBuffer.getShort(); + Map propertiesMap = null; if (propertiesLength > 0) { byteBuffer.get(bytesContent, 0, propertiesLength); String properties = new String(bytesContent, 0, propertiesLength, MessageDecoder.CHARSET_UTF8); - Map propertiesMap = MessageDecoder.string2messageProperties(properties); + propertiesMap = MessageDecoder.string2messageProperties(properties); keys = propertiesMap.get(MessageConst.PROPERTY_KEYS); @@ -369,8 +370,9 @@ public class CommitLog { queueOffset, // 7 keys, // 8 uniqKey, //9 - sysFlag, // 9 - preparedTransactionOffset// 10 + sysFlag, // 10 + preparedTransactionOffset, // 11 + propertiesMap // 12 ); } catch (Exception e) { } diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLogDispatcher.java b/store/src/main/java/org/apache/rocketmq/store/CommitLogDispatcher.java new file mode 100644 index 00000000..e1564a9e --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLogDispatcher.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.store; + +/** + * Dispatcher of commit log. + */ +public interface CommitLogDispatcher { + + void dispatch(final DispatchRequest request); +} diff --git a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java index 919c6372..d03ff0f3 100644 --- a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java +++ b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java @@ -20,6 +20,7 @@ import java.io.File; import java.nio.ByteBuffer; import java.util.List; import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.store.config.StorePathConfigHelper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,6 +41,7 @@ public class ConsumeQueue { private final int mappedFileSize; private long maxPhysicOffset = -1; private volatile long minLogicOffset = 0; + private ConsumeQueueExt consumeQueueExt = null; public ConsumeQueue( final String topic, @@ -61,11 +63,24 @@ public class ConsumeQueue { this.mappedFileQueue = new MappedFileQueue(queueDir, mappedFileSize, null); this.byteBufferIndex = ByteBuffer.allocate(CQ_STORE_UNIT_SIZE); + + if (defaultMessageStore.getMessageStoreConfig().isEnableConsumeQueueExt()) { + this.consumeQueueExt = new ConsumeQueueExt( + topic, + queueId, + StorePathConfigHelper.getStorePathConsumeQueueExt(defaultMessageStore.getMessageStoreConfig().getStorePathRootDir()), + defaultMessageStore.getMessageStoreConfig().getMappedFileSizeConsumeQueueExt(), + defaultMessageStore.getMessageStoreConfig().getBitMapLengthConsumeQueueExt() + ); + } } public boolean load() { boolean result = this.mappedFileQueue.load(); log.info("load consume queue " + this.topic + "-" + this.queueId + " " + (result ? "OK" : "Failed")); + if (isExtReadEnable()) { + result &= this.consumeQueueExt.load(); + } return result; } @@ -82,6 +97,7 @@ public class ConsumeQueue { ByteBuffer byteBuffer = mappedFile.sliceByteBuffer(); long processOffset = mappedFile.getFileFromOffset(); long mappedFileOffset = 0; + long maxExtAddr = 1; while (true) { for (int i = 0; i < mappedFileSizeLogics; i += CQ_STORE_UNIT_SIZE) { long offset = byteBuffer.getLong(); @@ -91,6 +107,9 @@ public class ConsumeQueue { if (offset >= 0 && size > 0) { mappedFileOffset = i + CQ_STORE_UNIT_SIZE; this.maxPhysicOffset = offset; + if (isExtAddr(tagsCode)) { + maxExtAddr = tagsCode; + } } else { log.info("recover current consume queue file over, " + mappedFile.getFileName() + " " + offset + " " + size + " " + tagsCode); @@ -123,6 +142,12 @@ public class ConsumeQueue { this.mappedFileQueue.setFlushedWhere(processOffset); this.mappedFileQueue.setCommittedWhere(processOffset); this.mappedFileQueue.truncateDirtyFiles(processOffset); + + if (isExtReadEnable()) { + this.consumeQueueExt.recover(); + log.info("Truncate consume queue extend file by max {}", maxExtAddr); + this.consumeQueueExt.truncateByMaxAddress(maxExtAddr); + } } } @@ -200,7 +225,7 @@ public class ConsumeQueue { int logicFileSize = this.mappedFileSize; this.maxPhysicOffset = phyOffet - 1; - + long maxExtAddr = 1; while (true) { MappedFile mappedFile = this.mappedFileQueue.getLastMappedFile(); if (mappedFile != null) { @@ -213,7 +238,7 @@ public class ConsumeQueue { for (int i = 0; i < logicFileSize; i += CQ_STORE_UNIT_SIZE) { long offset = byteBuffer.getLong(); int size = byteBuffer.getInt(); - byteBuffer.getLong(); + long tagsCode = byteBuffer.getLong(); if (0 == i) { if (offset >= phyOffet) { @@ -225,6 +250,10 @@ public class ConsumeQueue { mappedFile.setCommittedPosition(pos); mappedFile.setFlushedPosition(pos); this.maxPhysicOffset = offset; + // This maybe not take effect, when not every consume queue has extend file. + if (isExtAddr(tagsCode)) { + maxExtAddr = tagsCode; + } } } else { @@ -239,6 +268,9 @@ public class ConsumeQueue { mappedFile.setCommittedPosition(pos); mappedFile.setFlushedPosition(pos); this.maxPhysicOffset = offset; + if (isExtAddr(tagsCode)) { + maxExtAddr = tagsCode; + } if (pos == logicFileSize) { return; @@ -252,6 +284,10 @@ public class ConsumeQueue { break; } } + + if (isExtReadEnable()) { + this.consumeQueueExt.truncateByMaxAddress(maxExtAddr); + } } public long getLastOffset() { @@ -285,7 +321,12 @@ public class ConsumeQueue { } public boolean flush(final int flushLeastPages) { - return this.mappedFileQueue.flush(flushLeastPages); + boolean result = this.mappedFileQueue.flush(flushLeastPages); + if (isExtReadEnable()) { + result = result & this.consumeQueueExt.flush(flushLeastPages); + } + + return result; } public int deleteExpiredFile(long offset) { @@ -296,6 +337,7 @@ public class ConsumeQueue { public void correctMinOffset(long phyMinOffset) { MappedFile mappedFile = this.mappedFileQueue.getFirstMappedFile(); + long minExtAddr = 1; if (mappedFile != null) { SelectMappedBufferResult result = mappedFile.selectMappedBuffer(0); if (result != null) { @@ -303,12 +345,16 @@ public class ConsumeQueue { for (int i = 0; i < result.getSize(); i += ConsumeQueue.CQ_STORE_UNIT_SIZE) { long offsetPy = result.getByteBuffer().getLong(); result.getByteBuffer().getInt(); - result.getByteBuffer().getLong(); + long tagsCode = result.getByteBuffer().getLong(); if (offsetPy >= phyMinOffset) { this.minLogicOffset = result.getMappedFile().getFileFromOffset() + i; log.info("Compute logical min offset: {}, topic: {}, queueId: {}", this.getMinOffsetInQueue(), this.topic, this.queueId); + // This maybe not take effect, when not every consume queue has extend file. + if (isExtAddr(tagsCode)) { + minExtAddr = tagsCode; + } break; } } @@ -319,24 +365,43 @@ public class ConsumeQueue { } } } + + if (isExtReadEnable()) { + this.consumeQueueExt.truncateByMinAddress(minExtAddr); + } } public long getMinOffsetInQueue() { return this.minLogicOffset / CQ_STORE_UNIT_SIZE; } - public void putMessagePositionInfoWrapper(long offset, int size, long tagsCode, long storeTimestamp, - long logicOffset) { + public void putMessagePositionInfoWrapper(DispatchRequest request) { final int maxRetries = 30; boolean canWrite = this.defaultMessageStore.getRunningFlags().isCQWriteable(); for (int i = 0; i < maxRetries && canWrite; i++) { - boolean result = this.putMessagePositionInfo(offset, size, tagsCode, logicOffset); + long tagsCode = request.getTagsCode(); + if (isExtWriteEnable()) { + ConsumeQueueExt.CqExtUnit cqExtUnit = new ConsumeQueueExt.CqExtUnit(); + cqExtUnit.setFilterBitMap(request.getBitMap()); + cqExtUnit.setMsgStoreTime(request.getStoreTimestamp()); + cqExtUnit.setTagsCode(request.getTagsCode()); + + long extAddr = this.consumeQueueExt.put(cqExtUnit); + if (isExtAddr(extAddr)) { + tagsCode = extAddr; + } else { + log.warn("Save consume queue extend fail, So just save tagsCode! {}, topic:{}, queueId:{}, offset:{}", cqExtUnit, + topic, queueId, request.getCommitLogOffset()); + } + } + boolean result = this.putMessagePositionInfo(request.getCommitLogOffset(), + request.getMsgSize(), tagsCode, request.getConsumeQueueOffset()); if (result) { - this.defaultMessageStore.getStoreCheckpoint().setLogicsMsgTimestamp(storeTimestamp); + this.defaultMessageStore.getStoreCheckpoint().setLogicsMsgTimestamp(request.getStoreTimestamp()); return; } else { // XXX: warn and notify me - log.warn("[BUG]put commit log position info to " + topic + ":" + queueId + " " + offset + log.warn("[BUG]put commit log position info to " + topic + ":" + queueId + " " + request.getCommitLogOffset() + " failed, retry " + i + " times"); try { @@ -423,6 +488,20 @@ public class ConsumeQueue { return null; } + public ConsumeQueueExt.CqExtUnit getExt(final long offset) { + if (isExtReadEnable()) { + return this.consumeQueueExt.get(offset); + } + return null; + } + + public boolean getExt(final long offset, ConsumeQueueExt.CqExtUnit cqExtUnit) { + if (isExtReadEnable()) { + return this.consumeQueueExt.get(offset, cqExtUnit); + } + return false; + } + public long getMinLogicOffset() { return minLogicOffset; } @@ -457,6 +536,9 @@ public class ConsumeQueue { this.maxPhysicOffset = -1; this.minLogicOffset = 0; this.mappedFileQueue.destroy(); + if (isExtReadEnable()) { + this.consumeQueueExt.destroy(); + } } public long getMessageTotalInQueue() { @@ -469,5 +551,27 @@ public class ConsumeQueue { public void checkSelf() { mappedFileQueue.checkSelf(); + if (isExtReadEnable()) { + this.consumeQueueExt.checkSelf(); + } + } + + protected boolean isExtReadEnable() { + return this.consumeQueueExt != null; + } + + protected boolean isExtWriteEnable() { + return this.consumeQueueExt != null + && this.defaultMessageStore.getMessageStoreConfig().isEnableConsumeQueueExt(); + } + + /** + * Check {@code tagsCode} is address of extend file or tags code. + * + * @param tagsCode + * @return + */ + public boolean isExtAddr(long tagsCode) { + return isExtReadEnable() && this.consumeQueueExt.isExtAddr(tagsCode); } } diff --git a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueueExt.java b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueueExt.java new file mode 100644 index 00000000..1a177e99 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueueExt.java @@ -0,0 +1,638 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.store; + +import org.apache.rocketmq.common.constant.LoggerName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Extend of consume queue, to store something not important, + * such as message store time, filter bit map and etc. + *

    + *

  • 1. This class is used only by {@link ConsumeQueue}
  • + *
  • 2. And is week reliable.
  • + *
  • 3. Be careful, address returned is always less than 0.
  • + *
  • 4. Pls keep this file small.
  • + */ +public class ConsumeQueueExt { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + + private final MappedFileQueue mappedFileQueue; + private final String topic; + private final int queueId; + + private final String storePath; + private final int mappedFileSize; + private ByteBuffer tempContainer; + + public static final int END_BLANK_DATA_LENGTH = 4; + + /** + * Addr can not exceed this value.For compatible. + */ + public static final long MAX_ADDR = Integer.MIN_VALUE - 1L; + public static final long MAX_REAL_OFFSET = MAX_ADDR - Long.MIN_VALUE; + + /** + * Constructor. + * + * @param topic topic + * @param queueId id of queue + * @param storePath root dir of files to store. + * @param mappedFileSize file size + * @param bitMapLength bit map length. + */ + public ConsumeQueueExt(final String topic, + final int queueId, + final String storePath, + final int mappedFileSize, + final int bitMapLength) { + + this.storePath = storePath; + this.mappedFileSize = mappedFileSize; + + this.topic = topic; + this.queueId = queueId; + + String queueDir = this.storePath + + File.separator + topic + + File.separator + queueId; + + this.mappedFileQueue = new MappedFileQueue(queueDir, mappedFileSize, null); + + if (bitMapLength > 0) { + this.tempContainer = ByteBuffer.allocate( + bitMapLength / Byte.SIZE + ); + } + } + + /** + * Check whether {@code address} point to extend file. + *

    + * Just test {@code address} is less than 0. + *

    + * + * @param address + * @return + */ + public boolean isExtAddr(final long address) { + return address <= MAX_ADDR; + } + + /** + * Transform {@code address}(decorated by {@link #decorate}) to offset in mapped file. + *

    + * if {@code address} is less than 0, return {@code address} - {@link java.lang.Long#MIN_VALUE}; + * else, just return {@code address} + *

    + * + * @param address + * @return + */ + public long unDecorate(final long address) { + if (isExtAddr(address)) { + return address - Long.MIN_VALUE; + } + return address; + } + + /** + * Decorate {@code offset} from mapped file, in order to distinguish with tagsCode(saved in cq originally). + *

    + * if {@code offset} is greater than or equal to 0, then return {@code offset} + {@link java.lang.Long#MIN_VALUE}; + * else, just return {@code offset} + *

    + * + * @param offset + * @return ext address(value is less than 0) + */ + public long decorate(final long offset) { + if (!isExtAddr(offset)) { + return offset + Long.MIN_VALUE; + } + return offset; + } + + /** + * Get data from buffer. + * + * @param address less than 0 + * @return + */ + public CqExtUnit get(final long address) { + CqExtUnit cqExtUnit = new CqExtUnit(); + if (get(address, cqExtUnit)) { + return cqExtUnit; + } + + return null; + } + + /** + * Get data from buffer, and set to {@code cqExtUnit} + * + * @param address less than 0 + * @param cqExtUnit + * @return + */ + public boolean get(final long address, final CqExtUnit cqExtUnit) { + if (!isExtAddr(address)) { + return false; + } + + final int mappedFileSize = this.mappedFileSize; + final long realOffset = unDecorate(address); + + MappedFile mappedFile = this.mappedFileQueue.findMappedFileByOffset(realOffset, realOffset == 0); + if (mappedFile == null) { + return false; + } + + int pos = (int) (realOffset % mappedFileSize); + + SelectMappedBufferResult bufferResult = mappedFile.selectMappedBuffer(pos); + if (bufferResult == null) { + log.warn("[BUG] Consume queue extend unit({}) is not found!", realOffset); + return false; + } + boolean ret = false; + try { + ret = cqExtUnit.read(bufferResult.getByteBuffer()); + } finally { + bufferResult.release(); + } + + return ret; + } + + /** + * Save to mapped buffer of file and return address. + *

    + * Be careful, this method is not thread safe. + *

    + * + * @param cqExtUnit + * @return success: < 0: fail: >=0 + */ + public long put(final CqExtUnit cqExtUnit) { + final int retryTimes = 3; + try { + int size = cqExtUnit.calcUnitSize(); + if (size > CqExtUnit.MAX_EXT_UNIT_SIZE) { + log.error("Size of cq ext unit is greater than {}, {}", CqExtUnit.MAX_EXT_UNIT_SIZE, cqExtUnit); + return 1; + } + if (this.mappedFileQueue.getMaxOffset() + size > MAX_REAL_OFFSET) { + log.warn("Capacity of ext is maximum!{}, {}", this.mappedFileQueue.getMaxOffset(), size); + return 1; + } + // unit size maybe change.but, the same most of the time. + if (this.tempContainer == null || this.tempContainer.capacity() < size) { + this.tempContainer = ByteBuffer.allocate(size); + } + + for (int i = 0; i < retryTimes; i++) { + MappedFile mappedFile = this.mappedFileQueue.getLastMappedFile(); + + if (mappedFile == null || mappedFile.isFull()) { + mappedFile = this.mappedFileQueue.getLastMappedFile(0); + } + + if (mappedFile == null) { + log.error("Create mapped file when save consume queue extend, {}", cqExtUnit); + continue; + } + final int wrotePosition = mappedFile.getWrotePosition(); + final int blankSize = this.mappedFileSize - wrotePosition - END_BLANK_DATA_LENGTH; + + // check whether has enough space. + if (size > blankSize) { + fullFillToEnd(mappedFile, wrotePosition); + log.info("No enough space(need:{}, has:{}) of file {}, so fill to end", + size, blankSize, mappedFile.getFileName()); + continue; + } + + if (mappedFile.appendMessage(cqExtUnit.write(this.tempContainer), 0, size)) { + return decorate(wrotePosition + mappedFile.getFileFromOffset()); + } + } + } catch (Throwable e) { + log.error("Save consume queue extend error, " + cqExtUnit, e); + } + + return 1; + } + + protected void fullFillToEnd(final MappedFile mappedFile, final int wrotePosition) { + ByteBuffer mappedFileBuffer = mappedFile.sliceByteBuffer(); + mappedFileBuffer.position(wrotePosition); + + // ending. + mappedFileBuffer.putShort((short) -1); + + mappedFile.setWrotePosition(this.mappedFileSize); + } + + /** + * Load data from file when startup. + * + * @return + */ + public boolean load() { + boolean result = this.mappedFileQueue.load(); + log.info("load consume queue extend" + this.topic + "-" + this.queueId + " " + (result ? "OK" : "Failed")); + return result; + } + + /** + * Check whether the step size in mapped file queue is correct. + */ + public void checkSelf() { + this.mappedFileQueue.checkSelf(); + } + + /** + * Recover. + */ + public void recover() { + final List mappedFiles = this.mappedFileQueue.getMappedFiles(); + if (mappedFiles == null || mappedFiles.isEmpty()) { + return; + } + + // load all files, consume queue will truncate extend files. + int index = 0; + + MappedFile mappedFile = mappedFiles.get(index); + ByteBuffer byteBuffer = mappedFile.sliceByteBuffer(); + long processOffset = mappedFile.getFileFromOffset(); + long mappedFileOffset = 0; + CqExtUnit extUnit = new CqExtUnit(); + while (true) { + extUnit.readBySkip(byteBuffer); + + // check whether write sth. + if (extUnit.getSize() > 0) { + mappedFileOffset += extUnit.getSize(); + continue; + } + + index++; + if (index < mappedFiles.size()) { + mappedFile = mappedFiles.get(index); + byteBuffer = mappedFile.sliceByteBuffer(); + processOffset = mappedFile.getFileFromOffset(); + mappedFileOffset = 0; + log.info("Recover next consume queue extend file, " + mappedFile.getFileName()); + continue; + } + + log.info("All files of consume queue extend has been recovered over, last mapped file " + + mappedFile.getFileName()); + break; + } + + processOffset += mappedFileOffset; + this.mappedFileQueue.setFlushedWhere(processOffset); + this.mappedFileQueue.setCommittedWhere(processOffset); + this.mappedFileQueue.truncateDirtyFiles(processOffset); + } + + /** + * Delete files before {@code minAddress}. + * + * @param minAddress less than 0 + */ + public void truncateByMinAddress(final long minAddress) { + if (!isExtAddr(minAddress)) { + return; + } + + log.info("Truncate consume queue ext by min {}.", minAddress); + + List willRemoveFiles = new ArrayList(); + + List mappedFiles = this.mappedFileQueue.getMappedFiles(); + final long realOffset = unDecorate(minAddress); + + for (MappedFile file : mappedFiles) { + long fileTailOffset = file.getFileFromOffset() + this.mappedFileSize; + + if (fileTailOffset < realOffset) { + log.info("Destroy consume queue ext by min: file={}, fileTailOffset={}, minOffset={}", file.getFileName(), + fileTailOffset, realOffset); + if (file.destroy(1000)) { + willRemoveFiles.add(file); + } + } + } + + this.mappedFileQueue.deleteExpiredFile(willRemoveFiles); + } + + /** + * Delete files after {@code maxAddress}, and reset wrote/commit/flush position to last file. + * + * @param maxAddress less than 0 + */ + public void truncateByMaxAddress(final long maxAddress) { + if (!isExtAddr(maxAddress)) { + return; + } + + log.info("Truncate consume queue ext by max {}.", maxAddress); + + CqExtUnit cqExtUnit = get(maxAddress); + if (cqExtUnit == null) { + log.error("[BUG] address {} of consume queue extend not found!", maxAddress); + return; + } + + final long realOffset = unDecorate(maxAddress); + + this.mappedFileQueue.truncateDirtyFiles(realOffset + cqExtUnit.getSize()); + } + + /** + * flush buffer to file. + * + * @param flushLeastPages + * @return + */ + public boolean flush(final int flushLeastPages) { + return this.mappedFileQueue.flush(flushLeastPages); + } + + /** + * delete files and directory. + */ + public void destroy() { + this.mappedFileQueue.destroy(); + } + + /** + * Max address(value is less than 0). + *

    + *

    + * Be careful: it's an address just when invoking this method. + *

    + * + * @return + */ + public long getMaxAddress() { + MappedFile mappedFile = this.mappedFileQueue.getLastMappedFile(); + if (mappedFile == null) { + return decorate(0); + } + return decorate(mappedFile.getFileFromOffset() + mappedFile.getWrotePosition()); + } + + /** + * Minus address saved in file. + * + * @return + */ + public long getMinAddress() { + MappedFile firstFile = this.mappedFileQueue.getFirstMappedFile(); + if (firstFile == null) { + return decorate(0); + } + return decorate(firstFile.getFileFromOffset()); + } + + /** + * Store unit. + */ + public static class CqExtUnit { + public static final short MIN_EXT_UNIT_SIZE + = 2 * 1 // size, 32k max + + 8 * 2 // msg time + tagCode + + 2; // bitMapSize + + public static final int MAX_EXT_UNIT_SIZE = Short.MAX_VALUE; + + public CqExtUnit() {} + + public CqExtUnit(Long tagsCode, long msgStoreTime, byte[] filterBitMap) { + this.tagsCode = tagsCode == null ? 0 : tagsCode; + this.msgStoreTime = msgStoreTime; + this.filterBitMap = filterBitMap; + this.bitMapSize = (short) (filterBitMap == null ? 0 : filterBitMap.length); + this.size = (short) (MIN_EXT_UNIT_SIZE + this.bitMapSize); + } + + /** + * unit size + */ + private short size; + /** + * has code of tags + */ + private long tagsCode; + /** + * the time to store into commit log of message + */ + private long msgStoreTime; + /** + * size of bit map + */ + private short bitMapSize; + /** + * filter bit map + */ + private byte[] filterBitMap; + + /** + * build unit from buffer from current position. + * + * @param buffer + * @return + */ + private boolean read(final ByteBuffer buffer) { + if (buffer.position() + 2 > buffer.limit()) { + return false; + } + + this.size = buffer.getShort(); + + if (this.size < 1) { + return false; + } + + this.tagsCode = buffer.getLong(); + this.msgStoreTime = buffer.getLong(); + this.bitMapSize = buffer.getShort(); + + if (this.bitMapSize < 1) { + return true; + } + + if (this.filterBitMap == null || this.filterBitMap.length != this.bitMapSize) { + this.filterBitMap = new byte[bitMapSize]; + } + + buffer.get(this.filterBitMap); + return true; + } + + /** + * Only read first 2 byte to get unit size. + *

    + * if size > 0, then skip buffer position with size. + *

    + *

    + * if size <= 0, nothing to do. + *

    + * + * @param buffer + */ + private void readBySkip(final ByteBuffer buffer) { + ByteBuffer temp = buffer.slice(); + + short tempSize = temp.getShort(); + this.size = tempSize; + + if (tempSize > 0) { + buffer.position(buffer.position() + this.size); + } + } + + /** + * Transform unit data to byte array. + *

    + *

  • 1. @{code container} can be null, it will be created if null.
  • + *
  • 2. if capacity of @{code container} is less than unit size, it will be created also.
  • + *
  • 3. Pls be sure that size of unit is not greater than {@link #MAX_EXT_UNIT_SIZE}
  • + * + * @param container + * @return + */ + private byte[] write(final ByteBuffer container) { + this.bitMapSize = (short) (filterBitMap == null ? 0 : filterBitMap.length); + this.size = (short) (MIN_EXT_UNIT_SIZE + this.bitMapSize); + + ByteBuffer temp = container; + + if (temp == null || temp.capacity() < this.size) { + temp = ByteBuffer.allocate(this.size); + } + + temp.flip(); + temp.limit(this.size); + + temp.putShort(this.size); + temp.putLong(this.tagsCode); + temp.putLong(this.msgStoreTime); + temp.putShort(this.bitMapSize); + if (this.bitMapSize > 0) { + temp.put(this.filterBitMap); + } + + return temp.array(); + } + + /** + * Calculate unit size by current data. + * + * @return + */ + private int calcUnitSize() { + int sizeTemp = MIN_EXT_UNIT_SIZE + (filterBitMap == null ? 0 : filterBitMap.length); + return sizeTemp; + } + + public long getTagsCode() { + return tagsCode; + } + + public void setTagsCode(final long tagsCode) { + this.tagsCode = tagsCode; + } + + public long getMsgStoreTime() { + return msgStoreTime; + } + + public void setMsgStoreTime(final long msgStoreTime) { + this.msgStoreTime = msgStoreTime; + } + + public byte[] getFilterBitMap() { + if (this.bitMapSize < 1) { + return null; + } + return filterBitMap; + } + + public void setFilterBitMap(final byte[] filterBitMap) { + this.filterBitMap = filterBitMap; + // not safe transform, but size will be calculate by #calcUnitSize + this.bitMapSize = (short) (filterBitMap == null ? 0 : filterBitMap.length); + } + + public short getSize() { + return size; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof CqExtUnit)) return false; + + CqExtUnit cqExtUnit = (CqExtUnit) o; + + if (bitMapSize != cqExtUnit.bitMapSize) return false; + if (msgStoreTime != cqExtUnit.msgStoreTime) return false; + if (size != cqExtUnit.size) return false; + if (tagsCode != cqExtUnit.tagsCode) return false; + if (!Arrays.equals(filterBitMap, cqExtUnit.filterBitMap)) return false; + + return true; + } + + @Override + public int hashCode() { + int result = (int) size; + result = 31 * result + (int) (tagsCode ^ (tagsCode >>> 32)); + result = 31 * result + (int) (msgStoreTime ^ (msgStoreTime >>> 32)); + result = 31 * result + (int) bitMapSize; + result = 31 * result + (filterBitMap != null ? Arrays.hashCode(filterBitMap) : 0); + return result; + } + + @Override + public String toString() { + return "CqExtUnit{" + + "size=" + size + + ", tagsCode=" + tagsCode + + ", msgStoreTime=" + msgStoreTime + + ", bitMapSize=" + bitMapSize + + ", filterBitMap=" + Arrays.toString(filterBitMap) + + '}'; + } + } +} diff --git a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageFilter.java b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageFilter.java index 13500261..9db87f31 100644 --- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageFilter.java +++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageFilter.java @@ -18,26 +18,33 @@ package org.apache.rocketmq.store; import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import java.nio.ByteBuffer; +import java.util.Map; + public class DefaultMessageFilter implements MessageFilter { - @Override - public boolean isMessageMatched(SubscriptionData subscriptionData, Long tagsCode) { - if (tagsCode == null) { - return true; - } + private SubscriptionData subscriptionData; - if (null == subscriptionData) { - return true; - } + public DefaultMessageFilter(final SubscriptionData subscriptionData) { + this.subscriptionData = subscriptionData; + } - if (subscriptionData.isClassFilterMode()) + @Override + public boolean isMatchedByConsumeQueue(Long tagsCode, ConsumeQueueExt.CqExtUnit cqExtUnit) { + if (null == tagsCode || null == subscriptionData) { return true; + } - if (subscriptionData.getSubString().equals(SubscriptionData.SUB_ALL)) { + if (subscriptionData.isClassFilterMode()) { return true; } - return subscriptionData.getCodeSet().contains(tagsCode.intValue()); + return subscriptionData.getSubString().equals(SubscriptionData.SUB_ALL) + || subscriptionData.getCodeSet().contains(tagsCode.intValue()); } + @Override + public boolean isMatchedByCommitLog(ByteBuffer msgBuffer, Map properties) { + return true; + } } diff --git a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java index 0edfeec4..7bed62c7 100644 --- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedList; import java.util.Map; import java.util.Map.Entry; import java.util.Set; @@ -41,7 +42,6 @@ import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.message.MessageDecoder; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.message.MessageExtBatch; -import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; import org.apache.rocketmq.common.running.RunningStats; import org.apache.rocketmq.common.sysflag.MessageSysFlag; import org.apache.rocketmq.store.config.BrokerRole; @@ -60,8 +60,6 @@ import static org.apache.rocketmq.store.config.BrokerRole.SLAVE; public class DefaultMessageStore implements MessageStore { private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); - private final MessageFilter messageFilter = new DefaultMessageFilter(); - private final MessageStoreConfig messageStoreConfig; // CommitLog private final CommitLog commitLog; @@ -103,6 +101,8 @@ public class DefaultMessageStore implements MessageStore { private AtomicLong printTimes = new AtomicLong(0); + private final LinkedList dispatcherList; + public DefaultMessageStore(final MessageStoreConfig messageStoreConfig, final BrokerStatsManager brokerStatsManager, final MessageArrivingListener messageArrivingListener, final BrokerConfig brokerConfig) throws IOException { this.messageArrivingListener = messageArrivingListener; @@ -133,6 +133,10 @@ public class DefaultMessageStore implements MessageStore { this.allocateMappedFileService.start(); this.indexService.start(); + + this.dispatcherList = new LinkedList<>(); + this.dispatcherList.addLast(new CommitLogDispatcherBuildConsumeQueue()); + this.dispatcherList.addLast(new CommitLogDispatcherBuildIndex()); } public void truncateDirtyLogicFiles(long phyOffset) { @@ -409,7 +413,7 @@ public class DefaultMessageStore implements MessageStore { } public GetMessageResult getMessage(final String group, final String topic, final int queueId, final long offset, final int maxMsgNums, - final SubscriptionData subscriptionData) { + final MessageFilter messageFilter) { if (this.shutdown) { log.warn("message store has shutdown, so getMessage is forbidden"); return null; @@ -464,6 +468,7 @@ public class DefaultMessageStore implements MessageStore { int i = 0; final int maxFilterMessageCount = Math.max(16000, maxMsgNums * ConsumeQueue.CQ_STORE_UNIT_SIZE); final boolean diskFallRecorded = this.messageStoreConfig.isDiskFallRecorded(); + ConsumeQueueExt.CqExtUnit cqExtUnit = new ConsumeQueueExt.CqExtUnit(); for (; i < bufferConsumeQueue.getSize() && i < maxFilterMessageCount; i += ConsumeQueue.CQ_STORE_UNIT_SIZE) { long offsetPy = bufferConsumeQueue.getByteBuffer().getLong(); int sizePy = bufferConsumeQueue.getByteBuffer().getInt(); @@ -483,29 +488,51 @@ public class DefaultMessageStore implements MessageStore { break; } - if (this.messageFilter.isMessageMatched(subscriptionData, tagsCode)) { - SelectMappedBufferResult selectResult = this.commitLog.getMessage(offsetPy, sizePy); - if (selectResult != null) { - this.storeStatsService.getGetMessageTransferedMsgCount().incrementAndGet(); - getResult.addMessage(selectResult); - status = GetMessageStatus.FOUND; - nextPhyFileStartOffset = Long.MIN_VALUE; + boolean extRet = false; + if (consumeQueue.isExtAddr(tagsCode)) { + extRet = consumeQueue.getExt(tagsCode, cqExtUnit); + if (extRet) { + tagsCode = cqExtUnit.getTagsCode(); } else { - if (getResult.getBufferTotalSize() == 0) { - status = GetMessageStatus.MESSAGE_WAS_REMOVING; - } - - nextPhyFileStartOffset = this.commitLog.rollNextFile(offsetPy); + // can't find ext content.Client will filter messages by tag also. + log.error("[BUG] can't find consume queue extend file content!addr={}, offsetPy={}, sizePy={}, topic={}, group={}", + tagsCode, offsetPy, sizePy, topic, group); } - } else { + } + + if (messageFilter != null + && !messageFilter.isMatchedByConsumeQueue(tagsCode, extRet ? cqExtUnit : null)) { if (getResult.getBufferTotalSize() == 0) { status = GetMessageStatus.NO_MATCHED_MESSAGE; } - if (log.isDebugEnabled()) { - log.debug("message type not matched, client: " + subscriptionData + " server: " + tagsCode); + continue; + } + + SelectMappedBufferResult selectResult = this.commitLog.getMessage(offsetPy, sizePy); + if (null == selectResult) { + if (getResult.getBufferTotalSize() == 0) { + status = GetMessageStatus.MESSAGE_WAS_REMOVING; + } + + nextPhyFileStartOffset = this.commitLog.rollNextFile(offsetPy); + continue; + } + + if (messageFilter != null + && !messageFilter.isMatchedByCommitLog(selectResult.getByteBuffer().slice(), null)) { + if (getResult.getBufferTotalSize() == 0) { + status = GetMessageStatus.NO_MATCHED_MESSAGE; } + // release... + selectResult.release(); + continue; } + + this.storeStatsService.getGetMessageTransferedMsgCount().incrementAndGet(); + getResult.addMessage(selectResult); + status = GetMessageStatus.FOUND; + nextPhyFileStartOffset = Long.MIN_VALUE; } if (diskFallRecorded) { @@ -1318,27 +1345,14 @@ public class DefaultMessageStore implements MessageStore { } public void doDispatch(DispatchRequest req) { - final int tranType = MessageSysFlag.getTransactionValue(req.getSysFlag()); - switch (tranType) { - case MessageSysFlag.TRANSACTION_NOT_TYPE: - case MessageSysFlag.TRANSACTION_COMMIT_TYPE: - DefaultMessageStore.this.putMessagePositionInfo(req.getTopic(), req.getQueueId(), req.getCommitLogOffset(), req.getMsgSize(), - req.getTagsCode(), req.getStoreTimestamp(), req.getConsumeQueueOffset()); - break; - case MessageSysFlag.TRANSACTION_PREPARED_TYPE: - case MessageSysFlag.TRANSACTION_ROLLBACK_TYPE: - break; - } - - if (DefaultMessageStore.this.getMessageStoreConfig().isMessageIndexEnable()) { - DefaultMessageStore.this.indexService.buildIndex(req); + for (CommitLogDispatcher dispatcher : this.dispatcherList) { + dispatcher.dispatch(req); } } - public void putMessagePositionInfo(String topic, int queueId, long offset, int size, long tagsCode, long storeTimestamp, - long logicOffset) { - ConsumeQueue cq = this.findConsumeQueue(topic, queueId); - cq.putMessagePositionInfoWrapper(offset, size, tagsCode, storeTimestamp, logicOffset); + public void putMessagePositionInfo(DispatchRequest dispatchRequest) { + ConsumeQueue cq = this.findConsumeQueue(dispatchRequest.getTopic(), dispatchRequest.getQueueId()); + cq.putMessagePositionInfoWrapper(dispatchRequest); } public BrokerStatsManager getBrokerStatsManager() { @@ -1354,6 +1368,20 @@ public class DefaultMessageStore implements MessageStore { return remainTransientStoreBufferNumbs() == 0; } + @Override + public LinkedList getDispatcherList() { + return this.dispatcherList; + } + + @Override + public ConsumeQueue getConsumeQueue(String topic, int queueId) { + ConcurrentHashMap map = consumeQueueTable.get(topic); + if (map == null) { + return null; + } + return map.get(queueId); + } + public void unlockMappedFile(final MappedFile mappedFile) { this.scheduledExecutorService.schedule(new Runnable() { @Override @@ -1363,6 +1391,33 @@ public class DefaultMessageStore implements MessageStore { }, 6, TimeUnit.SECONDS); } + class CommitLogDispatcherBuildConsumeQueue implements CommitLogDispatcher { + + @Override + public void dispatch(DispatchRequest request) { + final int tranType = MessageSysFlag.getTransactionValue(request.getSysFlag()); + switch (tranType) { + case MessageSysFlag.TRANSACTION_NOT_TYPE: + case MessageSysFlag.TRANSACTION_COMMIT_TYPE: + DefaultMessageStore.this.putMessagePositionInfo(request); + break; + case MessageSysFlag.TRANSACTION_PREPARED_TYPE: + case MessageSysFlag.TRANSACTION_ROLLBACK_TYPE: + break; + } + } + } + + class CommitLogDispatcherBuildIndex implements CommitLogDispatcher { + + @Override + public void dispatch(DispatchRequest request) { + if (DefaultMessageStore.this.messageStoreConfig.isMessageIndexEnable()) { + DefaultMessageStore.this.indexService.buildIndex(request); + } + } + } + class CleanCommitLogService { private final static int MAX_MANUAL_DELETE_FILE_TIMES = 20; @@ -1695,7 +1750,8 @@ public class DefaultMessageStore implements MessageStore { && DefaultMessageStore.this.brokerConfig.isLongPollingEnable()) { DefaultMessageStore.this.messageArrivingListener.arriving(dispatchRequest.getTopic(), dispatchRequest.getQueueId(), dispatchRequest.getConsumeQueueOffset() + 1, - dispatchRequest.getTagsCode()); + dispatchRequest.getTagsCode(), dispatchRequest.getStoreTimestamp(), + dispatchRequest.getBitMap(), dispatchRequest.getPropertiesMap()); } // FIXED BUG By shijia this.reputFromOffset += size; diff --git a/store/src/main/java/org/apache/rocketmq/store/DispatchRequest.java b/store/src/main/java/org/apache/rocketmq/store/DispatchRequest.java index 33529dae..3d33eaf3 100644 --- a/store/src/main/java/org/apache/rocketmq/store/DispatchRequest.java +++ b/store/src/main/java/org/apache/rocketmq/store/DispatchRequest.java @@ -16,6 +16,8 @@ */ package org.apache.rocketmq.store; +import java.util.Map; + public class DispatchRequest { private final String topic; private final int queueId; @@ -30,6 +32,8 @@ public class DispatchRequest { private final int sysFlag; private final long preparedTransactionOffset; + private final Map propertiesMap; + private byte[] bitMap; public DispatchRequest( final String topic, @@ -42,7 +46,8 @@ public class DispatchRequest { final String keys, final String uniqKey, final int sysFlag, - final long preparedTransactionOffset + final long preparedTransactionOffset, + final Map propertiesMap ) { this.topic = topic; this.queueId = queueId; @@ -57,6 +62,7 @@ public class DispatchRequest { this.sysFlag = sysFlag; this.preparedTransactionOffset = preparedTransactionOffset; this.success = true; + this.propertiesMap = propertiesMap; } public DispatchRequest(int size) { @@ -81,6 +87,7 @@ public class DispatchRequest { this.sysFlag = 0; this.preparedTransactionOffset = 0; this.success = false; + this.propertiesMap = null; } public DispatchRequest(int size, boolean success) { @@ -105,6 +112,7 @@ public class DispatchRequest { this.sysFlag = 0; this.preparedTransactionOffset = 0; this.success = success; + this.propertiesMap = null; } public String getTopic() { @@ -155,4 +163,15 @@ public class DispatchRequest { return uniqKey; } + public Map getPropertiesMap() { + return propertiesMap; + } + + public byte[] getBitMap() { + return bitMap; + } + + public void setBitMap(byte[] bitMap) { + this.bitMap = bitMap; + } } diff --git a/store/src/main/java/org/apache/rocketmq/store/MappedFile.java b/store/src/main/java/org/apache/rocketmq/store/MappedFile.java index 550e578c..a9a00a85 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MappedFile.java +++ b/store/src/main/java/org/apache/rocketmq/store/MappedFile.java @@ -244,6 +244,31 @@ public class MappedFile extends ReferenceResource { return false; } + /** + * Content of data from offset to offset + length will be wrote to file. + * + * @param data + * @param offset The offset of the subarray to be used. + * @param length The length of the subarray to be used. + * @return + */ + public boolean appendMessage(final byte[] data, final int offset, final int length) { + int currentPos = this.wrotePosition.get(); + + if ((currentPos + length) <= this.fileSize) { + try { + this.fileChannel.position(currentPos); + this.fileChannel.write(ByteBuffer.wrap(data, offset, length)); + } catch (Throwable e) { + log.error("Error occurred when append message to mappedFile.", e); + } + this.wrotePosition.addAndGet(length); + return true; + } + + return false; + } + /** * @param flushLeastPages * @return The current flushed position diff --git a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java index 5c6c62c6..a8fa3648 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java +++ b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java @@ -121,7 +121,7 @@ public class MappedFileQueue { this.deleteExpiredFile(willRemoveFiles); } - private void deleteExpiredFile(List files) { + void deleteExpiredFile(List files) { if (!files.isEmpty()) { diff --git a/store/src/main/java/org/apache/rocketmq/store/MessageArrivingListener.java b/store/src/main/java/org/apache/rocketmq/store/MessageArrivingListener.java index 2523c1ac..dee1bc75 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MessageArrivingListener.java +++ b/store/src/main/java/org/apache/rocketmq/store/MessageArrivingListener.java @@ -17,6 +17,9 @@ package org.apache.rocketmq.store; +import java.util.Map; + public interface MessageArrivingListener { - void arriving(String topic, int queueId, long logicOffset, long tagsCode); + void arriving(String topic, int queueId, long logicOffset, long tagsCode, + long msgStoreTime, byte[] filterBitMap, Map properties); } diff --git a/store/src/main/java/org/apache/rocketmq/store/MessageFilter.java b/store/src/main/java/org/apache/rocketmq/store/MessageFilter.java index 859ce994..6b34758b 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MessageFilter.java +++ b/store/src/main/java/org/apache/rocketmq/store/MessageFilter.java @@ -16,8 +16,30 @@ */ package org.apache.rocketmq.store; -import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import java.nio.ByteBuffer; +import java.util.Map; public interface MessageFilter { - boolean isMessageMatched(final SubscriptionData subscriptionData, final Long tagsCode); + /** + * match by tags code or filter bit map which is calculated when message received + * and stored in consume queue ext. + * + * @param tagsCode tagsCode + * @param cqExtUnit extend unit of consume queue + * @return + */ + boolean isMatchedByConsumeQueue(final Long tagsCode, + final ConsumeQueueExt.CqExtUnit cqExtUnit); + + /** + * match by message content which are stored in commit log. + *
    {@code msgBuffer} and {@code properties} are not all null.If invoked in store, + * {@code properties} is null;If invoked in {@code PullRequestHoldService}, {@code msgBuffer} is null. + * + * @param msgBuffer message buffer in commit log, may be null if not invoked in store. + * @param properties message properties, should decode from buffer if null by yourself. + * @return + */ + boolean isMatchedByCommitLog(final ByteBuffer msgBuffer, + final Map properties); } diff --git a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java index 65c546b7..e841c083 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java @@ -17,10 +17,10 @@ package org.apache.rocketmq.store; import java.util.HashMap; +import java.util.LinkedList; import java.util.Set; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.message.MessageExtBatch; -import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; public interface MessageStore { @@ -37,7 +37,7 @@ public interface MessageStore { PutMessageResult putMessages(final MessageExtBatch messageExtBatch); GetMessageResult getMessage(final String group, final String topic, final int queueId, - final long offset, final int maxMsgNums, final SubscriptionData subscriptionData); + final long offset, final int maxMsgNums, final MessageFilter messageFilter); long getMaxOffsetInQuque(final String topic, final int queueId); @@ -105,4 +105,8 @@ public interface MessageStore { long lockTimeMills(); boolean isTransientStorePoolDeficient(); + + LinkedList getDispatcherList(); + + ConsumeQueue getConsumeQueue(String topic, int queueId); } diff --git a/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java b/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java index 7ae2ab5c..29f800c2 100644 --- a/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java +++ b/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java @@ -34,6 +34,13 @@ public class MessageStoreConfig { private int mapedFileSizeCommitLog = 1024 * 1024 * 1024; // ConsumeQueue file size,default is 30W private int mapedFileSizeConsumeQueue = 300000 * ConsumeQueue.CQ_STORE_UNIT_SIZE; + // enable consume queue ext + private boolean enableConsumeQueueExt = false; + // ConsumeQueue extend file size, 48M + private int mappedFileSizeConsumeQueueExt = 48 * 1024 * 1024; + // Bit count of filter bit map. + // this will be set by pipe of calculate filter bit map. + private int bitMapLengthConsumeQueueExt = 64; // CommitLog flush interval // flush data to disk @@ -191,6 +198,30 @@ public class MessageStoreConfig { this.mapedFileSizeConsumeQueue = mapedFileSizeConsumeQueue; } + public boolean isEnableConsumeQueueExt() { + return enableConsumeQueueExt; + } + + public void setEnableConsumeQueueExt(boolean enableConsumeQueueExt) { + this.enableConsumeQueueExt = enableConsumeQueueExt; + } + + public int getMappedFileSizeConsumeQueueExt() { + return mappedFileSizeConsumeQueueExt; + } + + public void setMappedFileSizeConsumeQueueExt(int mappedFileSizeConsumeQueueExt) { + this.mappedFileSizeConsumeQueueExt = mappedFileSizeConsumeQueueExt; + } + + public int getBitMapLengthConsumeQueueExt() { + return bitMapLengthConsumeQueueExt; + } + + public void setBitMapLengthConsumeQueueExt(int bitMapLengthConsumeQueueExt) { + this.bitMapLengthConsumeQueueExt = bitMapLengthConsumeQueueExt; + } + public int getFlushIntervalCommitLog() { return flushIntervalCommitLog; } diff --git a/store/src/main/java/org/apache/rocketmq/store/config/StorePathConfigHelper.java b/store/src/main/java/org/apache/rocketmq/store/config/StorePathConfigHelper.java index aebebafa..ef1d670a 100644 --- a/store/src/main/java/org/apache/rocketmq/store/config/StorePathConfigHelper.java +++ b/store/src/main/java/org/apache/rocketmq/store/config/StorePathConfigHelper.java @@ -24,6 +24,10 @@ public class StorePathConfigHelper { return rootDir + File.separator + "consumequeue"; } + public static String getStorePathConsumeQueueExt(final String rootDir) { + return rootDir + File.separator + "consumequeue_ext"; + } + public static String getStorePathIndex(final String rootDir) { return rootDir + File.separator + "index"; } diff --git a/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java b/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java index e08a6f54..d45b994a 100644 --- a/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java +++ b/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java @@ -32,6 +32,7 @@ import org.apache.rocketmq.common.message.MessageDecoder; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.running.RunningStats; import org.apache.rocketmq.store.ConsumeQueue; +import org.apache.rocketmq.store.ConsumeQueueExt; import org.apache.rocketmq.store.DefaultMessageStore; import org.apache.rocketmq.store.MessageExtBrokerInner; import org.apache.rocketmq.store.PutMessageResult; @@ -248,11 +249,24 @@ public class ScheduleMessageService extends ConfigManager { try { long nextOffset = offset; int i = 0; + ConsumeQueueExt.CqExtUnit cqExtUnit = new ConsumeQueueExt.CqExtUnit(); for (; i < bufferCQ.getSize(); i += ConsumeQueue.CQ_STORE_UNIT_SIZE) { long offsetPy = bufferCQ.getByteBuffer().getLong(); int sizePy = bufferCQ.getByteBuffer().getInt(); long tagsCode = bufferCQ.getByteBuffer().getLong(); + if (cq.isExtAddr(tagsCode)) { + if (cq.getExt(tagsCode, cqExtUnit)) { + tagsCode = cqExtUnit.getTagsCode(); + } else { + //can't find ext content.So re compute tags code. + log.error("[BUG] can't find consume queue extend file content!addr={}, offsetPy={}, sizePy={}", + tagsCode, offsetPy, sizePy); + long msgStoreTime = defaultMessageStore.getCommitLog().pickupStoreTimestamp(offsetPy, sizePy); + tagsCode = computeDeliverTimestamp(delayLevel, msgStoreTime); + } + } + long now = System.currentTimeMillis(); long deliverTimestamp = this.correctDeliverTimestamp(now, tagsCode); diff --git a/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueExtTest.java b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueExtTest.java new file mode 100644 index 00000000..5dbc5848 --- /dev/null +++ b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueExtTest.java @@ -0,0 +1,251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.store; + +import org.junit.Test; + +import java.io.File; +import java.util.Random; + +import static org.assertj.core.api.Assertions.assertThat; + +public class ConsumeQueueExtTest { + + private static final String topic = "abc"; + private static final int queueId = 0; + private static final String storePath = "." + File.separator + "unit_test_store"; + private static final int bitMapLength = 64; + private static final int unitSizeWithBitMap = ConsumeQueueExt.CqExtUnit.MIN_EXT_UNIT_SIZE + bitMapLength / Byte.SIZE; + private static final int cqExtFileSize = 10 * unitSizeWithBitMap; + private static final int unitCount = 20; + + + protected ConsumeQueueExt genExt() { + return new ConsumeQueueExt( + topic, queueId, storePath, cqExtFileSize, bitMapLength + ); + } + + protected byte[] genBitMap(int bitMapLength) { + byte[] bytes = new byte[bitMapLength / Byte.SIZE]; + + Random random = new Random(System.currentTimeMillis()); + random.nextBytes(bytes); + + return bytes; + } + + protected ConsumeQueueExt.CqExtUnit genUnit(boolean hasBitMap) { + ConsumeQueueExt.CqExtUnit cqExtUnit = new ConsumeQueueExt.CqExtUnit(); + + cqExtUnit.setTagsCode(Math.abs((new Random(System.currentTimeMillis())).nextInt())); + cqExtUnit.setMsgStoreTime(System.currentTimeMillis()); + if (hasBitMap) { + cqExtUnit.setFilterBitMap(genBitMap(bitMapLength)); + } + + return cqExtUnit; + } + + 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 void putSth(ConsumeQueueExt consumeQueueExt, boolean getAfterPut, + boolean unitSameSize, int unitCount) { + for (int i = 0; i < unitCount; i++) { + ConsumeQueueExt.CqExtUnit putUnit = + unitSameSize ? genUnit(true) : genUnit(i % 2 == 0); + + long addr = consumeQueueExt.put(putUnit); + assertThat(addr).isLessThan(0); + + if (getAfterPut) { + ConsumeQueueExt.CqExtUnit getUnit = consumeQueueExt.get(addr); + + assertThat(getUnit).isNotNull(); + assertThat(putUnit).isEqualTo(getUnit); + } + + try { + Thread.sleep(10); + } catch (InterruptedException e) { + e.printStackTrace(); + assertThat(false).isTrue(); + } + } + } + + @Test + public void testPut() { + ConsumeQueueExt consumeQueueExt = genExt(); + + try { + putSth(consumeQueueExt, true, false, unitCount); + } finally { + consumeQueueExt.destroy(); + deleteDirectory(storePath); + } + } + + @Test + public void testGet() { + ConsumeQueueExt consumeQueueExt = genExt(); + + putSth(consumeQueueExt, false, false, unitCount); + + try { + // from start. + long addr = consumeQueueExt.decorate(0); + + ConsumeQueueExt.CqExtUnit unit = new ConsumeQueueExt.CqExtUnit(); + while (true) { + boolean ret = consumeQueueExt.get(addr, unit); + + if (!ret) { + break; + } + + assertThat(unit.getSize()).isGreaterThanOrEqualTo(ConsumeQueueExt.CqExtUnit.MIN_EXT_UNIT_SIZE); + + addr += unit.getSize(); + } + } finally { + consumeQueueExt.destroy(); + deleteDirectory(storePath); + } + } + + @Test + public void testGet_invalidAddress() { + ConsumeQueueExt consumeQueueExt = genExt(); + + putSth(consumeQueueExt, false, true, unitCount); + + try { + ConsumeQueueExt.CqExtUnit unit = consumeQueueExt.get(0); + + assertThat(unit).isNull(); + + long addr = (cqExtFileSize / unitSizeWithBitMap) * unitSizeWithBitMap; + addr += unitSizeWithBitMap; + + unit = consumeQueueExt.get(addr); + assertThat(unit).isNull(); + } finally { + consumeQueueExt.destroy(); + deleteDirectory(storePath); + } + } + + @Test + public void testRecovery() { + ConsumeQueueExt putCqExt = genExt(); + + putSth(putCqExt, false, true, unitCount); + + ConsumeQueueExt loadCqExt = genExt(); + + loadCqExt.load(); + + loadCqExt.recover(); + + try { + assertThat(loadCqExt.getMinAddress()).isEqualTo(Long.MIN_VALUE); + + // same unit size. + int countPerFile = (cqExtFileSize - ConsumeQueueExt.END_BLANK_DATA_LENGTH) / unitSizeWithBitMap; + + int lastFileUnitCount = unitCount % countPerFile; + + int fileCount = unitCount / countPerFile + 1; + if (lastFileUnitCount == 0) { + fileCount -= 1; + } + + if (lastFileUnitCount == 0) { + assertThat(loadCqExt.unDecorate(loadCqExt.getMaxAddress()) % cqExtFileSize).isEqualTo(0); + } else { + assertThat(loadCqExt.unDecorate(loadCqExt.getMaxAddress())) + .isEqualTo(lastFileUnitCount * unitSizeWithBitMap + (fileCount - 1) * cqExtFileSize); + } + } finally { + putCqExt.destroy(); + loadCqExt.destroy(); + deleteDirectory(storePath); + } + } + + @Test + public void testTruncateByMinOffset() { + ConsumeQueueExt consumeQueueExt = genExt(); + + putSth(consumeQueueExt, false, true, unitCount * 2); + + try { + // truncate first one file. + long address = consumeQueueExt.decorate((long) (cqExtFileSize * 1.5)); + + long expectMinAddress = consumeQueueExt.decorate(cqExtFileSize); + + consumeQueueExt.truncateByMinAddress(address); + + long minAddress = consumeQueueExt.getMinAddress(); + + assertThat(expectMinAddress).isEqualTo(minAddress); + } finally { + consumeQueueExt.destroy(); + deleteDirectory(storePath); + } + } + + @Test + public void testTruncateByMaxOffset() { + ConsumeQueueExt consumeQueueExt = genExt(); + + putSth(consumeQueueExt, false, true, unitCount * 2); + + try { + // truncate, only first 3 files exist. + long address = consumeQueueExt.decorate(cqExtFileSize * 2 + unitSizeWithBitMap); + + long expectMaxAddress = address + unitSizeWithBitMap; + + consumeQueueExt.truncateByMaxAddress(address); + + long maxAddress = consumeQueueExt.getMaxAddress(); + + assertThat(expectMaxAddress).isEqualTo(maxAddress); + } finally { + consumeQueueExt.destroy(); + deleteDirectory(storePath); + } + } +} diff --git a/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java new file mode 100644 index 00000000..9c42fb91 --- /dev/null +++ b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.store; + +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.message.MessageDecoder; +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.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +public class ConsumeQueueTest { + + private static final String msg = "Once, there was a chance for me!"; + private static final byte[] msgBody = msg.getBytes(); + + private static final String topic = "abc"; + private static final int queueId = 0; + private static final String storePath = "." + File.separator + "unit_test_store"; + private static final int commitLogFileSize = 1024 * 8; + private static final int cqFileSize = 10 * 20; + private static final int cqExtFileSize = 10 * (ConsumeQueueExt.CqExtUnit.MIN_EXT_UNIT_SIZE + 64); + + 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 = 0; i < 1; 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() throws Exception { + MessageStoreConfig messageStoreConfig = buildStoreConfig( + commitLogFileSize, cqFileSize, true, cqExtFileSize + ); + + BrokerConfig brokerConfig = new BrokerConfig(); + + DefaultMessageStore master = new DefaultMessageStore( + messageStoreConfig, + new BrokerStatsManager(brokerConfig.getBrokerClusterName()), + new MessageArrivingListener() { + @Override + public void arriving(String topic, int queueId, long logicOffset, long tagsCode, + long msgStoreTime, byte[] filterBitMap, Map properties) { + } + } + , brokerConfig); + + assertThat(master.load()).isTrue(); + + master.start(); + + return master; + } + + protected void putMsg(DefaultMessageStore master) throws Exception { + long totalMsgs = 200; + + for (long i = 0; i < totalMsgs; i++) { + master.putMessage(buildMessage()); + } + } + + 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(); + } + + @Test + public void testConsumeQueueWithExtendData() { + DefaultMessageStore master = null; + try { + master = gen(); + } catch (Exception e) { + e.printStackTrace(); + assertThat(Boolean.FALSE).isTrue(); + } + + master.getDispatcherList().addFirst(new CommitLogDispatcher() { + + @Override + public void dispatch(DispatchRequest request) { + runCount++; + } + + private int runCount = 0; + }); + + try { + try { + putMsg(master); + // wait build consume queue + Thread.sleep(1000); + } catch (Exception e) { + e.printStackTrace(); + assertThat(Boolean.FALSE).isTrue(); + } + + ConsumeQueue cq = master.getConsumeQueueTable().get(topic).get(queueId); + + assertThat(cq).isNotNull(); + + long index = 0; + + while (index < cq.getMaxOffsetInQueue()) { + SelectMappedBufferResult bufferResult = cq.getIndexBuffer(index); + + assertThat(bufferResult).isNotNull(); + + ByteBuffer buffer = bufferResult.getByteBuffer(); + + assertThat(buffer).isNotNull(); + try { + ConsumeQueueExt.CqExtUnit cqExtUnit = new ConsumeQueueExt.CqExtUnit(); + for (int i = 0; i < bufferResult.getSize(); i += ConsumeQueue.CQ_STORE_UNIT_SIZE) { + long phyOffset = buffer.getLong(); + int size = buffer.getInt(); + long tagsCode = buffer.getLong(); + + assertThat(phyOffset).isGreaterThanOrEqualTo(0); + assertThat(size).isGreaterThan(0); + assertThat(tagsCode).isLessThan(0); + + boolean ret = cq.getExt(tagsCode, cqExtUnit); + + assertThat(ret).isTrue(); + assertThat(cqExtUnit).isNotNull(); + assertThat(cqExtUnit.getSize()).isGreaterThan((short) 0); + assertThat(cqExtUnit.getMsgStoreTime()).isGreaterThan(0); + assertThat(cqExtUnit.getTagsCode()).isGreaterThan(0); + } + + } finally { + bufferResult.release(); + } + + index += cqFileSize / ConsumeQueue.CQ_STORE_UNIT_SIZE; + } + } finally { + master.shutdown(); + master.destroy(); + deleteDirectory(storePath); + } + } +} diff --git a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java index 5c9c46f7..75f1de94 100644 --- a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java @@ -20,6 +20,7 @@ package org.apache.rocketmq.store; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.SocketAddress; +import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.store.config.FlushDiskType; @@ -124,7 +125,8 @@ public class DefaultMessageStoreTest { private class MyMessageArrivingListener implements MessageArrivingListener { @Override - public void arriving(String topic, int queueId, long logicOffset, long tagsCode) { + public void arriving(String topic, int queueId, long logicOffset, long tagsCode, long msgStoreTime, + byte[] filterBitMap, Map properties) { } } } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java index 19bff895..409ea332 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java @@ -41,6 +41,7 @@ import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo; import org.apache.rocketmq.common.protocol.body.GroupList; import org.apache.rocketmq.common.protocol.body.KVTable; import org.apache.rocketmq.common.protocol.body.ProducerConnection; +import org.apache.rocketmq.common.protocol.body.QueryConsumeQueueResponseBody; import org.apache.rocketmq.common.protocol.body.QueueTimeSpan; import org.apache.rocketmq.common.protocol.body.SubscriptionGroupWrapper; import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper; @@ -469,4 +470,12 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt { UnsupportedEncodingException { return this.defaultMQAdminExtImpl.getNameServerConfig(nameServers); } + + @Override + public QueryConsumeQueueResponseBody queryConsumeQueue(String brokerAddr, String topic, int queueId, long index, int count, String consumerGroup) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException { + return this.defaultMQAdminExtImpl.queryConsumeQueue( + brokerAddr, topic, queueId, index, count, consumerGroup + ); + } } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java index a31b69d9..157ae21e 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java @@ -63,6 +63,7 @@ import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo; import org.apache.rocketmq.common.protocol.body.GroupList; import org.apache.rocketmq.common.protocol.body.KVTable; import org.apache.rocketmq.common.protocol.body.ProducerConnection; +import org.apache.rocketmq.common.protocol.body.QueryConsumeQueueResponseBody; import org.apache.rocketmq.common.protocol.body.QueueTimeSpan; import org.apache.rocketmq.common.protocol.body.SubscriptionGroupWrapper; import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper; @@ -955,4 +956,11 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner { return this.mqClientInstance.getMQClientAPIImpl().getNameServerConfig(nameServers, timeoutMillis); } + @Override + public QueryConsumeQueueResponseBody queryConsumeQueue(String brokerAddr, String topic, int queueId, long index, int count, String consumerGroup) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException { + return this.mqClientInstance.getMQClientAPIImpl().queryConsumeQueue( + brokerAddr, topic, queueId, index, count, consumerGroup, timeoutMillis + ); + } } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java index 493cf540..82add92b 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java @@ -39,6 +39,7 @@ import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo; import org.apache.rocketmq.common.protocol.body.GroupList; import org.apache.rocketmq.common.protocol.body.KVTable; import org.apache.rocketmq.common.protocol.body.ProducerConnection; +import org.apache.rocketmq.common.protocol.body.QueryConsumeQueueResponseBody; import org.apache.rocketmq.common.protocol.body.QueueTimeSpan; import org.apache.rocketmq.common.protocol.body.SubscriptionGroupWrapper; import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper; @@ -241,4 +242,25 @@ public interface MQAdminExt extends MQAdmin { Map getNameServerConfig(final List nameServers) throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException, UnsupportedEncodingException; + + /** + * query consume queue data + * + * @param brokerAddr broker ip address + * @param topic topic + * @param queueId id of queue + * @param index start offset + * @param count how many + * @param consumerGroup group + * @return + * @throws InterruptedException + * @throws RemotingTimeoutException + * @throws RemotingSendRequestException + * @throws RemotingConnectException + * @throws MQClientException + */ + QueryConsumeQueueResponseBody queryConsumeQueue(final String brokerAddr, + final String topic, final int queueId, + final long index, final int count, final String consumerGroup) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException; } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java index 9bd37e84..6398291a 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java @@ -59,6 +59,7 @@ import org.apache.rocketmq.tools.command.namesrv.UpdateNamesrvConfigCommand; import org.apache.rocketmq.tools.command.namesrv.WipeWritePermSubCommand; import org.apache.rocketmq.tools.command.offset.CloneGroupOffsetCommand; import org.apache.rocketmq.tools.command.offset.ResetOffsetByTimeCommand; +import org.apache.rocketmq.tools.command.queue.QueryConsumeQueueCommand; import org.apache.rocketmq.tools.command.stats.StatsAllSubCommand; import org.apache.rocketmq.tools.command.topic.AllocateMQSubCommand; import org.apache.rocketmq.tools.command.topic.DeleteTopicSubCommand; @@ -189,6 +190,8 @@ public class MQAdminStartup { initCommand(new GetNamesrvConfigCommand()); initCommand(new UpdateNamesrvConfigCommand()); initCommand(new GetBrokerConfigCommand()); + + initCommand(new QueryConsumeQueueCommand()); } private static void initLogback() throws JoranException { diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/queue/QueryConsumeQueueCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/queue/QueryConsumeQueueCommand.java new file mode 100644 index 00000000..611addd5 --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/queue/QueryConsumeQueueCommand.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.tools.command.queue; + +import com.alibaba.fastjson.JSON; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.PosixParser; +import org.apache.rocketmq.common.protocol.body.ConsumeQueueData; +import org.apache.rocketmq.common.protocol.body.QueryConsumeQueueResponseBody; +import org.apache.rocketmq.common.protocol.route.TopicRouteData; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.srvutil.ServerUtil; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.SubCommand; + +public class QueryConsumeQueueCommand implements SubCommand { + + public static void main(String[] args) { + QueryConsumeQueueCommand cmd = new QueryConsumeQueueCommand(); + + Options options = ServerUtil.buildCommandlineOptions(new Options()); + String[] subargs = new String[]{"-t TopicTest", "-q 0", "-i 6447", "-b 100.81.165.119:10911"}; + final CommandLine commandLine = + ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), + new PosixParser()); + cmd.execute(commandLine, options, null); + } + + @Override + public String commandName() { + return "queryCq"; + } + + @Override + public String commandDesc() { + return "Query cq command."; + } + + @Override + public Options buildCommandlineOptions(Options options) { + Option opt = new Option("t", "topic", true, "topic name"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("q", "queue", true, "queue num, ie. 1"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("i", "index", true, "start queue index."); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("c", "count", true, "how many."); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("b", "broker", true, "broker addr."); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("g", "consumer", true, "consumer group."); + opt.setRequired(false); + options.addOption(opt); + + return options; + } + + @Override + public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) { + DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(rpcHook); + + defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis())); + + try { + defaultMQAdminExt.start(); + + String topic = commandLine.getOptionValue("t").trim(); + int queueId = Integer.valueOf(commandLine.getOptionValue("q").trim()); + long index = Long.valueOf(commandLine.getOptionValue("i").trim()); + int count = Integer.valueOf(commandLine.getOptionValue("c", "10").trim()); + String broker = null; + if (commandLine.hasOption("b")) { + broker = commandLine.getOptionValue("b").trim(); + } + String consumerGroup = null; + if (commandLine.hasOption("g")) { + consumerGroup = commandLine.getOptionValue("g").trim(); + } + + if (broker == null || broker == "") { + TopicRouteData topicRouteData = defaultMQAdminExt.examineTopicRouteInfo(topic); + + if (topicRouteData == null || topicRouteData.getBrokerDatas() == null + || topicRouteData.getBrokerDatas().isEmpty()) { + throw new Exception("No topic route data!"); + } + + broker = topicRouteData.getBrokerDatas().get(0).getBrokerAddrs().get(0L); + } + + QueryConsumeQueueResponseBody queryConsumeQueueResponseBody = defaultMQAdminExt.queryConsumeQueue( + broker, topic, queueId, index, count, consumerGroup + ); + + if (queryConsumeQueueResponseBody.getSubscriptionData() != null) { + System.out.printf("Subscription data: \n%s\n", JSON.toJSONString(queryConsumeQueueResponseBody.getSubscriptionData(), true)); + System.out.print("======================================\n"); + } + + if (queryConsumeQueueResponseBody.getFilterData() != null) { + System.out.printf("Filter data: \n%s\n", queryConsumeQueueResponseBody.getFilterData()); + System.out.print("======================================\n"); + } + + System.out.printf("Queue data: \nmax: %d, min: %d\n", queryConsumeQueueResponseBody.getMaxQueueIndex(), + queryConsumeQueueResponseBody.getMinQueueIndex()); + System.out.print("======================================\n"); + + if (queryConsumeQueueResponseBody.getQueueData() != null) { + + long i = index; + for (ConsumeQueueData queueData : queryConsumeQueueResponseBody.getQueueData()) { + StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append("idx: " + i + "\n"); + + stringBuilder.append(queueData.toString() + "\n"); + + stringBuilder.append("======================================\n"); + + System.out.print(stringBuilder.toString()); + i++; + } + + } + + } catch (Exception e) { + e.printStackTrace(); + } finally { + defaultMQAdminExt.shutdown(); + } + } +} -- GitLab