You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by do...@apache.org on 2021/08/16 06:47:44 UTC
[incubator-inlong] branch INLONG-570 updated: [INLONG-625] Add unit
tests for metamanage.metastore.impl.* (#1438)
This is an automated email from the ASF dual-hosted git repository.
dockerzhang pushed a commit to branch INLONG-570
in repository https://gitbox.apache.org/repos/asf/incubator-inlong.git
The following commit(s) were added to refs/heads/INLONG-570 by this push:
new 141ba81 [INLONG-625] Add unit tests for metamanage.metastore.impl.* (#1438)
141ba81 is described below
commit 141ba819938a6b17cbc8cf257accb30be32f5adc
Author: gosonzhang <46...@qq.com>
AuthorDate: Mon Aug 16 14:47:15 2021 +0800
[INLONG-625] Add unit tests for metamanage.metastore.impl.* (#1438)
---
NOTICE | 2 +-
.../client/consumer/ConsumerSamplePrint.java | 9 +
.../inlong/tubemq/corebase/TBaseConstants.java | 3 +
.../inlong/tubemq/corebase/TokenConstants.java | 7 -
.../inlong/tubemq/corebase/cluster/TopicInfo.java | 16 +
.../tubemq/corebase/utils/AbstractSamplePrint.java | 4 +
.../tubemq/corebase/utils/KeyBuilderUtils.java | 62 +
.../tubemq/corebase/utils/SettingValidUtils.java | 13 +
.../inlong/tubemq/corebase/utils/Tuple2.java | 3 +-
.../inlong/tubemq/corebase/utils/Tuple3.java | 10 +-
.../corebase/utils/{Tuple3.java => Tuple4.java} | 25 +-
.../server/broker/msgstore/MessageStore.java | 8 +-
.../server/broker/utils/BrokerSamplePrint.java | 9 +
.../server/broker/utils/DiskSamplePrint.java | 9 +
.../server/broker/web/AbstractWebHandler.java | 12 +-
.../server/broker/web/BrokerAdminServlet.java | 302 +--
.../tubemq/server/common/TServerConstants.java | 46 +-
.../exception/LoadMetaException.java} | 67 +-
.../tubemq/server/common/fielddef/WebFieldDef.java | 95 +-
.../server/common/heartbeat/HeartbeatManager.java | 43 +-
.../server/common/paramcheck/PBParameterUtils.java | 75 +-
.../server/common/statusdef/CleanPolType.java} | 37 +-
.../server/common/statusdef/EnableStatus.java | 58 +
.../server/common/statusdef/ManageStatus.java | 126 ++
.../tubemq/server/common/statusdef/StepStatus.java | 75 +
.../server/common/statusdef/TopicStatus.java | 56 +
.../server/common/statusdef/TopicStsChgType.java | 54 +
.../tubemq/server/common/utils/ProcessResult.java | 36 +-
.../utils/SerialIdUtils.java} | 71 +-
.../server/common/utils/WebParameterUtils.java | 1599 ++++++++------
.../inlong/tubemq/server/master/TMaster.java | 1019 ++-------
.../server/master/balance/DefaultLoadBalancer.java | 107 +-
.../tubemq/server/master/balance/LoadBalancer.java | 24 +-
.../server/master/bdbstore/BdbStoreService.java | 113 -
.../master/bdbstore/DefaultBdbStoreService.java | 1560 -------------
.../bdbstore/bdbentitys/BdbBlackGroupEntity.java | 45 +
.../bdbstore/bdbentitys/BdbBrokerConfEntity.java | 92 +-
.../bdbentitys/BdbClusterSettingEntity.java | 115 +-
.../bdbentitys/BdbConsumeGroupSettingEntity.java | 20 +
.../bdbentitys/BdbConsumerGroupEntity.java | 5 +
.../bdbentitys/BdbGroupFilterCondEntity.java | 170 +-
.../bdbentitys/BdbGroupFlowCtrlEntity.java | 144 +-
.../bdbentitys/BdbTopicAuthControlEntity.java | 129 +-
.../bdbstore/bdbentitys/BdbTopicConfEntity.java | 55 +-
.../server/master/metamanage/DataOpErrCode.java | 63 +
.../server/master/metamanage/MetaDataManager.java | 2072 +++++++++++++++++
.../keepalive/AliveObserver.java} | 58 +-
.../keepalive/KeepAlive.java} | 74 +-
.../metastore/BdbMetaStoreServiceImpl.java | 1442 ++++++++++++
.../metamanage/metastore/MetaStoreService.java | 342 +++
.../metamanage/metastore/TStoreConstants.java | 52 +
.../metastore/dao/entity/BaseEntity.java | 326 +++
.../metastore/dao/entity/BrokerConfEntity.java | 465 ++++
.../metastore/dao/entity/ClusterSettingEntity.java | 398 ++++
.../dao/entity/GroupConsumeCtrlEntity.java | 326 +++
.../metastore/dao/entity/GroupResCtrlEntity.java | 386 ++++
.../metastore/dao/entity/TopicCtrlEntity.java | 294 +++
.../metastore/dao/entity/TopicDeployEntity.java | 366 +++
.../metastore/dao/entity/TopicPropGroup.java | 464 ++++
.../metastore/dao/mapper/AbstractMapper.java} | 62 +-
.../metastore/dao/mapper/BrokerConfigMapper.java | 47 +
.../metastore/dao/mapper/ClusterConfigMapper.java} | 65 +-
.../dao/mapper/GroupConsumeCtrlMapper.java | 62 +
.../metastore/dao/mapper/GroupResCtrlMapper.java | 47 +
.../metastore/dao/mapper/TopicCtrlMapper.java | 46 +
.../metastore/dao/mapper/TopicDeployMapper.java | 76 +
.../impl/bdbimpl/BdbBrokerConfigMapperImpl.java | 367 +++
.../impl/bdbimpl/BdbClusterConfigMapperImpl.java | 199 ++
.../bdbimpl/BdbGroupConsumeCtrlMapperImpl.java | 459 ++++
.../impl/bdbimpl/BdbGroupResCtrlMapperImpl.java | 225 ++
.../impl/bdbimpl/BdbTopicCtrlMapperImpl.java | 239 ++
.../impl/bdbimpl/BdbTopicDeployMapperImpl.java | 581 +++++
.../metastore/impl/bdbimpl/TBDBStoreTables.java | 33 +
...{BrokerInfoHolder.java => BrokerAbnHolder.java} | 261 +--
.../nodemanage/nodebroker/BrokerConfManager.java | 2081 -----------------
.../nodemanage/nodebroker/BrokerPSInfoHolder.java | 179 ++
.../nodemanage/nodebroker/BrokerRunManager.java | 105 +
.../nodemanage/nodebroker/BrokerRunStatusInfo.java | 409 ++++
.../nodemanage/nodebroker/BrokerSyncData.java | 344 +++
.../nodebroker/BrokerSyncStatusInfo.java | 825 -------
.../nodemanage/nodebroker/BrokerTopicInfoView.java | 426 ++++
.../nodemanage/nodebroker/DefBrokerRunManager.java | 516 +++++
.../nodemanage/nodebroker/TopicPSInfoManager.java | 184 +-
.../server/master/utils/BdbStoreSamplePrint.java | 9 +
.../master/utils/BrokerStatusSamplePrint.java | 82 +
.../server/master/web/MasterStatusCheckFilter.java | 12 +-
.../inlong/tubemq/server/master/web/WebServer.java | 3 +-
.../server/master/web/action/screen/Master.java | 33 +-
.../server/master/web/action/screen/Tubeweb.java | 6 +-
.../server/master/web/action/screen/Webapi.java | 89 +-
.../web/action/screen/cluster/ClusterManager.java | 6 +-
.../web/action/screen/config/BrokerList.java | 12 +-
.../master/web/handler/AbstractWebHandler.java | 6 +-
.../master/web/handler/BrokerProcessResult.java | 55 +
.../master/web/handler/GroupProcessResult.java} | 34 +-
.../master/web/handler/TopicProcessResult.java} | 32 +-
.../web/handler/WebAdminFlowRuleHandler.java | 610 ++---
.../web/handler/WebAdminGroupCtrlHandler.java | 2329 +++++++++-----------
.../web/handler/WebAdminTopicAuthHandler.java | 566 ++---
.../master/web/handler/WebBrokerConfHandler.java | 1118 ++++++++++
.../web/handler/WebBrokerDefConfHandler.java | 1660 --------------
.../web/handler/WebBrokerTopicConfHandler.java | 1548 -------------
.../web/handler/WebGroupConsumeCtrlHandler.java | 427 ++++
.../master/web/handler/WebGroupResCtrlHandler.java | 416 ++++
.../master/web/handler/WebMasterInfoHandler.java | 467 ++--
.../master/web/handler/WebOtherInfoHandler.java | 128 +-
.../master/web/handler/WebTopicCtrlHandler.java | 374 ++++
.../master/web/handler/WebTopicDeployHandler.java | 983 +++++++++
.../tubemq/server/common/HeartbeatManagerTest.java | 10 +-
.../tubemq/server/common/WebParameterTest.java | 50 -
.../server/common/WebParameterUtilsTest.java | 289 +++
.../metastore/dao/entity/BaseEntityTest.java | 157 ++
.../metastore/dao/entity/BrokerConfEntityTest.java | 198 ++
.../dao/entity/ClusterSettingEntityTest.java | 185 ++
.../dao/entity/GroupConsumeCtrlEntityTest.java | 96 +
.../dao/entity/GroupResCtrlEntityTest.java | 137 ++
.../metastore/dao/entity/TopicCtrlEntityTest.java | 125 ++
.../dao/entity/TopicDeployEntityTest.java | 161 ++
.../metastore/dao/entity/TopicPropGroupTest.java | 156 ++
.../nodebroker/BrokerInfoHolderTest.java | 46 -
120 files changed, 21376 insertions(+), 12735 deletions(-)
diff --git a/NOTICE b/NOTICE
index f12016a..deddb19 100644
--- a/NOTICE
+++ b/NOTICE
@@ -4,7 +4,7 @@ Copyright 2019-2021 The Apache Software Foundation.
This product includes software developed at
The Apache Software Foundation (https://www.apache.org/).
-The initial codebase donated to the ASF by Tencent, copyright 2012-2019.
+The initial codebase was donated to the ASF by Tencent, copyright 2012-2019.
=======================================================================
diff --git a/inlong-tubemq/tubemq-client/src/main/java/org/apache/inlong/tubemq/client/consumer/ConsumerSamplePrint.java b/inlong-tubemq/tubemq-client/src/main/java/org/apache/inlong/tubemq/client/consumer/ConsumerSamplePrint.java
index 0efe9bc..394cc39 100644
--- a/inlong-tubemq/tubemq-client/src/main/java/org/apache/inlong/tubemq/client/consumer/ConsumerSamplePrint.java
+++ b/inlong-tubemq/tubemq-client/src/main/java/org/apache/inlong/tubemq/client/consumer/ConsumerSamplePrint.java
@@ -71,4 +71,13 @@ public class ConsumerSamplePrint extends AbstractSamplePrint {
//
}
+ @Override
+ public void printWarn(String err) {
+ //
+ }
+
+ @Override
+ public void printError(String err) {
+ //
+ }
}
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/TBaseConstants.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/TBaseConstants.java
index e74c512..0eba017 100644
--- a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/TBaseConstants.java
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/TBaseConstants.java
@@ -21,10 +21,13 @@ public class TBaseConstants {
public static final int META_VALUE_UNDEFINED = -2;
+ public static final int BUILDER_DEFAULT_SIZE = 512;
+
public static final int META_DEFAULT_MASTER_PORT = 8715;
public static final int META_DEFAULT_MASTER_TLS_PORT = 8716;
public static final int META_DEFAULT_BROKER_PORT = 8123;
public static final int META_DEFAULT_BROKER_TLS_PORT = 8124;
+ public static final int META_DEFAULT_BROKER_WEB_PORT = 8081;
public static final int META_STORE_INS_BASE = 10000;
public static final String META_DEFAULT_CHARSET_NAME = "UTF-8";
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/TokenConstants.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/TokenConstants.java
index af12f7c..35dc766 100644
--- a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/TokenConstants.java
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/TokenConstants.java
@@ -29,13 +29,6 @@ public class TokenConstants {
public static final String BLANK = " ";
- public static final String TOKEN_STORE_NUM = "storeNum";
- public static final String TOKEN_QRY_PRIORITY_ID = "qryPriorityId";
- public static final String TOKEN_DATA_UNFLUSHHOLD = "unFlushDataHold";
- public static final String TOKEN_MCACHE_MSG_CNT = "memCacheMsgCntInK";
- public static final String TOKEN_TLS_PORT = "TLSPort";
- public static final String TOKEN_MCACHE_MSG_SIZE = "memCacheMsgSizeInMB";
- public static final String TOKEN_MCACHE_FLUSH_INTVL = "memCacheFlushIntvl";
public static final String TOKEN_MSG_TYPE = "$msgType$";
public static final String TOKEN_MSG_TIME = "$msgTime$";
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/cluster/TopicInfo.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/cluster/TopicInfo.java
index 02f38c0..5b333ab 100644
--- a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/cluster/TopicInfo.java
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/cluster/TopicInfo.java
@@ -19,6 +19,7 @@ package org.apache.inlong.tubemq.corebase.cluster;
import java.io.Serializable;
import org.apache.inlong.tubemq.corebase.TokenConstants;
+import org.apache.inlong.tubemq.corebase.utils.Tuple2;
public class TopicInfo implements Serializable {
@@ -74,6 +75,21 @@ public class TopicInfo implements Serializable {
this.acceptSubscribe = acceptSubscribe;
}
+ // return result <isChanged, isScaleOut>
+ public Tuple2<Boolean, Boolean> updAndJudgeTopicInfo(TopicInfo newTopicInfo) {
+ boolean isChanged = false;
+ if (this.acceptPublish != newTopicInfo.acceptPublish) {
+ isChanged = true;
+ this.acceptPublish = newTopicInfo.acceptPublish;
+ }
+ if (this.acceptSubscribe != newTopicInfo.acceptSubscribe) {
+ isChanged = true;
+ this.acceptSubscribe = newTopicInfo.acceptSubscribe;
+ }
+ return new Tuple2<>(isChanged, (this.partitionNum != newTopicInfo.partitionNum
+ || this.topicStoreNum != newTopicInfo.topicStoreNum));
+ }
+
public int getTopicStoreNum() {
return topicStoreNum;
}
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/AbstractSamplePrint.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/AbstractSamplePrint.java
index 2f9704a..ba69b58 100644
--- a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/AbstractSamplePrint.java
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/AbstractSamplePrint.java
@@ -51,4 +51,8 @@ public abstract class AbstractSamplePrint {
public abstract void printExceptionCaught(Throwable e);
public abstract void printExceptionCaught(Throwable e, String hostName, String nodeName);
+
+ public abstract void printWarn(String err);
+
+ public abstract void printError(String err);
}
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/KeyBuilderUtils.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/KeyBuilderUtils.java
new file mode 100644
index 0000000..8208e2f
--- /dev/null
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/KeyBuilderUtils.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.corebase.utils;
+
+import org.apache.inlong.tubemq.corebase.TBaseConstants;
+import org.apache.inlong.tubemq.corebase.TokenConstants;
+
+
+
+
+
+public class KeyBuilderUtils {
+
+ public static String buildGroupTopicRecKey(String groupName, String topicName) {
+ return new StringBuilder(TBaseConstants.BUILDER_DEFAULT_SIZE)
+ .append(topicName).append(TokenConstants.ATTR_SEP)
+ .append(groupName).toString();
+ }
+
+ public static Tuple2<String, String> splitRecKey2GroupTopic(String recordKey) {
+ Tuple2<String, String> ret = new Tuple2<>();
+ if (recordKey == null) {
+ return ret;
+ }
+ String[] items = recordKey.split(TokenConstants.ATTR_SEP);
+ // return [topicName, groupName]
+ if (items.length < 2) {
+ ret.setF0AndF1(items[0], "");
+ } else {
+ ret.setF0AndF1(items[0], items[1]);
+ }
+ return ret;
+ }
+
+ public static String buildTopicConfRecKey(int brokerId, String topicName) {
+ return new StringBuilder(TBaseConstants.BUILDER_DEFAULT_SIZE)
+ .append(brokerId).append(TokenConstants.ATTR_SEP)
+ .append(topicName).toString();
+ }
+
+ public static String buildAddressInfo(String brokerIp, int brokerPort) {
+ return new StringBuilder(TBaseConstants.BUILDER_DEFAULT_SIZE)
+ .append(brokerIp).append(TokenConstants.ATTR_SEP)
+ .append(brokerPort).toString();
+ }
+
+}
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/SettingValidUtils.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/SettingValidUtils.java
index e24a0f3..e4da259 100644
--- a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/SettingValidUtils.java
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/SettingValidUtils.java
@@ -23,6 +23,19 @@ import org.apache.inlong.tubemq.corebase.TBaseConstants;
public class SettingValidUtils {
+ public static int validAndGetMsgSizeInMB(int inMaxMsgSizeInMB) {
+ return MixedUtils.mid(inMaxMsgSizeInMB,
+ TBaseConstants.META_MIN_ALLOWED_MESSAGE_SIZE_MB,
+ TBaseConstants.META_MAX_ALLOWED_MESSAGE_SIZE_MB);
+ }
+
+ public static int validAndGetMsgSizeBtoMB(int inMaxMsgSizeInB) {
+ return MixedUtils.mid(inMaxMsgSizeInB,
+ TBaseConstants.META_MAX_MESSAGE_DATA_SIZE,
+ TBaseConstants.META_MAX_MESSAGE_DATA_SIZE_UPPER_LIMIT)
+ / TBaseConstants.META_MB_UNIT_SIZE;
+ }
+
public static int validAndXfeMaxMsgSizeFromMBtoB(int inMaxMsgSizeInMB) {
return MixedUtils.mid(inMaxMsgSizeInMB,
TBaseConstants.META_MIN_ALLOWED_MESSAGE_SIZE_MB,
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/Tuple2.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/Tuple2.java
index a7064ce..12540a0 100644
--- a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/Tuple2.java
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/Tuple2.java
@@ -47,8 +47,7 @@ public class Tuple2<T0, T1> {
* @param value1 The value for field 1
*/
public Tuple2(T0 value0, T1 value1) {
- this.f0 = value0;
- this.f1 = value1;
+ setF0AndF1(value0, value1);
}
public T0 getF0() {
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/Tuple3.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/Tuple3.java
index 25fbe1c..74e6286 100644
--- a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/Tuple3.java
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/Tuple3.java
@@ -41,9 +41,7 @@ public class Tuple3<T0, T1, T2> {
* @param value2 The value for field 2
*/
public Tuple3(T0 value0, T1 value1, T2 value2) {
- this.f0 = value0;
- this.f1 = value1;
- this.f2 = value2;
+ setFieldsValue(value0, value1, value2);
}
public T0 getF0() {
@@ -57,4 +55,10 @@ public class Tuple3<T0, T1, T2> {
public T2 getF2() {
return f2;
}
+
+ public void setFieldsValue(T0 value0, T1 value1, T2 value2) {
+ this.f0 = value0;
+ this.f1 = value1;
+ this.f2 = value2;
+ }
}
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/Tuple3.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/Tuple4.java
similarity index 75%
copy from inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/Tuple3.java
copy to inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/Tuple4.java
index 25fbe1c..d4de540 100644
--- a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/Tuple3.java
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/Tuple4.java
@@ -17,7 +17,7 @@
package org.apache.inlong.tubemq.corebase.utils;
-public class Tuple3<T0, T1, T2> {
+public class Tuple4<T0, T1, T2, T3> {
/** Field 0 of the tuple. */
private T0 f0 = null;
@@ -25,11 +25,13 @@ public class Tuple3<T0, T1, T2> {
private T1 f1 = null;
/** Field 2 of the tuple. */
private T2 f2 = null;
+ /** Field 3 of the tuple. */
+ private T3 f3 = null;
/**
* Creates a new tuple where all fields are null.
*/
- public Tuple3() {
+ public Tuple4() {
}
@@ -39,11 +41,10 @@ public class Tuple3<T0, T1, T2> {
* @param value0 The value for field 0
* @param value1 The value for field 1
* @param value2 The value for field 2
+ * @param value3 The value for field 3
*/
- public Tuple3(T0 value0, T1 value1, T2 value2) {
- this.f0 = value0;
- this.f1 = value1;
- this.f2 = value2;
+ public Tuple4(T0 value0, T1 value1, T2 value2, T3 value3) {
+ setFieldsValue(value0, value1, value2, value3);
}
public T0 getF0() {
@@ -57,4 +58,16 @@ public class Tuple3<T0, T1, T2> {
public T2 getF2() {
return f2;
}
+
+ public T3 getF3() {
+ return f3;
+ }
+
+ public void setFieldsValue(T0 value0, T1 value1, T2 value2, T3 value3) {
+ this.f0 = value0;
+ this.f1 = value1;
+ this.f2 = value2;
+ this.f3 = value3;
+ }
+
}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/MessageStore.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/MessageStore.java
index 91aea5f..4d2d7fd 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/MessageStore.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/MessageStore.java
@@ -38,7 +38,6 @@ import org.apache.inlong.tubemq.corebase.protobuf.generated.ClientBroker;
import org.apache.inlong.tubemq.corebase.utils.MixedUtils;
import org.apache.inlong.tubemq.corebase.utils.ThreadUtils;
import org.apache.inlong.tubemq.server.broker.BrokerConfig;
-import org.apache.inlong.tubemq.server.broker.metadata.ClusterConfigHolder;
import org.apache.inlong.tubemq.server.broker.metadata.TopicMetadata;
import org.apache.inlong.tubemq.server.broker.msgstore.disk.GetMessageResult;
import org.apache.inlong.tubemq.server.broker.msgstore.disk.MsgFileStatisInfo;
@@ -603,12 +602,7 @@ public class MessageStore implements Closeable {
private int validAndGetMemCacheSize(TopicMetadata topicMetadata) {
int memCacheSize = topicMetadata.getMemCacheMsgSize();
- if (memCacheSize <= topicMetadata.getMinMemCacheSize()) {
- logger.info(new StringBuilder(512)
- .append("[Data Store] ").append(getTopic())
- .append(" writeCacheMaxSize changed, from ")
- .append(memCacheSize).append(" to ")
- .append(ClusterConfigHolder.getMinMemCacheSize()).toString());
+ if (memCacheSize < topicMetadata.getMinMemCacheSize()) {
memCacheSize = topicMetadata.getMinMemCacheSize();
}
return memCacheSize;
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/utils/BrokerSamplePrint.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/utils/BrokerSamplePrint.java
index 8d23394..55fc036 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/utils/BrokerSamplePrint.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/utils/BrokerSamplePrint.java
@@ -80,4 +80,13 @@ public class BrokerSamplePrint extends AbstractSamplePrint {
//
}
+ @Override
+ public void printWarn(String err) {
+ //
+ }
+
+ @Override
+ public void printError(String err) {
+ //
+ }
}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/utils/DiskSamplePrint.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/utils/DiskSamplePrint.java
index 50f936f..e30c30a 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/utils/DiskSamplePrint.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/utils/DiskSamplePrint.java
@@ -117,4 +117,13 @@ public class DiskSamplePrint extends AbstractSamplePrint {
}
}
+ @Override
+ public void printWarn(String err) {
+ //
+ }
+
+ @Override
+ public void printError(String err) {
+ //
+ }
}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/web/AbstractWebHandler.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/web/AbstractWebHandler.java
index 8b23afb..cc4583d 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/web/AbstractWebHandler.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/web/AbstractWebHandler.java
@@ -52,29 +52,29 @@ public abstract class AbstractWebHandler extends HttpServlet {
@Override
protected void doPost(HttpServletRequest req,
HttpServletResponse resp) throws IOException {
- StringBuilder strBuffer = new StringBuilder(1024);
+ StringBuilder sBuffer = new StringBuilder(1024);
try {
String method = req.getParameter("method");
if (method == null) {
- strBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
+ sBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
.append("Please take with method parameter! \"}");
} else {
WebApiRegInfo webApiRegInfo = getWebApiRegInfo(method);
if (webApiRegInfo == null) {
- strBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
+ sBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
.append("Unsupported method ").append(method).append("\"}");
} else {
- webApiRegInfo.method.invoke(webApiRegInfo.webHandler, req, strBuffer);
+ webApiRegInfo.method.invoke(webApiRegInfo.webHandler, req, sBuffer);
}
}
} catch (Throwable e) {
- strBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
+ sBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
.append("Bad request from server: ")
.append(e.getMessage())
.append("\"}");
}
- resp.getWriter().write(strBuffer.toString());
+ resp.getWriter().write(sBuffer.toString());
resp.setCharacterEncoding(req.getCharacterEncoding());
resp.setStatus(HttpServletResponse.SC_OK);
resp.flushBuffer();
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/web/BrokerAdminServlet.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/web/BrokerAdminServlet.java
index db80778..a6cda77 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/web/BrokerAdminServlet.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/web/BrokerAdminServlet.java
@@ -26,7 +26,6 @@ import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import javax.servlet.http.HttpServletRequest;
-
import org.apache.inlong.tubemq.corebase.TokenConstants;
import org.apache.inlong.tubemq.corebase.utils.MixedUtils;
import org.apache.inlong.tubemq.corebase.utils.TStringUtils;
@@ -44,6 +43,7 @@ import org.apache.inlong.tubemq.server.common.fielddef.WebFieldDef;
import org.apache.inlong.tubemq.server.common.utils.ProcessResult;
import org.apache.inlong.tubemq.server.common.utils.WebParameterUtils;
+
/***
* Broker's web servlet. Used for admin operation, like query consumer's status etc.
*/
@@ -128,7 +128,7 @@ public class BrokerAdminServlet extends AbstractWebHandler {
int index = 0;
ProcessResult result = new ProcessResult();
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.COMPSGROUPNAME, false, null, result)) {
+ WebFieldDef.COMPSGROUPNAME, false, null, sBuilder, result)) {
WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
return;
}
@@ -216,7 +216,7 @@ public class BrokerAdminServlet extends AbstractWebHandler {
StringBuilder sBuilder) {
ProcessResult result = new ProcessResult();
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.COMPSTOPICNAME, false, null, result)) {
+ WebFieldDef.COMPSTOPICNAME, false, null, sBuilder, result)) {
WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
return;
}
@@ -276,25 +276,25 @@ public class BrokerAdminServlet extends AbstractWebHandler {
* Get memory store status info.
*
* @param req
- * @param sBuilder process result
+ * @param sBuffer process result
* @throws Exception
*/
public void adminGetMemStoreStatisInfo(HttpServletRequest req,
- StringBuilder sBuilder) {
+ StringBuilder sBuffer) {
ProcessResult result = new ProcessResult();
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.COMPSTOPICNAME, false, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.COMPSTOPICNAME, false, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
Set<String> topicNameSet = (Set<String>) result.retData1;
if (!WebParameterUtils.getBooleanParamValue(req,
- WebFieldDef.NEEDREFRESH, false, false, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.NEEDREFRESH, false, false, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
boolean requireRefresh = (boolean) result.retData1;
- sBuilder.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"Success!\",\"detail\":[");
+ sBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"Success!\",\"detail\":[");
Map<String, ConcurrentHashMap<Integer, MessageStore>> messageTopicStores =
broker.getStoreManager().getMessageStores();
int index = 0;
@@ -306,10 +306,10 @@ public class BrokerAdminServlet extends AbstractWebHandler {
}
String topicName = entry.getKey();
if (recordId++ > 0) {
- sBuilder.append(",");
+ sBuffer.append(",");
}
index = 0;
- sBuilder.append("{\"topicName\":\"").append(topicName).append("\",\"storeStatisInfo\":[");
+ sBuffer.append("{\"topicName\":\"").append(topicName).append("\",\"storeStatisInfo\":[");
ConcurrentHashMap<Integer, MessageStore> partStoreMap = entry.getValue();
if (partStoreMap != null) {
for (Entry<Integer, MessageStore> subEntry : partStoreMap.entrySet()) {
@@ -318,62 +318,62 @@ public class BrokerAdminServlet extends AbstractWebHandler {
continue;
}
if (index++ > 0) {
- sBuilder.append(",");
+ sBuffer.append(",");
}
- sBuilder.append("{\"storeId\":").append(subEntry.getKey())
+ sBuffer.append("{\"storeId\":").append(subEntry.getKey())
.append(",\"memStatis\":").append(msgStore.getCurMemMsgSizeStatisInfo(requireRefresh))
.append(",\"fileStatis\":")
.append(msgStore.getCurFileMsgSizeStatisInfo(requireRefresh)).append("}");
}
}
- sBuilder.append("]}");
+ sBuffer.append("]}");
}
- sBuilder.append("],\"totalCount\":").append(recordId).append("}");
+ sBuffer.append("],\"totalCount\":").append(recordId).append("}");
}
/***
* Manual set offset.
*
* @param req
- * @param sBuilder process result
+ * @param sBuffer process result
* @throws Exception
*/
public void adminManualSetCurrentOffSet(HttpServletRequest req,
- StringBuilder sBuilder) {
+ StringBuilder sBuffer) {
ProcessResult result = new ProcessResult();
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.TOPICNAME, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.TOPICNAME, true, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final String topicName = (String) result.retData1;
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.GROUPNAME, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.GROUPNAME, true, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final String groupName = (String) result.retData1;
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.MODIFYUSER, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.MODIFYUSER, true, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final String modifyUser = (String) result.retData1;
if (!WebParameterUtils.getIntParamValue(req,
- WebFieldDef.PARTITIONID, true, -1, 0, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.PARTITIONID, true, -1, 0, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
int partitionId = (Integer) result.retData1;
if (!WebParameterUtils.getLongParamValue(req,
- WebFieldDef.MANUALOFFSET, true, -1, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.MANUALOFFSET, true, -1, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final long manualOffset = (Long) result.retData1;
List<String> topicList = broker.getMetadataManager().getTopics();
if (!topicList.contains(topicName)) {
- sBuilder.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
+ sBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
.append("Invalid parameter: not found the topicName configure!")
.append("\"}");
return;
@@ -386,20 +386,20 @@ public class BrokerAdminServlet extends AbstractWebHandler {
//
}
if (store == null) {
- sBuilder.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
+ sBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
.append("Invalid parameter: not found the store by topicName!")
.append("\"}");
return;
}
if (manualOffset < store.getIndexMinOffset()) {
- sBuilder.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
+ sBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
.append("Invalid parameter: manualOffset lower than Current MinOffset:(")
.append(manualOffset).append("<").append(store.getIndexMinOffset())
.append(")\"}");
return;
}
if (manualOffset > store.getIndexMaxOffset()) {
- sBuilder.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
+ sBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
.append("Invalid parameter: manualOffset bigger than Current MaxOffset:(")
.append(manualOffset).append(">").append(store.getIndexMaxOffset())
.append(")\"}");
@@ -410,11 +410,11 @@ public class BrokerAdminServlet extends AbstractWebHandler {
offsetService.resetOffset(store, groupName,
topicName, partitionId, manualOffset, modifyUser);
if (oldOffset < 0) {
- sBuilder.append("{\"result\":false,\"errCode\":401,\"errMsg\":\"")
+ sBuffer.append("{\"result\":false,\"errCode\":401,\"errMsg\":\"")
.append("Manual update current Offset failure!")
.append("\"}");
} else {
- sBuilder.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"")
+ sBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"")
.append("Manual update current Offset success!")
.append("\",\"oldOffset\":").append(oldOffset).append("}");
}
@@ -424,84 +424,84 @@ public class BrokerAdminServlet extends AbstractWebHandler {
* Query snapshot message set.
*
* @param req
- * @param sBuilder process result
+ * @param sBuffer process result
* @throws Exception
*/
public void adminQuerySnapshotMessageSet(HttpServletRequest req,
- StringBuilder sBuilder) throws Exception {
+ StringBuilder sBuffer) throws Exception {
ProcessResult result = new ProcessResult();
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.TOPICNAME, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.TOPICNAME, true, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final String topicName = (String) result.retData1;
if (!WebParameterUtils.getIntParamValue(req,
- WebFieldDef.PARTITIONID, false, -1, 0, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.PARTITIONID, false, -1, 0, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final int partitionId = (Integer) result.retData1;
if (!WebParameterUtils.getIntParamValue(req,
- WebFieldDef.MSGCOUNT, false, 3, 3, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.MSGCOUNT, false, 3, 3, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
int msgCount = (Integer) result.retData1;
msgCount = Math.max(msgCount, 1);
if (msgCount > 50) {
- sBuilder.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
+ sBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
.append("Over max allowed msgCount value, allowed count is 50!")
.append("\"}");
return;
}
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.FILTERCONDS, false, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.FILTERCONDS, false, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
Set<String> filterCondStrSet = (Set<String>) result.retData1;
broker.getBrokerServiceServer()
- .getMessageSnapshot(topicName, partitionId, msgCount, filterCondStrSet, sBuilder);
+ .getMessageSnapshot(topicName, partitionId, msgCount, filterCondStrSet, sBuffer);
}
/***
* Query consumer group offset.
*
* @param req
- * @param sBuilder process result
+ * @param sBuffer process result
* @throws Exception
*/
public void adminQueryCurrentGroupOffSet(HttpServletRequest req,
- StringBuilder sBuilder) {
+ StringBuilder sBuffer) {
ProcessResult result = new ProcessResult();
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.TOPICNAME, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.TOPICNAME, true, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final String topicName = (String) result.retData1;
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.GROUPNAME, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.GROUPNAME, true, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final String groupName = (String) result.retData1;
if (!WebParameterUtils.getIntParamValue(req,
- WebFieldDef.PARTITIONID, true, -1, 0, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.PARTITIONID, true, -1, 0, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
int partitionId = (Integer) result.retData1;
if (!WebParameterUtils.getBooleanParamValue(req,
- WebFieldDef.REQUIREREALOFFSET, false, false, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.REQUIREREALOFFSET, false, false, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final boolean requireRealOffset = (Boolean) result.retData1;
List<String> topicList = broker.getMetadataManager().getTopics();
if (!topicList.contains(topicName)) {
- sBuilder.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
+ sBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
.append("Invalid parameter: not found the topicName configure!")
.append("\"}");
return;
@@ -515,7 +515,7 @@ public class BrokerAdminServlet extends AbstractWebHandler {
//
}
if (store == null) {
- sBuilder.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
+ sBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
.append("Invalid parameter: not found the store by topicName!")
.append("\"}");
return;
@@ -525,7 +525,7 @@ public class BrokerAdminServlet extends AbstractWebHandler {
long maxDataOffset = store.getDataMaxOffset();
long minPartOffset = store.getIndexMinOffset();
long maxPartOffset = store.getIndexMaxOffset();
- sBuilder.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"")
+ sBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"")
.append("OK!")
.append("\",\"tmpOffset\":").append(tmpOffset)
.append(",\"minOffset\":").append(minPartOffset)
@@ -544,51 +544,51 @@ public class BrokerAdminServlet extends AbstractWebHandler {
curRdDltDataOffset = curReadDataOffset < 0 ? -2 : maxDataOffset - curReadDataOffset;
}
if (curReadDataOffset < 0) {
- sBuilder.append(",\"zkOffset\":").append(zkOffset)
+ sBuffer.append(",\"zkOffset\":").append(zkOffset)
.append(",\"curReadDataOffset\":-1,\"curRdDltDataOffset\":-1");
} else {
- sBuilder.append(",\"zkOffset\":").append(zkOffset)
+ sBuffer.append(",\"zkOffset\":").append(zkOffset)
.append(",\"curReadDataOffset\":").append(curReadDataOffset)
.append(",\"curRdDltDataOffset\":").append(curRdDltDataOffset);
}
}
- sBuilder.append("}");
+ sBuffer.append("}");
}
public void adminQueryConsumerRegisterInfo(HttpServletRequest req,
- StringBuilder sBuilder) {
+ StringBuilder sBuffer) {
Map<String, ConsumerNodeInfo> map =
broker.getBrokerServiceServer().getConsumerRegisterMap();
int totalCnt = 0;
- sBuilder.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"Success!\",\"dataSet\":[");
+ sBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"Success!\",\"dataSet\":[");
for (Entry<String, ConsumerNodeInfo> entry : map.entrySet()) {
if (entry.getKey() == null || entry.getValue() == null) {
continue;
}
if (totalCnt++ > 0) {
- sBuilder.append(",");
+ sBuffer.append(",");
}
- sBuilder.append("{\"Partition\":\"").append(entry.getKey())
+ sBuffer.append("{\"Partition\":\"").append(entry.getKey())
.append("\",\"Consumer\":\"")
.append(entry.getValue().getConsumerId())
.append("\",\"index\":").append(totalCnt).append("}");
}
- sBuilder.append("],\"totalCnt\":").append(totalCnt).append("}");
+ sBuffer.append("],\"totalCnt\":").append(totalCnt).append("}");
}
/***
* Query topic's publish info on the Broker.
*
* @param req
- * @param sBuilder process result
+ * @param sBuffer process result
*/
public void adminQueryPubInfo(HttpServletRequest req,
- StringBuilder sBuilder) {
+ StringBuilder sBuffer) {
ProcessResult result = new ProcessResult();
// get the topic set to be queried
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.COMPSTOPICNAME, false, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.COMPSTOPICNAME, false, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
// get target consume group name
@@ -598,41 +598,41 @@ public class BrokerAdminServlet extends AbstractWebHandler {
broker.getStoreManager().getTopicPublishInfos(topicSet);
// builder result
int totalCnt = 0;
- sBuilder.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"Success!\",\"dataSet\":[");
+ sBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"Success!\",\"dataSet\":[");
for (Map.Entry<String, Map<Integer, TopicPubStoreInfo>> entry
: topicStorePubInfoMap.entrySet()) {
if (totalCnt++ > 0) {
- sBuilder.append(",");
+ sBuffer.append(",");
}
- sBuilder.append("{\"topicName\":\"").append(entry.getKey())
+ sBuffer.append("{\"topicName\":\"").append(entry.getKey())
.append("\",\"offsetInfo\":[");
Map<Integer, TopicPubStoreInfo> storeInfoMap = entry.getValue();
int itemCnt = 0;
for (Map.Entry<Integer, TopicPubStoreInfo> entry1 : storeInfoMap.entrySet()) {
if (itemCnt++ > 0) {
- sBuilder.append(",");
+ sBuffer.append(",");
}
TopicPubStoreInfo pubStoreInfo = entry1.getValue();
- pubStoreInfo.buildPubStoreInfo(sBuilder);
+ pubStoreInfo.buildPubStoreInfo(sBuffer);
}
- sBuilder.append("],\"itemCount\":").append(itemCnt).append("}");
+ sBuffer.append("],\"itemCount\":").append(itemCnt).append("}");
}
- sBuilder.append("],\"dataCount\":").append(totalCnt).append("}");
+ sBuffer.append("],\"dataCount\":").append(totalCnt).append("}");
}
/***
* Query all consumer groups booked on the Broker.
*
* @param req
- * @param sBuilder process result
+ * @param sBuffer process result
*/
public void adminQueryBookedGroup(HttpServletRequest req,
- StringBuilder sBuilder) {
+ StringBuilder sBuffer) {
// get divide info
ProcessResult result = new ProcessResult();
if (!WebParameterUtils.getBooleanParamValue(req,
- WebFieldDef.WITHDIVIDE, false, false, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.WITHDIVIDE, false, false, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
boolean withDivide = (boolean) result.retData1;
@@ -640,67 +640,67 @@ public class BrokerAdminServlet extends AbstractWebHandler {
int itemCnt = 0;
int totalCnt = 0;
OffsetService offsetService = broker.getOffsetManager();
- sBuilder.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"Success!\",\"dataSet\":[");
+ sBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"Success!\",\"dataSet\":[");
if (withDivide) {
// query in-memory group name set
Set<String> onlineGroups = offsetService.getInMemoryGroups();
- sBuilder.append("{\"type\":\"in-cache\",\"groupName\":[");
+ sBuffer.append("{\"type\":\"in-cache\",\"groupName\":[");
for (String group : onlineGroups) {
if (itemCnt++ > 0) {
- sBuilder.append(",");
+ sBuffer.append(",");
}
- sBuilder.append("\"").append(group).append("\"");
+ sBuffer.append("\"").append(group).append("\"");
}
- sBuilder.append("],\"groupCount\":").append(itemCnt).append("}");
+ sBuffer.append("],\"groupCount\":").append(itemCnt).append("}");
totalCnt++;
- sBuilder.append(",");
+ sBuffer.append(",");
// query in-zk group name set
itemCnt = 0;
Set<String> onZKGroup = offsetService.getUnusedGroupInfo();
- sBuilder.append("{\"type\":\"in-zk\",\"groupName\":[");
+ sBuffer.append("{\"type\":\"in-zk\",\"groupName\":[");
for (String group : onZKGroup) {
if (itemCnt++ > 0) {
- sBuilder.append(",");
+ sBuffer.append(",");
}
- sBuilder.append("\"").append(group).append("\"");
+ sBuffer.append("\"").append(group).append("\"");
}
- sBuilder.append("],\"groupCount\":").append(itemCnt).append("}");
+ sBuffer.append("],\"groupCount\":").append(itemCnt).append("}");
totalCnt++;
} else {
Set<String> allGroups = offsetService.getBookedGroups();
- sBuilder.append("{\"type\":\"all\",\"groupName\":[");
+ sBuffer.append("{\"type\":\"all\",\"groupName\":[");
for (String group : allGroups) {
if (itemCnt++ > 0) {
- sBuilder.append(",");
+ sBuffer.append(",");
}
- sBuilder.append("\"").append(group).append("\"");
+ sBuffer.append("\"").append(group).append("\"");
}
- sBuilder.append("],\"groupCount\":").append(itemCnt).append("}");
+ sBuffer.append("],\"groupCount\":").append(itemCnt).append("}");
totalCnt++;
}
- sBuilder.append("],\"dataCount\":").append(totalCnt).append("}");
+ sBuffer.append("],\"dataCount\":").append(totalCnt).append("}");
}
/***
* Query consumer group offset.
*
* @param req
- * @param sBuilder process result
+ * @param sBuffer process result
*/
public void adminQueryGroupOffSet(HttpServletRequest req,
- StringBuilder sBuilder) {
+ StringBuilder sBuffer) {
ProcessResult result = new ProcessResult();
// get group list
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.COMPSGROUPNAME, false, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.COMPSGROUPNAME, false, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
Set<String> inGroupNameSet = (Set<String>) result.retData1;
// get the topic set to be queried
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.COMPSTOPICNAME, false, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.COMPSTOPICNAME, false, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
// get target consume group name
@@ -723,65 +723,65 @@ public class BrokerAdminServlet extends AbstractWebHandler {
getGroupOffsetInfo(WebFieldDef.COMPSGROUPNAME, qryGroupNameSet, topicSet);
// builder result
int totalCnt = 0;
- sBuilder.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"Success!\",\"dataSet\":[");
+ sBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"Success!\",\"dataSet\":[");
for (Map.Entry<String, Map<String, Map<Integer, GroupOffsetInfo>>> entry
: groupOffsetMaps.entrySet()) {
if (totalCnt++ > 0) {
- sBuilder.append(",");
+ sBuffer.append(",");
}
Map<String, Map<Integer, GroupOffsetInfo>> topicPartMap = entry.getValue();
- sBuilder.append("{\"groupName\":\"").append(entry.getKey())
+ sBuffer.append("{\"groupName\":\"").append(entry.getKey())
.append("\",\"subInfo\":[");
int topicCnt = 0;
for (Map.Entry<String, Map<Integer, GroupOffsetInfo>> entry1 : topicPartMap.entrySet()) {
if (topicCnt++ > 0) {
- sBuilder.append(",");
+ sBuffer.append(",");
}
Map<Integer, GroupOffsetInfo> partOffMap = entry1.getValue();
- sBuilder.append("{\"topicName\":\"").append(entry1.getKey())
+ sBuffer.append("{\"topicName\":\"").append(entry1.getKey())
.append("\",\"offsets\":[");
int partCnt = 0;
for (Map.Entry<Integer, GroupOffsetInfo> entry2 : partOffMap.entrySet()) {
if (partCnt++ > 0) {
- sBuilder.append(",");
+ sBuffer.append(",");
}
GroupOffsetInfo offsetInfo = entry2.getValue();
- offsetInfo.buildOffsetInfo(sBuilder);
+ offsetInfo.buildOffsetInfo(sBuffer);
}
- sBuilder.append("],\"partCount\":").append(partCnt).append("}");
+ sBuffer.append("],\"partCount\":").append(partCnt).append("}");
}
- sBuilder.append("],\"topicCount\":").append(topicCnt).append("}");
+ sBuffer.append("],\"topicCount\":").append(topicCnt).append("}");
}
- sBuilder.append("],\"totalCnt\":").append(totalCnt).append("}");
+ sBuffer.append("],\"totalCnt\":").append(totalCnt).append("}");
}
/***
* Add or Modify consumer group offset.
*
* @param req
- * @param sBuilder process result
+ * @param sBuffer process result
*/
public void adminSetGroupOffSet(HttpServletRequest req,
- StringBuilder sBuilder) {
+ StringBuilder sBuffer) {
ProcessResult result = new ProcessResult();
// get group list
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.COMPSGROUPNAME, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.COMPSGROUPNAME, true, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final Set<String> groupNameSet = (Set<String>) result.retData1;
// get set mode
if (!WebParameterUtils.getBooleanParamValue(req,
- WebFieldDef.MANUALSET, true, false, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.MANUALSET, true, false, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
boolean manualSet = (Boolean) result.retData1;
// get modify user
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.MODIFYUSER, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.MODIFYUSER, true, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
List<Tuple3<String, Integer, Long>> resetOffsets;
@@ -790,14 +790,14 @@ public class BrokerAdminServlet extends AbstractWebHandler {
// get offset json info
if (!WebParameterUtils.getJsonDictParamValue(req,
WebFieldDef.OFFSETJSON, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
Map<String, Long> manOffsets =
(Map<String, Long>) result.retData1;
// valid and transfer offset format
if (!validManOffsetResetInfo(WebFieldDef.OFFSETJSON, manOffsets, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
resetOffsets =
@@ -805,8 +805,8 @@ public class BrokerAdminServlet extends AbstractWebHandler {
} else {
// get the topic set to be set
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.COMPSTOPICNAME, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.COMPSTOPICNAME, true, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
Set<String> topicSet = (Set<String>) result.retData1;
@@ -814,58 +814,58 @@ public class BrokerAdminServlet extends AbstractWebHandler {
resetOffsets = buildOffsetResetInfo(topicSet);
}
broker.getOffsetManager().modifyGroupOffset(groupNameSet, resetOffsets, modifier);
- sBuilder.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"OK\"}");
+ sBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"OK\"}");
}
/***
* Clone consume group offset, clone A group's offset to other group.
*
* @param req
- * @param sBuilder process result
+ * @param sBuffer process result
*/
public void adminCloneGroupOffSet(HttpServletRequest req,
- StringBuilder sBuilder) {
+ StringBuilder sBuffer) {
ProcessResult result = new ProcessResult();
// get source consume group name
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.SRCGROUPNAME, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.SRCGROUPNAME, true, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final String srcGroupName = (String) result.retData1;
// get source consume group's topic set cloned to target group
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.COMPSTOPICNAME, false, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.COMPSTOPICNAME, false, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
Set<String> srcTopicNameSet = (Set<String>) result.retData1;
// valid topic and get topic's partitionIds
if (!validAndGetTopicPartInfo(srcGroupName,
WebFieldDef.SRCGROUPNAME, srcTopicNameSet, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final Map<String, Set<Integer>> topicPartMap =
(Map<String, Set<Integer>>) result.retData1;
// get target consume group name
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.TGTCOMPSGROUPNAME, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.TGTCOMPSGROUPNAME, true, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
Set<String> tgtGroupNameSet = (Set<String>) result.retData1;
// get modify user
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.MODIFYUSER, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.MODIFYUSER, true, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final String modifier = (String) result.retData1;
// check sourceGroup if existed
Set<String> bookedGroups = broker.getOffsetManager().getBookedGroups();
if (!bookedGroups.contains(srcGroupName)) {
- WebParameterUtils.buildFailResult(sBuilder,
+ WebParameterUtils.buildFailResult(sBuffer,
new StringBuilder(512).append("Parameter ")
.append(WebFieldDef.SRCGROUPNAME.name).append(": ")
.append(srcGroupName)
@@ -879,49 +879,49 @@ public class BrokerAdminServlet extends AbstractWebHandler {
List<Tuple3<String, Integer, Long>> resetOffsets = buildOffsetResetInfo(srcGroupOffsets);
broker.getOffsetManager().modifyGroupOffset(tgtGroupNameSet, resetOffsets, modifier);
// builder return result
- sBuilder.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"OK\"}");
+ sBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"OK\"}");
}
/***
* Remove consume group offset.
*
* @param req
- * @param sBuilder process result
+ * @param sBuffer process result
*/
public void adminRemoveGroupOffSet(HttpServletRequest req,
- StringBuilder sBuilder) {
+ StringBuilder sBuffer) {
ProcessResult result = new ProcessResult();
// get consume group name
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.COMPSGROUPNAME, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.COMPSGROUPNAME, true, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final Set<String> groupNameSet = (Set<String>) result.retData1;
// get modify user
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.MODIFYUSER, true, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.MODIFYUSER, true, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
final String modifier = (String) result.retData1;
// get need removed offset's topic
if (!WebParameterUtils.getStringParamValue(req,
- WebFieldDef.COMPSTOPICNAME, false, null, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.COMPSTOPICNAME, false, null, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
// get target consume group name
Set<String> topicNameSet = (Set<String>) result.retData1;
// get set mode
if (!WebParameterUtils.getBooleanParamValue(req,
- WebFieldDef.ONLYMEM, false, false, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebFieldDef.ONLYMEM, false, false, sBuffer, result)) {
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
boolean onlyMemory = (Boolean) result.retData1;
if (!validAndGetGroupTopicInfo(groupNameSet, topicNameSet, result)) {
- WebParameterUtils.buildFailResult(sBuilder, result.errInfo);
+ WebParameterUtils.buildFailResult(sBuffer, result.errInfo);
return;
}
Map<String, Map<String, Set<Integer>>> groupTopicPartMap =
@@ -929,7 +929,7 @@ public class BrokerAdminServlet extends AbstractWebHandler {
broker.getOffsetManager().deleteGroupOffset(
onlyMemory, groupTopicPartMap, modifier);
// builder return result
- sBuilder.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"OK\"}");
+ sBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"OK\"}");
}
// build reset offset info
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/TServerConstants.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/TServerConstants.java
index bb2b386..fb36112 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/TServerConstants.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/TServerConstants.java
@@ -25,10 +25,50 @@ public final class TServerConstants {
public static final String TOKEN_JOB_TOPICS = "topics";
public static final String TOKEN_JOB_STORE_MGR = "messageStoreManager";
public static final String TOKEN_DEFAULT_FLOW_CONTROL = "default_master_ctrl";
- public static final String TOKEN_DEFAULT_CLUSTER_SETTING = "default_cluster_config";
- public static final String TOKEN_MAX_MSG_SIZE = "maxMsgSize";
- public static final String TOKEN_BLANK_FILTER_CONDITION = ",,";
+ public static final long DEFAULT_DATA_VERSION = 0L;
+
+
+ public static final String BLANK_FLOWCTRL_RULES = "[]";
+ public static final String BLANK_FILTER_ITEM_STR = ",,";
+
+ public static final int QRY_PRIORITY_DEF_VALUE = 301;
+ public static final int QRY_PRIORITY_MIN_VALUE = 101;
+ public static final int QRY_PRIORITY_MAX_VALUE = 303;
+
+ public static final int TOPIC_STOREBLOCK_NUM_MIN = 1;
+
+ public static final int TOPIC_PARTITION_NUM_MIN = 1;
+
+ public static final int TOPIC_DSK_UNFLUSHTHRESHOLD_MIN = 0;
+ public static final int TOPIC_DSK_UNFLUSHTHRESHOLD_DEF = 1000;
+
+ public static final int TOPIC_DSK_UNFLUSHINTERVAL_MIN = 1;
+ public static final int TOPIC_DSK_UNFLUSHINTERVAL_DEF = 10000;
+
+ public static final int TOPIC_DSK_UNFLUSHDATAHOLD_MIN = 0;
+
+ public static final int TOPIC_CACHESIZE_MB_MIN = 2;
+ public static final int TOPIC_CACHESIZE_MB_DEF = 3;
+ public static final int TOPIC_CACHESIZE_MB_MAX = 2048;
+
+ public static final int TOPIC_CACHEINTVL_MIN = 4000;
+ public static final int TOPIC_CACHEINTVL_DEF = 20000;
+
+ public static final int TOPIC_CACHECNT_INK_MIN = 1;
+ public static final int TOPIC_CACHECNT_INK_DEF = 10;
+
+ public static final String TOPIC_POLICY_DEF = "delete,168h";
+
+ public static final int TOPIC_RET_PERIOD_IN_SEC_DEF = 14515200;
+
+ public static final int GROUP_BROKER_CLIENT_RATE_MIN = 0;
+
+ public static final int BROKER_REGION_ID_MIN = 0;
+ public static final int BROKER_REGION_ID_DEF = 0;
+
+ public static final int BROKER_GROUP_ID_MIN = 0;
+ public static final int BROKER_GROUP_ID_DEF = 0;
public static final int CFG_MODAUTHTOKEN_MAX_LENGTH = 128;
public static final int CFG_ROWLOCK_DEFAULT_DURATION = 30000;
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/nodemanage/nodebroker/TargetValidResult.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/exception/LoadMetaException.java
similarity index 64%
copy from inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/nodemanage/nodebroker/TargetValidResult.java
copy to inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/exception/LoadMetaException.java
index e8f9fe1..778abde 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/nodemanage/nodebroker/TargetValidResult.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/exception/LoadMetaException.java
@@ -1,32 +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.inlong.tubemq.server.master.nodemanage.nodebroker;
-
-
-public class TargetValidResult {
- public boolean result;
- public int errCode;
- public String errInfo;
-
- public TargetValidResult(boolean result, int errCode, final String errInfo) {
- this.result = result;
- this.errCode = errCode;
- this.errInfo = errInfo;
- }
-
-}
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.server.common.exception;
+
+public class LoadMetaException extends RuntimeException {
+
+ static final long serialVersionUID = 5286701925988728790L;
+
+ public LoadMetaException(String msg) {
+ super(msg);
+ }
+
+ public LoadMetaException(String msg, Throwable cause) {
+ super(msg, cause);
+ }
+
+ public LoadMetaException(Throwable cause) {
+ super(cause);
+ }
+}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/fielddef/WebFieldDef.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/fielddef/WebFieldDef.java
index 32c0ccc..c73b4ea 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/fielddef/WebFieldDef.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/fielddef/WebFieldDef.java
@@ -91,8 +91,8 @@ public enum WebFieldDef {
ADMINAUTHTOKEN(23, "confModAuthToken", "authToken", WebFieldType.STRING,
"Admin api operation authorization code",
TServerConstants.CFG_MODAUTHTOKEN_MAX_LENGTH),
- MAXMSGSIZE(24, "maxMsgSizeInMB", "maxMsgSizeInMB", WebFieldType.INT,
- "Max allowed message size, unit MB", RegexDef.TMP_NUMBER),
+ BROKERWEBPORT(24, "brokerWebPort", "bWebPort", WebFieldType.INT,
+ "Broker web port", RegexDef.TMP_NUMBER),
CREATEDATE(25, "createDate", "cDate", WebFieldType.STRING,
"Record creation date", TBaseConstants.META_MAX_DATEVALUE_LENGTH),
@@ -140,14 +140,14 @@ public enum WebFieldDef {
UNFMCACHEINTERVAL(43, "memCacheFlushIntvl", "unfMemInt", WebFieldType.INT,
"Maximum allowed disk unflushing data size", RegexDef.TMP_NUMBER),
MAXMSGSIZEINMB(44, "maxMsgSizeInMB", "maxMsgSizeInMB", WebFieldType.INT,
- "Maximum allowed message length", RegexDef.TMP_NUMBER),
+ "Maximum allowed message length, unit MB", RegexDef.TMP_NUMBER),
ACCEPTPUBLISH(45, "acceptPublish", "accPub", WebFieldType.BOOLEAN,
"Enable publishing"),
ACCEPTSUBSCRIBE(46, "acceptSubscribe", "accSub", WebFieldType.BOOLEAN,
"Enable subscription"),
- DELETEPOLICY(47, "deletePolicy", "delPolicy",
- WebFieldType.DELPOLICY, "File aging strategy"),
+ DELETEPOLICY(47, "deletePolicy", "delPolicy", WebFieldType.DELPOLICY,
+ "File aging strategy", TServerConstants.CFG_DELETEPOLICY_MAX_LENGTH),
TOPICJSONSET(48, "topicJsonSet", "topicSet",
WebFieldType.JSONSET, "The topic info set that needs to be added or modified"),
BROKERIP(49, "brokerIp", "brokerIp", WebFieldType.STRING,
@@ -171,10 +171,87 @@ public enum WebFieldDef {
CONDSTATUS(56, "condStatus", "condStatus", WebFieldType.INT,
"Group control rule status id", RegexDef.TMP_NUMBER),
FILTERJSONSET(57, "filterCondJsonSet", "filterJsonSet",
- WebFieldType.JSONSET, "The batch filter condition configure json array");
-
-
-
+ WebFieldType.JSONSET, "The batch filter condition configure json array"),
+ DATASTORETYPE(58, "dataStoreType", "dStType", WebFieldType.INT,
+ "Data store type", RegexDef.TMP_NUMBER),
+ DATAPATH(59, "dataPath", "dPath",
+ WebFieldType.STRING, "Data path"),
+
+ ATTRIBUTES(60, "attributes", "attrs",
+ WebFieldType.STRING, "Attributes"),
+ RECORDKEY(61, "recordKey", "recKey",
+ WebFieldType.STRING, "Record key"),
+ FLOWCTRLENABLE(62, "flowCtrlEnable", "fCtrlEn",
+ WebFieldType.BOOLEAN, "Flow control enable status"),
+ FLOWCTRLRULECOUNT(63, "flowCtrlRuleCount", "fCtrlCnt", WebFieldType.INT,
+ "The count of flow control info set", RegexDef.TMP_NUMBER),
+ RESCHECKENABLE(64, "resCheckEnable", "resChkEn",
+ WebFieldType.BOOLEAN, "Resource check enable status"),
+
+ ALWDBCRATE(65, "alwdBrokerClientRate", "abcr", WebFieldType.INT,
+ "Allowed broker client rate", RegexDef.TMP_NUMBER),
+ REASON(66, "reason", "rsn", WebFieldType.STRING,
+ "Reason", TBaseConstants.META_MAX_OPREASON_LENGTH, RegexDef.TMP_STRING),
+ FILTERENABLE(67, "filterEnable", "fltEn",
+ WebFieldType.BOOLEAN, "Filter consume enable status"),
+ MANAGESTATUS(68, "manageStatus", "mSts",
+ WebFieldType.STRING, "Broker manage status"),
+ GROUPID(69, "groupId", "gId",
+ WebFieldType.INT, "Group id", RegexDef.TMP_NUMBER),
+
+ TOPICSTATUSID(70, "topicStatusId", "tStsId", WebFieldType.INT,
+ "Status id", RegexDef.TMP_NUMBER),
+ AUTHCTRLENABLE(71, "enableAuthControl", "acEn",
+ WebFieldType.BOOLEAN, "Group authenticate control enable status"),
+ CONSUMEENABLE(72, "consumeEnable", "csmEn",
+ WebFieldType.BOOLEAN, "Consume enable status"),
+ GROUPCSMJSONSET(73, "groupCsmJsonSet", "csmJsonSet",
+ WebFieldType.JSONSET, "The batch group consume configure json array"),
+ WITHTOPIC(74, "withTopic", "wTopic",
+ WebFieldType.BOOLEAN, "With topic info."),
+
+ ISINCLUDE(75, "isInclude", "isInclude",
+ WebFieldType.BOOLEAN, "If include or un-include topic required"),
+ COMPBROKERIP(76, "brokerIp", "bIp", WebFieldType.COMPSTRING,
+ "Broker ip", TBaseConstants.META_MAX_BROKER_IP_LENGTH,
+ RegexDef.TMP_IPV4ADDRESS),
+ ISRESERVEDDATA(77, "isReservedData", "isRsvDt",
+ WebFieldType.BOOLEAN, "Whether to keep topic data in the broker"),
+ WITHGROUPAUTHINFO(78, "withGroupAuthInfo", "wGAI",
+ WebFieldType.BOOLEAN, "With topic group authorize info."),
+ WITHDEPLOYINFO(79, "withDeployInfo", "wDI",
+ WebFieldType.BOOLEAN, "With topic deploy info."),
+
+ TOPICCTRLSET(80, "topicCtrlJsonSet", "tCtrlSet", WebFieldType.JSONSET,
+ "The topic control info set that needs to be added or modified"),
+ GROUPRESCTRLSET(81, "groupResCtrlJsonSet", "gResCtrlSet",
+ WebFieldType.JSONSET,
+ "The group resource control info set that needs to be added or modified"),
+ @Deprecated
+ OLDALWDBCRATE(82, "allowedBClientRate", "abcr", WebFieldType.INT,
+ "Allowed broker client rate, same as alwdBrokerClientRate", RegexDef.TMP_NUMBER),
+ @Deprecated
+ GROUPJSONSET(83, "groupNameJsonSet", "gJsonSet", WebFieldType.JSONSET,
+ "The black list group set that needs to be added or modified"),
+ REJOINWAIT(84, "reJoinWait", "rjWait", WebFieldType.INT,
+ "The duration for consumer rejoin rebalance", RegexDef.TMP_NUMBER),
+
+ COMPSCONSUMERID(85, "consumerId", "csmId", WebFieldType.COMPSTRING,
+ "consumer id", TServerConstants.CFG_CONSUMER_CLIENTID_MAX_LENGTH,
+ RegexDef.TMP_CONSUMERID),
+ ISENABLE(86, "isEnable", "isEnable",
+ WebFieldType.BOOLEAN, "With status if enable."),
+ RELREASON(87, "relReason", "rRsn", WebFieldType.STRING,
+ "Release reason", TBaseConstants.META_MAX_OPREASON_LENGTH),
+ WITHDETAIL(88, "withDetail", "wDtl",
+ WebFieldType.BOOLEAN, "With broker configure detail info."),
+ ONLYABNORMAL(89, "onlyAbnormal", "oAbn",
+ WebFieldType.BOOLEAN, "only query abnormal broker info."),
+
+ ONLYAUTOFBD(90, "onlyAutoForbidden", "oAfb",
+ WebFieldType.BOOLEAN, "only auto forbidden abnormal broker info."),
+ ONLYENABLETLS(91, "onlyEnableTLS", "oEtls",
+ WebFieldType.BOOLEAN, "only enable tls broker info.");
public final int id;
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/heartbeat/HeartbeatManager.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/heartbeat/HeartbeatManager.java
index 6d2c091..b5c6d55 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/heartbeat/HeartbeatManager.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/heartbeat/HeartbeatManager.java
@@ -23,8 +23,11 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+
+import org.apache.inlong.tubemq.corebase.TErrCodeConstants;
import org.apache.inlong.tubemq.corebase.utils.TStringUtils;
import org.apache.inlong.tubemq.server.common.exception.HeartbeatException;
+import org.apache.inlong.tubemq.server.common.utils.ProcessResult;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -163,10 +166,12 @@ public class HeartbeatManager {
* Register a node as broker.
*
* @param nodeId the id of a node to be registered.
+ * @param createId broker run-info block id
* @return the timeout info for the registered node
*/
- public TimeoutInfo regBrokerNode(final String nodeId) {
- return this.brokerRegMap.put(nodeId, new TimeoutInfo(this.brokerTimeoutDlt));
+ public TimeoutInfo regBrokerNode(String nodeId, String createId) {
+ return this.brokerRegMap.put(nodeId,
+ new TimeoutInfo(createId, this.brokerTimeoutDlt));
}
/**
@@ -208,10 +213,18 @@ public class HeartbeatManager {
* Unregister a node from the broker
*
* @param nodeId the id of node to be unregistered
- * @return the timeout of the node
+ * @return if the timeout delete, true: success, false: failure
*/
- public TimeoutInfo unRegBrokerNode(final String nodeId) {
- return brokerRegMap.remove(nodeId);
+ public boolean unRegBrokerNode(String nodeId, String createId) {
+ TimeoutInfo timeoutInfo = brokerRegMap.get(nodeId);
+ if (timeoutInfo == null) {
+ return true;
+ }
+ if (!createId.equals(timeoutInfo.getSecondKey())) {
+ return false;
+ }
+ timeoutInfo = brokerRegMap.remove(nodeId);
+ return true;
}
/**
@@ -240,14 +253,26 @@ public class HeartbeatManager {
* @param nodeId the id of node to be updated
* @throws HeartbeatException if the timeout info of the node is not found
*/
- public void updBrokerNode(final String nodeId) throws HeartbeatException {
+ public boolean updBrokerNode(String nodeId, String createId,
+ StringBuilder sBuffer, ProcessResult result) {
TimeoutInfo timeoutInfo = brokerRegMap.get(nodeId);
if (timeoutInfo == null) {
- throw new HeartbeatException(new StringBuilder(512)
- .append("Invalid node id:").append(nodeId)
- .append(", you have to append node first!").toString());
+ result.setFailResult(TErrCodeConstants.HB_NO_NODE,
+ sBuffer.append("Invalid node id:").append(nodeId)
+ .append(", you have to append node first!").toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
+ }
+ if (!createId.equals(timeoutInfo.getSecondKey())) {
+ result.setFailResult(TErrCodeConstants.HB_NO_NODE,
+ sBuffer.append("Invalid node block id:").append(nodeId)
+ .append(", you have to append node first!").toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
}
timeoutInfo.updTimeoutTime(this.brokerTimeoutDlt);
+ result.setSuccResult(null);
+ return result.isSuccess();
}
/**
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/paramcheck/PBParameterUtils.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/paramcheck/PBParameterUtils.java
index db660da..75b2929 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/paramcheck/PBParameterUtils.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/paramcheck/PBParameterUtils.java
@@ -17,8 +17,6 @@
package org.apache.inlong.tubemq.server.common.paramcheck;
-
-import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@@ -35,9 +33,9 @@ import org.apache.inlong.tubemq.server.broker.metadata.TopicMetadata;
import org.apache.inlong.tubemq.server.common.fielddef.WebFieldDef;
import org.apache.inlong.tubemq.server.common.utils.ProcessResult;
import org.apache.inlong.tubemq.server.master.MasterConfig;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbConsumeGroupSettingEntity;
-import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerConfManager;
-import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.TopicPSInfoManager;
+import org.apache.inlong.tubemq.server.master.metamanage.MetaDataManager;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.GroupResCtrlEntity;
+import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerRunManager;
import org.apache.inlong.tubemq.server.master.nodemanage.nodeconsumer.ConsumerBandInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -182,10 +180,10 @@ public class PBParameterUtils {
}
public static ParamCheckResult checkConsumerInputInfo(ConsumerInfo inConsumerInfo,
- final MasterConfig masterConfig,
- final BrokerConfManager defaultBrokerConfManager,
- final TopicPSInfoManager topicPSInfoManager,
- final StringBuilder strBuffer) throws Exception {
+ MasterConfig masterConfig,
+ MetaDataManager defMetaDataManager,
+ BrokerRunManager brokerRunManager,
+ StringBuilder strBuffer) throws Exception {
ParamCheckResult retResult = new ParamCheckResult();
if (!inConsumerInfo.isRequireBound()) {
retResult.setCheckData(inConsumerInfo);
@@ -204,36 +202,22 @@ public class PBParameterUtils {
"[Parameter error] totalSourceCount must over zero!");
return retResult;
}
- BdbConsumeGroupSettingEntity offsetResetGroupEntity =
- defaultBrokerConfManager.getBdbConsumeGroupSetting(inConsumerInfo.getGroup());
+ GroupResCtrlEntity offsetResetGroupEntity =
+ defMetaDataManager.confGetGroupResCtrlConf(inConsumerInfo.getGroup());
if (masterConfig.isStartOffsetResetCheck()) {
- if ((offsetResetGroupEntity == null)
- || (offsetResetGroupEntity.getEnableBind() != 1)) {
- if (offsetResetGroupEntity == null) {
- retResult.setCheckResult(false,
- TErrCodeConstants.BAD_REQUEST,
- "[unauthorized subscribe] ConsumeGroup must be authorized by administrator before"
- + " using bound subscribe, please contact to administrator!");
- } else {
- retResult.setCheckResult(false,
- TErrCodeConstants.BAD_REQUEST,
- "[unauthorized subscribe] ConsumeGroup's authorization status is not enable for"
- + " using bound subscribe, please contact to administrator!");
- }
+ if (offsetResetGroupEntity == null) {
+ retResult.setCheckResult(false,
+ TErrCodeConstants.BAD_REQUEST,
+ "[unauthorized subscribe] ConsumeGroup must be authorized by administrator before"
+ + " using bound subscribe, please contact to administrator!");
return retResult;
}
- Date currentDate = new Date();
- Date lastDate = offsetResetGroupEntity.getLastBindUsedDate();
- if (lastDate == null
- || (lastDate.before(currentDate)
- && (int) ((lastDate.getTime() - currentDate.getTime()) / (1000 * 3600 * 8)) > 1)) {
- defaultBrokerConfManager.confUpdBdbConsumeGroupLastUsedTime(inConsumerInfo.getGroup());
- }
}
int allowRate = (offsetResetGroupEntity != null
&& offsetResetGroupEntity.getAllowedBrokerClientRate() > 0)
? offsetResetGroupEntity.getAllowedBrokerClientRate() : masterConfig.getMaxGroupBrokerConsumeRate();
- int maxBrokerCount = topicPSInfoManager.getTopicMaxBrokerCount(inConsumerInfo.getTopicSet());
+ int maxBrokerCount =
+ brokerRunManager.getSubTopicMaxBrokerCount(inConsumerInfo.getTopicSet());
int curBClientRate = (int) Math.floor(maxBrokerCount / inConsumerInfo.getSourceCount());
if (curBClientRate > allowRate) {
int minClientCnt = (int) (maxBrokerCount / allowRate);
@@ -430,7 +414,7 @@ public class PBParameterUtils {
}
String tmpValue = brokerId.trim();
try {
- Integer.parseInt(tmpValue);
+ retResult.setCheckData(Integer.parseInt(tmpValue));
} catch (Throwable e) {
retResult.setCheckResult(false,
TErrCodeConstants.BAD_REQUEST,
@@ -438,7 +422,6 @@ public class PBParameterUtils {
strBuffer.delete(0, strBuffer.length());
return retResult;
}
- retResult.setCheckData(tmpValue);
return retResult;
}
@@ -522,7 +505,7 @@ public class PBParameterUtils {
result.setFailResult(strBuffer.append("Request miss necessary ")
.append(fieldDef.name).append(" data!").toString());
strBuffer.delete(0, strBuffer.length());
- return result.success;
+ return result.isSuccess();
}
String tmpValue = paramValue.trim();
if (tmpValue.length() > fieldDef.valMaxLen) {
@@ -530,10 +513,10 @@ public class PBParameterUtils {
.append("'s length over max value, allowed max length is ")
.append(fieldDef.valMaxLen).toString());
strBuffer.delete(0, strBuffer.length());
- return result.success;
+ return result.isSuccess();
}
result.setSuccResult(tmpValue);
- return result.success;
+ return result.isSuccess();
}
/**
@@ -551,9 +534,9 @@ public class PBParameterUtils {
ProcessResult result) {
if (!getStringParameter(WebFieldDef.TOPICNAME,
topicName, strBuffer, result)) {
- return result.success;
+ return result.isSuccess();
}
- String tmpValue = (String) result.retData1;
+ String tmpValue = (String) result.getRetData();
if (metadataManager.getTopicMetadata(tmpValue) == null) {
result.setFailResult(TErrCodeConstants.FORBIDDEN,
strBuffer.append(WebFieldDef.TOPICNAME.name)
@@ -561,7 +544,7 @@ public class PBParameterUtils {
.append(" not existed, please check your configure").toString());
strBuffer.delete(0, strBuffer.length());
}
- return result.success;
+ return result.isSuccess();
}
/**
@@ -580,9 +563,9 @@ public class PBParameterUtils {
ProcessResult result) {
if (!getStringParameter(WebFieldDef.TOPICNAME,
topicName, strBuffer, result)) {
- return result.success;
+ return result.isSuccess();
}
- String tmpValue = (String) result.retData1;
+ String tmpValue = (String) result.getRetData();
TopicMetadata topicMetadata = metadataManager.getTopicMetadata(tmpValue);
if (topicMetadata == null) {
result.setFailResult(TErrCodeConstants.FORBIDDEN,
@@ -590,7 +573,7 @@ public class PBParameterUtils {
.append(" ").append(tmpValue)
.append(" not existed, please check your configure").toString());
strBuffer.delete(0, strBuffer.length());
- return result.success;
+ return result.isSuccess();
}
if (metadataManager.isClosedTopic(tmpValue)) {
result.setFailResult(TErrCodeConstants.FORBIDDEN,
@@ -598,7 +581,7 @@ public class PBParameterUtils {
.append(" ").append(tmpValue)
.append(" has been closed").toString());
strBuffer.delete(0, strBuffer.length());
- return result.success;
+ return result.isSuccess();
}
int realPartition = partitionId < TBaseConstants.META_STORE_INS_BASE
? partitionId : partitionId % TBaseConstants.META_STORE_INS_BASE;
@@ -608,9 +591,9 @@ public class PBParameterUtils {
.append(" ").append(tmpValue).append("-").append(partitionId)
.append(" not existed, please check your configure").toString());
strBuffer.delete(0, strBuffer.length());
- return result.success;
+ return result.isSuccess();
}
result.setSuccResult(topicMetadata);
- return result.success;
+ return result.isSuccess();
}
}
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/SettingValidUtils.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/CleanPolType.java
similarity index 51%
copy from inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/SettingValidUtils.java
copy to inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/CleanPolType.java
index e24a0f3..f4225c4 100644
--- a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/utils/SettingValidUtils.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/CleanPolType.java
@@ -15,24 +15,37 @@
* limitations under the License.
*/
-package org.apache.inlong.tubemq.corebase.utils;
+package org.apache.inlong.tubemq.server.common.statusdef;
-import org.apache.inlong.tubemq.corebase.TBaseConstants;
+/*
+ * The file clean policy type
+ */
+public enum CleanPolType {
+ CLEAN_POL_DELETE(0, "Delete");
-public class SettingValidUtils {
+ private int code;
+ private String description;
- public static int validAndXfeMaxMsgSizeFromMBtoB(int inMaxMsgSizeInMB) {
- return MixedUtils.mid(inMaxMsgSizeInMB,
- TBaseConstants.META_MIN_ALLOWED_MESSAGE_SIZE_MB,
- TBaseConstants.META_MAX_ALLOWED_MESSAGE_SIZE_MB)
- * TBaseConstants.META_MB_UNIT_SIZE;
+ CleanPolType(int code, String description) {
+ this.code = code;
+ this.description = description;
}
- public static int validAndGetMaxMsgSizeInB(int inMaxMsgSizeInB) {
- return MixedUtils.mid(inMaxMsgSizeInB,
- TBaseConstants.META_MAX_MESSAGE_DATA_SIZE,
- TBaseConstants.META_MAX_MESSAGE_DATA_SIZE_UPPER_LIMIT);
+ public int getCode() {
+ return code;
}
+
+
+ public static CleanPolType valueOf(int code) {
+ for (CleanPolType status : CleanPolType.values()) {
+ if (status.getCode() == code) {
+ return status;
+ }
+ }
+ throw new IllegalArgumentException(String.format(
+ "unknown clean up policy type code %s", code));
+ }
+
}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/EnableStatus.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/EnableStatus.java
new file mode 100644
index 0000000..5117abc
--- /dev/null
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/EnableStatus.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.inlong.tubemq.server.common.statusdef;
+
+/*
+ * The enable status enumeration class
+ */
+public enum EnableStatus {
+ STATUS_UNDEFINE(-2, "Undefined."),
+ STATUS_DISABLE(0, "Disable."),
+ STATUS_ENABLE(2, "Enable.");
+
+ private int code;
+ private String description;
+
+
+ EnableStatus(int code, String description) {
+ this.code = code;
+ this.description = description;
+ }
+
+ public int getCode() {
+ return code;
+ }
+
+ public boolean isEnable() {
+ return this == EnableStatus.STATUS_ENABLE;
+ }
+
+ public String getDescription() {
+ return description;
+ }
+
+ public static EnableStatus valueOf(int code) {
+ for (EnableStatus status : EnableStatus.values()) {
+ if (status.getCode() == code) {
+ return status;
+ }
+ }
+ throw new IllegalArgumentException(String.format("unknown Enable status code %s", code));
+ }
+
+}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/ManageStatus.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/ManageStatus.java
new file mode 100644
index 0000000..ed811bb
--- /dev/null
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/ManageStatus.java
@@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.server.common.statusdef;
+
+import org.apache.inlong.tubemq.corebase.utils.Tuple2;
+
+/*
+ * The management status enumeration class of the Broker node
+ */
+public enum ManageStatus {
+
+ STATUS_MANAGE_UNDEFINED(-2, "-", false, false),
+ STATUS_MANAGE_APPLY(1, "draft", false, false),
+ STATUS_MANAGE_ONLINE(5, "online", true, true),
+ STATUS_MANAGE_ONLINE_NOT_WRITE(6, "only-read", false, true),
+ STATUS_MANAGE_ONLINE_NOT_READ(7, "only-write", true, false),
+ STATUS_MANAGE_OFFLINE(9, "offline", false, false);
+
+ private int code;
+ private String description;
+ private boolean isAcceptPublish;
+ private boolean isAcceptSubscribe;
+
+
+ ManageStatus(int code, String description,
+ boolean acceptPublish,
+ boolean acceptSubscribe) {
+ this.code = code;
+ this.description = description;
+ this.isAcceptPublish = acceptPublish;
+ this.isAcceptSubscribe = acceptSubscribe;
+ }
+
+ public boolean isOnlineStatus() {
+ return (this == ManageStatus.STATUS_MANAGE_ONLINE
+ || this == ManageStatus.STATUS_MANAGE_ONLINE_NOT_WRITE
+ || this == ManageStatus.STATUS_MANAGE_ONLINE_NOT_READ);
+ }
+
+ public boolean isApplied() {
+ return (this.code > ManageStatus.STATUS_MANAGE_APPLY.getCode());
+ }
+
+ public int getCode() {
+ return code;
+ }
+
+ public String getDescription() {
+ return description;
+ }
+
+ public Tuple2<Boolean, Boolean> getPubSubStatus() {
+ return new Tuple2<>(isAcceptPublish, isAcceptSubscribe);
+ }
+
+ public boolean isAcceptSubscribe() {
+ return isAcceptSubscribe;
+ }
+
+ public boolean isAcceptPublish() {
+ return isAcceptPublish;
+ }
+
+ public static ManageStatus valueOf(int code) {
+ for (ManageStatus status : ManageStatus.values()) {
+ if (status.getCode() == code) {
+ return status;
+ }
+ }
+ throw new IllegalArgumentException(String.format(
+ "unknown broker manage status code %s", code));
+ }
+
+ /**
+ * Change broker read write status
+ *
+ * @param oldStatus current broker manage status
+ * @param acceptPublish current broker reported publish status
+ * @param acceptSubscribe current broker reported subscribe status
+ * @return current broker's new manage status
+ */
+ public static ManageStatus getNewStatus(ManageStatus oldStatus,
+ Boolean acceptPublish,
+ Boolean acceptSubscribe) {
+ if (acceptPublish == null && acceptSubscribe == null) {
+ return oldStatus;
+ }
+ boolean newPublish = oldStatus.isAcceptPublish;
+ boolean newSubscribe = oldStatus.isAcceptSubscribe;
+ if (acceptPublish != null) {
+ newPublish = acceptPublish;
+ }
+ if (acceptSubscribe != null) {
+ newSubscribe = acceptSubscribe;
+ }
+ if (newPublish) {
+ if (newSubscribe) {
+ return ManageStatus.STATUS_MANAGE_ONLINE;
+ } else {
+ return ManageStatus.STATUS_MANAGE_ONLINE_NOT_READ;
+ }
+ } else {
+ if (newSubscribe) {
+ return ManageStatus.STATUS_MANAGE_ONLINE_NOT_WRITE;
+ } else {
+ return ManageStatus.STATUS_MANAGE_OFFLINE;
+ }
+ }
+ }
+
+}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/StepStatus.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/StepStatus.java
new file mode 100644
index 0000000..404c0ca
--- /dev/null
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/StepStatus.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.inlong.tubemq.server.common.statusdef;
+
+
+/*
+ * The step status of broker operation
+ */
+public enum StepStatus {
+
+ STEP_STATUS_UNDEFINED(-2, "idle", 0, 0),
+ STEP_STATUS_LOAD_DATA(1, "load_data", 0, 0),
+ STEP_STATUS_WAIT_ONLINE(2, "wait_online", 0, 0),
+ STEP_STATUS_WAIT_SYNC(3, "wait_sync", 0, 0),
+ STEP_STATUS_WAIT_SUBSCRIBE(4, "wait_sub", 55000, 40000),
+ STEP_STATUS_WAIT_PUBLISH(5, "wait_pub", 25000, 10000);
+
+ private int code;
+ private String description;
+ private long normalDelayDurIdnMs;
+ private long shortDelayDurIdnMs;
+
+
+
+
+ StepStatus(int code, String description,
+ long normalDelayDurIdnMs, long shortDelayDurIdnMs) {
+ this.code = code;
+ this.description = description;
+ this.normalDelayDurIdnMs = normalDelayDurIdnMs;
+ this.shortDelayDurIdnMs = shortDelayDurIdnMs;
+ }
+
+ public int getCode() {
+ return code;
+ }
+
+ public String getDescription() {
+ return description;
+ }
+
+ public long getNormalDelayDurInMs() {
+ return normalDelayDurIdnMs;
+ }
+
+ public long getShortDelayDurIdnMs() {
+ return shortDelayDurIdnMs;
+ }
+
+ public static StepStatus valueOf(int code) {
+ for (StepStatus status : StepStatus.values()) {
+ if (status.getCode() == code) {
+ return status;
+ }
+ }
+ throw new IllegalArgumentException(String.format(
+ "unknown broker step status code %s", code));
+ }
+
+}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/TopicStatus.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/TopicStatus.java
new file mode 100644
index 0000000..78dacd9
--- /dev/null
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/TopicStatus.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.server.common.statusdef;
+
+/*
+ * The topic state life cycle
+ */
+public enum TopicStatus {
+ STATUS_TOPIC_UNDEFINED(-2, "Undefined"),
+ STATUS_TOPIC_OK(0, "Normal"),
+ STATUS_TOPIC_SOFT_DELETE(1, "Soft deleted"),
+ STATUS_TOPIC_SOFT_REMOVE(2, "Soft removed"),
+ STATUS_TOPIC_HARD_REMOVE(3, "Hard removed");
+
+ private int code;
+ private String description;
+
+
+ TopicStatus(int code, String description) {
+ this.code = code;
+ this.description = description;
+ }
+
+ public int getCode() {
+ return code;
+ }
+
+ public String getDescription() {
+ return description;
+ }
+
+ public static TopicStatus valueOf(int code) {
+ for (TopicStatus status : TopicStatus.values()) {
+ if (status.getCode() == code) {
+ return status;
+ }
+ }
+ throw new IllegalArgumentException(String.format("unknown topic status code %s", code));
+ }
+
+}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/TopicStsChgType.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/TopicStsChgType.java
new file mode 100644
index 0000000..94dc285
--- /dev/null
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/statusdef/TopicStsChgType.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.server.common.statusdef;
+
+/*
+ * The topic status change step
+ */
+public enum TopicStsChgType {
+ STATUS_CHANGE_SOFT_DELETE(0, "Soft deleted"),
+ STATUS_CHANGE_REMOVE(1, "Soft removed"),
+ STATUS_CHANGE_REDO_SFDEL(2, "Redo soft delete");
+
+ private int code;
+ private String description;
+
+
+ TopicStsChgType(int code, String description) {
+ this.code = code;
+ this.description = description;
+ }
+
+ public int getCode() {
+ return code;
+ }
+
+ public String getDescription() {
+ return description;
+ }
+
+ public static TopicStsChgType valueOf(int code) {
+ for (TopicStsChgType changeType : TopicStsChgType.values()) {
+ if (changeType.getCode() == code) {
+ return changeType;
+ }
+ }
+ throw new IllegalArgumentException(String.format("unknown status change code %s", code));
+ }
+
+}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/utils/ProcessResult.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/utils/ProcessResult.java
index 80c51c7..55a630e 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/utils/ProcessResult.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/utils/ProcessResult.java
@@ -29,6 +29,13 @@ public class ProcessResult {
}
+ public ProcessResult(ProcessResult other) {
+ this.success = other.success;
+ this.errCode = other.errCode;
+ this.errInfo = other.errInfo;
+ this.retData1 = other.retData1;
+ }
+
public ProcessResult(Object retData) {
this.success = true;
this.retData1 = retData;
@@ -56,8 +63,35 @@ public class ProcessResult {
public void setSuccResult(Object retData) {
this.success = true;
- this.errInfo = "";
+ this.errInfo = "Ok!";
this.errCode = TErrCodeConstants.SUCCESS;
this.retData1 = retData;
}
+
+ public boolean isSuccess() {
+ return success;
+ }
+
+ public int getErrCode() {
+ return errCode;
+ }
+
+ public String getErrInfo() {
+ return errInfo;
+ }
+
+ public Object getRetData() {
+ return retData1;
+ }
+
+ public void setRetData(Object retData) {
+ this.retData1 = retData;
+ }
+
+ public void clear() {
+ this.success = true;
+ this.errCode = TErrCodeConstants.SUCCESS;
+ this.errInfo = "";
+ this.retData1 = null;
+ }
}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/nodemanage/nodebroker/TargetValidResult.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/utils/SerialIdUtils.java
similarity index 55%
copy from inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/nodemanage/nodebroker/TargetValidResult.java
copy to inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/utils/SerialIdUtils.java
index e8f9fe1..d91a96d 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/nodemanage/nodebroker/TargetValidResult.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/utils/SerialIdUtils.java
@@ -1,32 +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.inlong.tubemq.server.master.nodemanage.nodebroker;
-
-
-public class TargetValidResult {
- public boolean result;
- public int errCode;
- public String errInfo;
-
- public TargetValidResult(boolean result, int errCode, final String errInfo) {
- this.result = result;
- this.errCode = errCode;
- this.errInfo = errInfo;
- }
-
-}
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.server.common.utils;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+
+
+public class SerialIdUtils {
+
+ public static void updTimeStampSerialIdValue(final AtomicLong serialId) {
+ long curSerialId = serialId.get();
+ long newSerialId = System.currentTimeMillis();
+ do {
+ if (newSerialId > curSerialId) {
+ if (serialId.compareAndSet(curSerialId, newSerialId)) {
+ break;
+ }
+ }
+ curSerialId = serialId.get();
+ newSerialId = curSerialId + 10;
+ } while (true);
+ }
+}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/utils/WebParameterUtils.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/utils/WebParameterUtils.java
index c225ff5..eeb5d22 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/utils/WebParameterUtils.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/utils/WebParameterUtils.java
@@ -33,401 +33,691 @@ import java.util.TreeSet;
import javax.servlet.http.HttpServletRequest;
import org.apache.inlong.tubemq.corebase.TBaseConstants;
import org.apache.inlong.tubemq.corebase.TokenConstants;
+import org.apache.inlong.tubemq.corebase.policies.FlowCtrlItem;
+import org.apache.inlong.tubemq.corebase.policies.FlowCtrlRuleHandler;
import org.apache.inlong.tubemq.corebase.utils.TStringUtils;
import org.apache.inlong.tubemq.corebase.utils.Tuple2;
import org.apache.inlong.tubemq.server.broker.utils.DataStoreUtils;
import org.apache.inlong.tubemq.server.common.TServerConstants;
import org.apache.inlong.tubemq.server.common.TStatusConstants;
import org.apache.inlong.tubemq.server.common.fielddef.WebFieldDef;
+import org.apache.inlong.tubemq.server.common.statusdef.TopicStatus;
import org.apache.inlong.tubemq.server.master.TMaster;
import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbBrokerConfEntity;
-import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerConfManager;
-import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerSyncStatusInfo;
+import org.apache.inlong.tubemq.server.master.metamanage.DataOpErrCode;
+import org.apache.inlong.tubemq.server.master.metamanage.MetaDataManager;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.BaseEntity;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.TopicPropGroup;
public class WebParameterUtils {
private static final List<String> allowedDelUnits = Arrays.asList("s", "m", "h");
+ private static final List<Integer> allowedPriorityVal = Arrays.asList(1, 2, 3);
- /**
- * Parse the parameter value from an object value to a integer value
- *
- * @param paramName the parameter name
- * @param paramValue the parameter value which is an object for parsing
- * @param required a boolean value represent whether the parameter is must required
- * @param defaultValue a default value returned if failed to parse value from the given object
- * @return a int value of parameter
- * @throws Exception if failed to parse the object
- */
- public static int validIntDataParameter(String paramName, String paramValue,
- boolean required, int defaultValue,
- int minValue) throws Exception {
- String tmpParamValue = checkParamCommonRequires(paramName, paramValue, required);
- if (TStringUtils.isBlank(tmpParamValue)) {
- return defaultValue;
- }
- if (!tmpParamValue.matches(TBaseConstants.META_TMP_NUMBER_VALUE)) {
- throw new Exception(new StringBuilder(512)
- .append("the value of ").append(paramName)
- .append(" parameter must only contain numbers").toString());
- }
- int tmpInteger = Integer.parseInt(tmpParamValue);
- if (tmpInteger < minValue) {
- throw new Exception(new StringBuilder(512)
- .append("the value of ").append(paramName)
- .append(" parameter must >= ").append(minValue).toString());
- }
- return tmpInteger;
+
+ public static StringBuilder buildFailResult(StringBuilder sBuffer, String errMsg) {
+ return sBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
+ .append(errMsg).append("\"}");
}
- /**
- * Parse the parameter value from an object value to a boolean value
- *
- * @param paramName the parameter name
- * @param paramValue the parameter value which is an object for parsing
- * @param required a boolean value represent whether the parameter is must required
- * @param defaultValue a default value returned if failed to parse value from the given object
- * @return a boolean value of parameter
- * @throws Exception if failed to parse the object
- */
- public static boolean validBooleanDataParameter(String paramName, String paramValue,
- boolean required, boolean defaultValue) throws Exception {
- String tmpParamValue = checkParamCommonRequires(paramName, paramValue, required);
- if (TStringUtils.isBlank(tmpParamValue)) {
- return defaultValue;
- }
- return Boolean.parseBoolean(tmpParamValue);
+ public static StringBuilder buildFailResultWithBlankData(int errcode, String errMsg,
+ StringBuilder sBuffer) {
+ return sBuffer.append("{\"result\":false,\"errCode\":").append(errcode)
+ .append(",\"errMsg\":\"").append(errMsg).append("\",\"data\":[]}");
+ }
+
+ public static StringBuilder buildSuccessResult(StringBuilder sBuffer) {
+ return sBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"OK\",\"data\":[]}");
+ }
+
+ public static StringBuilder buildSuccessResult(StringBuilder sBuffer, String appendInfo) {
+ return sBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"")
+ .append(appendInfo).append("\",\"data\":[]}");
+ }
+
+ public static StringBuilder buildSuccessWithDataRetBegin(StringBuilder sBuffer) {
+ return sBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"OK\",\"data\":[");
+ }
+
+ public static StringBuilder buildSuccessWithDataRetEnd(
+ StringBuilder sBuffer, int totalCnt) {
+ return sBuffer.append("],\"count\":").append(totalCnt).append("}");
}
/**
- * Parse the parameter value from an object value to Date value
+ * Parse the parameter value required for add update and delete
*
- * @param paramName the parameter name
- * @param paramValue the parameter value which is an object for parsing
- * @param required a boolean value represent whether the parameter is must required
- * @param defaultValue a default value returned if failed to parse value from the given object
- * @return a Date value of parameter
- * @throws Exception if failed to parse the object
+ * @param paramCntr parameter container object
+ * @param isAdd if add operation
+ * @param defOpEntity default value set,
+ * if not null, it must fill required field values
+ * @param sBuffer string buffer
+ * @param result process result of parameter value
+ * @return the process result
*/
- public static Date validDateParameter(String paramName, String paramValue, int paramMaxLen,
- boolean required, Date defaultValue) throws Exception {
- String tmpParamValue = checkParamCommonRequires(paramName, paramValue, required);
- if (TStringUtils.isBlank(tmpParamValue)) {
- return defaultValue;
- }
- // yyyyMMddHHmmss
- if (tmpParamValue.length() > paramMaxLen) {
- throw new Exception(new StringBuilder(512)
- .append("the date format is yyyyMMddHHmmss of ")
- .append(paramName).append(" parameter").toString());
- }
- if (!tmpParamValue.matches(TBaseConstants.META_TMP_NUMBER_VALUE)) {
- throw new Exception(new StringBuilder(512).append("the value of ")
- .append(paramName).append(" parameter must only contain numbers").toString());
- }
- DateFormat sdf = new SimpleDateFormat(TBaseConstants.META_TMP_DATE_VALUE);
- Date date = sdf.parse(tmpParamValue);
- return date;
+ public static <T> boolean getAUDBaseInfo(T paramCntr, boolean isAdd,
+ BaseEntity defOpEntity,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ // check and get data version id
+ if (!WebParameterUtils.getLongParamValue(paramCntr, WebFieldDef.DATAVERSIONID,
+ false, (defOpEntity == null
+ ? TBaseConstants.META_VALUE_UNDEFINED : defOpEntity.getDataVerId()),
+ sBuffer, result)) {
+ return result.isSuccess();
+ }
+ final long dataVerId = (long) result.getRetData();
+ // check and get createUser or modifyUser
+ String createUsr = null;
+ Date createDate = new Date();
+ if (isAdd) {
+ // check create user field
+ if (!WebParameterUtils.getStringParamValue(paramCntr, WebFieldDef.CREATEUSER,
+ defOpEntity == null,
+ (defOpEntity == null ? createUsr : defOpEntity.getCreateUser()),
+ sBuffer, result)) {
+ return result.isSuccess();
+ }
+ createUsr = (String) result.getRetData();
+ // check and get create date
+ if (!WebParameterUtils.getDateParameter(paramCntr, WebFieldDef.CREATEDATE, false,
+ ((defOpEntity == null || defOpEntity.getCreateDate() == null)
+ ? createDate : defOpEntity.getCreateDate()),
+ sBuffer, result)) {
+ return result.isSuccess();
+ }
+ createDate = (Date) result.getRetData();
+ }
+ // check modify user field
+ if (!WebParameterUtils.getStringParamValue(paramCntr, WebFieldDef.MODIFYUSER,
+ (defOpEntity == null && !isAdd),
+ (defOpEntity == null ? createUsr : defOpEntity.getModifyUser()),
+ sBuffer, result)) {
+ return result.isSuccess();
+ }
+ String modifyUser = (String) result.getRetData();
+ // check and get modify date
+ if (!WebParameterUtils.getDateParameter(paramCntr, WebFieldDef.MODIFYDATE, false,
+ ((defOpEntity == null || defOpEntity.getModifyDate() == null)
+ ? createDate : defOpEntity.getModifyDate()),
+ sBuffer, result)) {
+ return result.isSuccess();
+ }
+ Date modifyDate = (Date) result.getRetData();
+ result.setSuccResult(new BaseEntity(dataVerId,
+ createUsr, createDate, modifyUser, modifyDate));
+ return result.isSuccess();
}
/**
- * Parse the parameter value from an object value to string value
+ * Parse the parameter value required for query
*
- * @param req http servlet request
- * @param paramName the parameter name
- * @param paramMaxLen the max length of string to return
- * @param required a boolean value represent whether the parameter is must required
- * @param defaultValue a default value returned if failed to parse value from the given object
- * @return a string value of parameter
- * @throws Exception if failed to parse the object
+ * @param req Http Servlet Request
+ * @param qryEntity query entity
+ * @param sBuffer string buffer
+ * @param result process result
+ * @return process result
*/
- public static String validStringParameter(HttpServletRequest req, String paramName,
- int paramMaxLen, boolean required,
- String defaultValue) throws Exception {
- return validStringParameter(paramName,
- req.getParameter(paramName), paramMaxLen, required, defaultValue);
+ public static boolean getQueriedOperateInfo(HttpServletRequest req, BaseEntity qryEntity,
+ StringBuilder sBuffer, ProcessResult result) {
+ // check and get data version id
+ if (!WebParameterUtils.getLongParamValue(req, WebFieldDef.DATAVERSIONID,
+ false, TBaseConstants.META_VALUE_UNDEFINED, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ long dataVerId = (long) result.getRetData();
+ // check createUser user field
+ if (!WebParameterUtils.getStringParamValue(req,
+ WebFieldDef.CREATEUSER, false, null, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ String createUser = (String) result.getRetData();
+ // check modify user field
+ if (!WebParameterUtils.getStringParamValue(req,
+ WebFieldDef.MODIFYUSER, false, null, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ String modifyUser = (String) result.getRetData();
+ // set query keys
+ qryEntity.updQueryKeyInfo(dataVerId, createUser, modifyUser);
+ result.setSuccResult(qryEntity);
+ return result.isSuccess();
}
/**
- * Parse the parameter value from an object value to string value
+ * Parse the parameter value required for query
*
- * @param paramName the parameter name
- * @param paramValue the parameter value which is an object for parsing
- * @param paramMaxLen the max length of string to return
- * @param required a boolean value represent whether the parameter is must required
- * @param defaultValue a default value returned if failed to parse value from the given object
- * @return a string value of parameter
- * @throws Exception if failed to parse the object
+ * @param paramCntr parameter container
+ * @param required whether required
+ * @param defValue default value
+ * @param minValue min value
+ * @param sBuffer string buffer
+ * @param result process result
+ * @return process result
*/
- public static String validStringParameter(String paramName, String paramValue,
- int paramMaxLen, boolean required,
- String defaultValue) throws Exception {
- String tmpParamValue =
- checkParamCommonRequires(paramName, paramValue, required);
- if (TStringUtils.isBlank(tmpParamValue)) {
- return defaultValue;
+ // get QryPriorityId parameter value
+ public static <T> boolean getQryPriorityIdParameter(T paramCntr, boolean required,
+ int defValue, int minValue,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ if (!getIntParamValue(paramCntr, WebFieldDef.QRYPRIORITYID,
+ required, defValue, minValue, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ int qryPriorityId = (int) result.getRetData();
+ if (qryPriorityId > 303 || qryPriorityId < 101) {
+ result.setFailResult(sBuffer.append("Illegal value in ")
+ .append(WebFieldDef.QRYPRIORITYID.name)
+ .append(" parameter: ").append(WebFieldDef.QRYPRIORITYID.name)
+ .append(" value must be greater than or equal")
+ .append(" to 101 and less than or equal to 303!").toString());
+ sBuffer.delete(0, sBuffer.length());
+ return false;
}
- if (paramMaxLen != TBaseConstants.META_VALUE_UNDEFINED) {
- if (tmpParamValue.length() > paramMaxLen) {
- throw new Exception(new StringBuilder(512).append("the max length of ")
- .append(paramName).append(" parameter is ")
- .append(paramMaxLen).append(" characters").toString());
- }
+ if (!allowedPriorityVal.contains(qryPriorityId % 100)) {
+ result.setFailResult(sBuffer.append("Illegal value in ")
+ .append(WebFieldDef.QRYPRIORITYID.name).append(" parameter: the units of ")
+ .append(WebFieldDef.QRYPRIORITYID.name).append(" must in ")
+ .append(allowedPriorityVal).toString());
+ sBuffer.delete(0, sBuffer.length());
+ return false;
}
- if (!tmpParamValue.matches(TBaseConstants.META_TMP_STRING_VALUE)) {
- throw new Exception(new StringBuilder(512).append("the value of ")
- .append(paramName).append(" parameter must begin with a letter, ")
- .append("can only contain characters,numbers,and underscores").toString());
+ if (!allowedPriorityVal.contains(qryPriorityId / 100)) {
+ result.setFailResult(sBuffer.append("Illegal value in ")
+ .append(WebFieldDef.QRYPRIORITYID.name)
+ .append(" parameter: the hundreds of ").append(WebFieldDef.QRYPRIORITYID.name)
+ .append(" must in ").append(allowedPriorityVal).toString());
+ sBuffer.delete(0, sBuffer.length());
+ return false;
}
- return tmpParamValue;
+ result.setSuccResult(qryPriorityId);
+ return result.isSuccess();
}
/**
- * Parse the parameter value from an object value to group string value
+ * Decode the deletePolicy parameter value from an object value
+ * the value must like {method},{digital}[s|m|h]
*
- * @param paramName the parameter name
- * @param paramValue the parameter value which is an object for parsing
- * @param paramMaxLen the max length of string to return
- * @param required a boolean value represent whether the parameter is must required
- * @param defaultValue a default value returned if failed to parse value from the given object
- * @return a string value of parameter
- * @throws Exception if failed to parse the object
+ * @param paramCntr parameter container object
+ * @param required a boolean value represent whether the parameter is must required
+ * @param defValue a default value returned if failed to parse value from the given object
+ * @param sBuffer string buffer
+ * @param result process result of parameter value
+ * @return the process result
*/
- public static String validGroupParameter(String paramName, String paramValue, int paramMaxLen,
- boolean required, String defaultValue) throws Exception {
- String tmpParamValue = checkParamCommonRequires(paramName, paramValue, required);
- if (TStringUtils.isBlank(tmpParamValue)) {
- return defaultValue;
+ public static <T> boolean getDeletePolicyParameter(T paramCntr, boolean required,
+ String defValue, StringBuilder sBuffer,
+ ProcessResult result) {
+ if (!WebParameterUtils.getStringParamValue(paramCntr,
+ WebFieldDef.DELETEPOLICY, required, defValue, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ String delPolicy = (String) result.getRetData();
+ if (TStringUtils.isBlank(delPolicy)) {
+ return result.isSuccess();
+ }
+ // check value format
+ String[] tmpStrs = delPolicy.split(",");
+ if (tmpStrs.length != 2) {
+ result.setFailResult(DataOpErrCode.DERR_ILLEGAL_VALUE.getCode(),
+ sBuffer.append("Value must include one and only one comma character,")
+ .append(" the format of ").append(WebFieldDef.DELETEPOLICY.name())
+ .append(" must like {method},{digital}[m|s|h]").toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
}
- if (paramMaxLen != TBaseConstants.META_VALUE_UNDEFINED) {
- if (tmpParamValue.length() > paramMaxLen) {
- throw new Exception(new StringBuilder(512).append("the max length of ")
- .append(paramName).append(" parameter is ")
- .append(paramMaxLen).append(" characters").toString());
+ if (TStringUtils.isBlank(tmpStrs[0])) {
+ result.setFailResult(DataOpErrCode.DERR_ILLEGAL_VALUE.getCode(),
+ sBuffer.append("Method value must not be blank, the format of ")
+ .append(WebFieldDef.DELETEPOLICY.name())
+ .append(" must like {method},{digital}[m|s|h]").toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
+ }
+ if (!"delete".equalsIgnoreCase(tmpStrs[0].trim())) {
+ result.setFailResult(DataOpErrCode.DERR_ILLEGAL_VALUE.getCode(),
+ sBuffer.append("Field ").append(WebFieldDef.DELETEPOLICY.name())
+ .append(" only support delete method now!").toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
+ }
+ String validValStr = tmpStrs[1];
+ String timeUnit = validValStr.substring(validValStr.length() - 1).toLowerCase();
+ if (Character.isLetter(timeUnit.charAt(0))) {
+ if (!allowedDelUnits.contains(timeUnit)) {
+ result.setFailResult(DataOpErrCode.DERR_ILLEGAL_VALUE.getCode(),
+ sBuffer.append("Field ").append(WebFieldDef.DELETEPOLICY.name())
+ .append(" only support [s|m|h] unit!").toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
}
}
- if (!tmpParamValue.matches(TBaseConstants.META_TMP_GROUP_VALUE)) {
- throw new Exception(new StringBuilder(512).append("the value of ")
- .append(paramName).append(" parameter must begin with a letter, ")
- .append("can only contain characters,numbers,hyphen,and underscores").toString());
+ long validDuration = 0;
+ try {
+ if (timeUnit.endsWith("s")) {
+ validDuration = Long.parseLong(validValStr.substring(0, validValStr.length() - 1)) * 1000;
+ } else if (timeUnit.endsWith("m")) {
+ validDuration = Long.parseLong(validValStr.substring(0, validValStr.length() - 1)) * 60000;
+ } else if (timeUnit.endsWith("h")) {
+ validDuration = Long.parseLong(validValStr.substring(0, validValStr.length() - 1)) * 3600000;
+ } else {
+ validDuration = Long.parseLong(validValStr) * 3600000;
+ }
+ } catch (Throwable e) {
+ result.setFailResult(DataOpErrCode.DERR_ILLEGAL_VALUE.getCode(),
+ sBuffer.append("The value of field ")
+ .append(WebFieldDef.DELETEPOLICY.name())
+ .append("'s valid duration must digits!").toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
}
- return tmpParamValue;
- }
-
- public static StringBuilder buildFailResult(StringBuilder strBuffer, String errMsg) {
- return strBuffer.append("{\"result\":false,\"errCode\":400,\"errMsg\":\"")
- .append(errMsg).append("\"}");
- }
-
- public static StringBuilder buildSuccessResult(StringBuilder strBuffer) {
- return strBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"OK\"}");
- }
-
- public static StringBuilder buildSuccessResult(StringBuilder strBuffer, String appendInfo) {
- return strBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"")
- .append(appendInfo).append("\"}");
+ if (validDuration <= 0 || validDuration > DataStoreUtils.MAX_FILE_VALID_DURATION) {
+ result.setFailResult(DataOpErrCode.DERR_ILLEGAL_VALUE.getCode(),
+ sBuffer.append("The value of field ")
+ .append(WebFieldDef.DELETEPOLICY.name())
+ .append(" must be greater than 0 and less than or equal to")
+ .append(DataStoreUtils.MAX_FILE_VALID_DURATION)
+ .append(" seconds!").toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
+ }
+ if (Character.isLetter(timeUnit.charAt(0))) {
+ result.setSuccResult(sBuffer.append("delete,")
+ .append(validValStr.substring(0, validValStr.length() - 1))
+ .append(timeUnit).toString());
+ } else {
+ result.setSuccResult(sBuffer.append("delete,")
+ .append(validValStr).append("h").toString());
+ }
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
}
- public static StringBuilder buildSuccessWithDataRetBegin(StringBuilder strBuffer) {
- return strBuffer.append("{\"result\":true,\"errCode\":0,\"errMsg\":\"OK\",\"data\":[");
+ /**
+ * get topic status parameter value
+ *
+ * @param req Http Servlet Request
+ * @param isRequired whether required
+ * @param defVal a default value returned if failed to parse value from the given object
+ * @param sBuffer string buffer
+ * @param result process result of parameter value
+ * @return the process result
+ */
+ public static boolean getTopicStatusParamValue(HttpServletRequest req,
+ boolean isRequired,
+ TopicStatus defVal,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ // get topicStatusId field
+ if (!WebParameterUtils.getIntParamValue(req, WebFieldDef.TOPICSTATUSID,
+ isRequired, defVal.getCode(), TopicStatus.STATUS_TOPIC_UNDEFINED.getCode(),
+ sBuffer, result)) {
+ return result.isSuccess();
+ }
+ int paramValue = (int) result.getRetData();
+ try {
+ TopicStatus topicStatus = TopicStatus.valueOf(paramValue);
+ result.setSuccResult(topicStatus);
+ } catch (Throwable e) {
+ result.setFailResult(DataOpErrCode.DERR_ILLEGAL_VALUE.getCode(),
+ sBuffer.append("The value of field ")
+ .append(WebFieldDef.TOPICSTATUSID.name)
+ .append(" invalid:").append(e.getMessage()).toString());
+ sBuffer.delete(0, sBuffer.length());
+ }
+ return result.isSuccess();
}
- public static StringBuilder buildSuccessWithDataRetEnd(
- StringBuilder strBuffer, int totalCnt) {
- return strBuffer.append("],\"count\":").append(totalCnt).append("}");
+ /**
+ * Parse the parameter value for TopicPropGroup class
+ *
+ * @param paramCntr parameter container object
+ * @param defVal default value
+ * @param sBuffer string buffer
+ * @param result process result of parameter value
+ * @return process result
+ */
+ public static <T> boolean getTopicPropInfo(T paramCntr, TopicPropGroup defVal,
+ StringBuilder sBuffer, ProcessResult result) {
+ TopicPropGroup newConf = new TopicPropGroup();
+ // get numTopicStores parameter value
+ if (!WebParameterUtils.getIntParamValue(paramCntr, WebFieldDef.NUMTOPICSTORES, false,
+ (defVal == null ? TBaseConstants.META_VALUE_UNDEFINED : defVal.getNumTopicStores()),
+ TServerConstants.TOPIC_STOREBLOCK_NUM_MIN, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ newConf.setNumTopicStores((int) result.retData1);
+ // get numPartitions parameter value
+ if (!WebParameterUtils.getIntParamValue(paramCntr, WebFieldDef.NUMPARTITIONS, false,
+ (defVal == null ? TBaseConstants.META_VALUE_UNDEFINED : defVal.getNumPartitions()),
+ TServerConstants.TOPIC_PARTITION_NUM_MIN, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ newConf.setNumPartitions((int) result.retData1);
+ // get unflushThreshold parameter value
+ if (!WebParameterUtils.getIntParamValue(paramCntr, WebFieldDef.UNFLUSHTHRESHOLD, false,
+ (defVal == null ? TBaseConstants.META_VALUE_UNDEFINED : defVal.getUnflushThreshold()),
+ TServerConstants.TOPIC_DSK_UNFLUSHTHRESHOLD_MIN, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ newConf.setUnflushThreshold((int) result.retData1);
+ // get unflushInterval parameter value
+ if (!WebParameterUtils.getIntParamValue(paramCntr, WebFieldDef.UNFLUSHINTERVAL, false,
+ (defVal == null ? TBaseConstants.META_VALUE_UNDEFINED : defVal.getUnflushInterval()),
+ TServerConstants.TOPIC_DSK_UNFLUSHINTERVAL_MIN, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ newConf.setUnflushInterval((int) result.retData1);
+ // get unflushDataHold parameter value
+ if (!WebParameterUtils.getIntParamValue(paramCntr, WebFieldDef.UNFLUSHDATAHOLD, false,
+ (defVal == null ? TBaseConstants.META_VALUE_UNDEFINED : defVal.getUnflushDataHold()),
+ TServerConstants.TOPIC_DSK_UNFLUSHDATAHOLD_MIN, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ newConf.setUnflushDataHold((int) result.retData1);
+ // get memCacheMsgSizeInMB parameter value
+ if (!WebParameterUtils.getIntParamValue(paramCntr, WebFieldDef.MCACHESIZEINMB, false,
+ (defVal == null ? TBaseConstants.META_VALUE_UNDEFINED : defVal.getMemCacheMsgSizeInMB()),
+ TServerConstants.TOPIC_CACHESIZE_MB_MIN,
+ TServerConstants.TOPIC_CACHESIZE_MB_MAX, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ newConf.setMemCacheMsgSizeInMB((int) result.retData1);
+ // get memCacheFlushIntvl parameter value
+ if (!WebParameterUtils.getIntParamValue(paramCntr, WebFieldDef.UNFMCACHEINTERVAL, false,
+ (defVal == null ? TBaseConstants.META_VALUE_UNDEFINED : defVal.getMemCacheFlushIntvl()),
+ TServerConstants.TOPIC_CACHEINTVL_MIN, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ newConf.setMemCacheFlushIntvl((int) result.retData1);
+ // get memCacheMsgCntInK parameter value
+ if (!WebParameterUtils.getIntParamValue(paramCntr, WebFieldDef.UNFMCACHECNTINK, false,
+ (defVal == null ? TBaseConstants.META_VALUE_UNDEFINED : defVal.getMemCacheMsgCntInK()),
+ TServerConstants.TOPIC_CACHECNT_INK_MIN, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ newConf.setMemCacheMsgCntInK((int) result.retData1);
+ // get deletePolicy parameter value
+ if (!WebParameterUtils.getDeletePolicyParameter(paramCntr, false,
+ (defVal == null ? null : defVal.getDeletePolicy()), sBuffer, result)) {
+ return result.isSuccess();
+ }
+ newConf.setDeletePolicy((String) result.retData1);
+ // get acceptPublish parameter value
+ if (!WebParameterUtils.getBooleanParamValue(paramCntr, WebFieldDef.ACCEPTPUBLISH, false,
+ (defVal == null ? null : defVal.getAcceptPublish()), sBuffer, result)) {
+ return result.isSuccess();
+ }
+ newConf.setAcceptPublish((Boolean) result.retData1);
+ // get acceptSubscribe parameter value
+ if (!WebParameterUtils.getBooleanParamValue(paramCntr, WebFieldDef.ACCEPTSUBSCRIBE, false,
+ (defVal == null ? null : defVal.getAcceptSubscribe()), sBuffer, result)) {
+ return result.isSuccess();
+ }
+ newConf.setAcceptSubscribe((Boolean) result.retData1);
+ result.setSuccResult(newConf);
+ return result.isSuccess();
}
/**
* Parse the parameter value from an object value to a long value
*
- * @param req Http Servlet Request
+ * @param paramCntr parameter container object
* @param fieldDef the parameter field definition
* @param required a boolean value represent whether the parameter is must required
* @param defValue a default value returned if the field not exist
+ * @param sBuffer string buffer
* @param result process result of parameter value
* @return process result
*/
- public static boolean getLongParamValue(HttpServletRequest req,
- WebFieldDef fieldDef,
- boolean required,
- long defValue,
- ProcessResult result) {
- if (!getStringParamValue(req, fieldDef, required, null, result)) {
- return result.success;
- }
- String paramValue = (String) result.retData1;
+ public static <T> boolean getLongParamValue(T paramCntr, WebFieldDef fieldDef,
+ boolean required, long defValue,
+ StringBuilder sBuffer, ProcessResult result) {
+ if (!getStringParamValue(paramCntr, fieldDef,
+ required, null, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ String paramValue = (String) result.getRetData();
if (paramValue == null) {
result.setSuccResult(defValue);
- return result.success;
+ return result.isSuccess();
}
try {
long paramIntVal = Long.parseLong(paramValue);
result.setSuccResult(paramIntVal);
} catch (Throwable e) {
- result.setFailResult(new StringBuilder(512)
- .append("Parameter ").append(fieldDef.name)
+ result.setFailResult(sBuffer.append("Parameter ").append(fieldDef.name)
.append(" parse error: ").append(e.getMessage()).toString());
+ sBuffer.delete(0, sBuffer.length());
+ }
+ return result.isSuccess();
+ }
+
+ /**
+ * Parse the parameter value from an object value to a Boolean value
+ *
+ * @param paramCntr parameter container object
+ * @param required a boolean value represent whether the parameter is must required
+ * @param defValue default value
+ * @param sBuffer string buffer
+ * @param result process result of parameter value
+ * @return process result
+ */
+ public static <T> boolean getFlowCtrlStatusParamValue(T paramCntr, boolean required,
+ Boolean defValue, StringBuilder sBuffer,
+ ProcessResult result) {
+ // check and get statusId field
+ if (!WebParameterUtils.getIntParamValue(paramCntr, WebFieldDef.STATUSID, required,
+ TBaseConstants.META_VALUE_UNDEFINED, 0, 1, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ int paramValue = (int) result.getRetData();
+ if (paramValue == TBaseConstants.META_VALUE_UNDEFINED) {
+ return defValue;
+ } else {
+ if (paramValue == 1) {
+ return Boolean.TRUE;
+ } else {
+ return Boolean.FALSE;
+ }
}
- return result.success;
}
/**
* Parse the parameter value from an object value to a integer value
*
- * @param req Http Servlet Request
+ * @param paramCntr parameter container object
* @param fieldDef the parameter field definition
* @param required a boolean value represent whether the parameter is must required
+ * @param sBuffer string buffer
* @param result process result of parameter value
* @return process result
*/
- public static boolean getIntParamValue(HttpServletRequest req,
- WebFieldDef fieldDef,
- boolean required,
- ProcessResult result) {
- return getIntParamValue(req, fieldDef, required,
+ public static <T> boolean getIntParamValue(T paramCntr, WebFieldDef fieldDef,
+ boolean required, StringBuilder sBuffer,
+ ProcessResult result) {
+ return getIntParamValue(paramCntr, fieldDef, required,
false, TBaseConstants.META_VALUE_UNDEFINED,
false, TBaseConstants.META_VALUE_UNDEFINED,
false, TBaseConstants.META_VALUE_UNDEFINED,
- result);
+ sBuffer, result);
}
/**
* Parse the parameter value from an object value to a integer value
*
- * @param req Http Servlet Request
+ * @param paramCntr parameter container object
* @param fieldDef the parameter field definition
* @param required a boolean value represent whether the parameter is must required
* @param defValue a default value returned if the field not exist
* @param minValue min value required
+ * @param sBuffer string buffer
* @param result process result of parameter value
* @return process result
*/
- public static boolean getIntParamValue(HttpServletRequest req,
- WebFieldDef fieldDef,
- boolean required,
- int defValue,
- int minValue,
- ProcessResult result) {
- return getIntParamValue(req, fieldDef, required, true, defValue,
- true, minValue, false, TBaseConstants.META_VALUE_UNDEFINED, result);
+ public static <T> boolean getIntParamValue(T paramCntr, WebFieldDef fieldDef,
+ boolean required, int defValue, int minValue,
+ StringBuilder sBuffer, ProcessResult result) {
+ return getIntParamValue(paramCntr, fieldDef, required,
+ true, defValue, true, minValue,
+ false, TBaseConstants.META_VALUE_UNDEFINED,
+ sBuffer, result);
}
/**
* Parse the parameter value from an object value to a integer value
*
- * @param req Http Servlet Request
+ * @param paramCntr parameter container object
* @param fieldDef the parameter field definition
* @param required a boolean value represent whether the parameter is must required
* @param defValue a default value returned if the field not exist
* @param minValue min value required
* @param maxValue max value allowed
+ * @param sBuffer string buffer
* @param result process result of parameter value
* @return process result
*/
- public static boolean getIntParamValue(HttpServletRequest req,
- WebFieldDef fieldDef,
- boolean required,
- int defValue,
- int minValue,
- int maxValue,
- ProcessResult result) {
- return getIntParamValue(req, fieldDef, required, true, defValue,
- true, minValue, true, maxValue, result);
+ public static <T> boolean getIntParamValue(T paramCntr, WebFieldDef fieldDef,
+ boolean required, int defValue,
+ int minValue, int maxValue,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ return getIntParamValue(paramCntr, fieldDef, required, true,
+ defValue, true, minValue, true, maxValue, sBuffer, result);
}
- private static boolean getIntParamValue(HttpServletRequest req,
- WebFieldDef fieldDef,
- boolean required,
- boolean hasDefVal,
- int defValue,
- boolean hasMinVal,
- int minValue,
- boolean hasMaxVal,
- int maxValue,
- ProcessResult result) {
- if (!getStringParamValue(req, fieldDef, required, null, result)) {
- return result.success;
+ // get int value from parameter string value
+ private static <T> boolean getIntParamValue(T paramCntr,
+ WebFieldDef fieldDef, boolean required,
+ boolean hasDefVal, int defValue,
+ boolean hasMinVal, int minValue,
+ boolean hasMaxVal, int maxValue,
+ StringBuilder sBuffer, ProcessResult result) {
+ if (!getStringParamValue(paramCntr, fieldDef,
+ required, null, sBuffer, result)) {
+ return result.isSuccess();
}
if (fieldDef.isCompFieldType()) {
- Set<Integer> tgtValueSet = new HashSet<Integer>();
- Set<String> valItemSet = (Set<String>) result.retData1;
+ Set<Integer> tgtValueSet = new HashSet<>();
+ Set<String> valItemSet = (Set<String>) result.getRetData();
if (valItemSet.isEmpty()) {
if (hasDefVal) {
tgtValueSet.add(defValue);
}
result.setSuccResult(tgtValueSet);
- return result.success;
+ return result.isSuccess();
}
for (String itemVal : valItemSet) {
if (!checkIntValueNorms(fieldDef, itemVal,
- hasMinVal, minValue, hasMaxVal, maxValue, result)) {
- return result.success;
+ hasMinVal, minValue, hasMaxVal, maxValue, sBuffer, result)) {
+ return result.isSuccess();
}
tgtValueSet.add((Integer) result.retData1);
}
result.setSuccResult(tgtValueSet);
} else {
- String paramValue = (String) result.retData1;
+ String paramValue = (String) result.getRetData();
if (paramValue == null) {
if (hasDefVal) {
result.setSuccResult(defValue);
}
- return result.success;
+ return result.isSuccess();
}
checkIntValueNorms(fieldDef, paramValue,
- hasMinVal, minValue, hasMaxVal, maxValue, result);
+ hasMinVal, minValue, hasMaxVal, maxValue, sBuffer, result);
}
- return result.success;
+ return result.isSuccess();
}
/**
* Parse the parameter value from an object value to a boolean value
*
- * @param req Http Servlet Request
+ * @param paramCntr parameter container object
* @param fieldDef the parameter field definition
* @param required a boolean value represent whether the parameter is must required
* @param defValue a default value returned if the field not exist
+ * @param sBuffer string buffer
* @param result process result
* @return valid result for the parameter value
*/
- public static boolean getBooleanParamValue(HttpServletRequest req,
- WebFieldDef fieldDef,
- boolean required,
- boolean defValue,
- ProcessResult result) {
- if (!getStringParamValue(req, fieldDef, required, null, result)) {
- return result.success;
- }
- String paramValue = (String) result.retData1;
+ public static <T> boolean getBooleanParamValue(T paramCntr, WebFieldDef fieldDef,
+ boolean required, Boolean defValue,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ if (!getStringParamValue(paramCntr, fieldDef,
+ required, null, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ String paramValue = (String) result.getRetData();
if (paramValue == null) {
result.setSuccResult(defValue);
- return result.success;
+ return result.isSuccess();
+ }
+ if (paramValue.equalsIgnoreCase("true")
+ || paramValue.equalsIgnoreCase("false")) {
+ result.setSuccResult(Boolean.parseBoolean(paramValue));
+ } else {
+ try {
+ result.setSuccResult(!(Long.parseLong(paramValue) == 0));
+ } catch (Throwable e) {
+ result.setSuccResult(defValue);
+ }
}
- result.setSuccResult(Boolean.parseBoolean(paramValue));
- return result.success;
+ return result.isSuccess();
}
/**
* Parse the parameter value from an object value
*
- * @param req Http Servlet Request
- * @param fieldDef the parameter field definition
+ * @param paramCntr parameter container object
+ * @param fieldDef the parameter field definition
* @param required a boolean value represent whether the parameter is must required
* @param defValue a default value returned if the field not exist
- * @param result process result
+ * @param sBuffer string buffer
+ * @param result process result
* @return valid result for the parameter value
*/
- public static boolean getStringParamValue(HttpServletRequest req,
- WebFieldDef fieldDef,
- boolean required,
- String defValue,
- ProcessResult result) {
+ public static <T> boolean getStringParamValue(T paramCntr, WebFieldDef fieldDef,
+ boolean required, String defValue,
+ StringBuilder sBuffer, ProcessResult result) {
+ String paramValue;
// get parameter value
- String paramValue = req.getParameter(fieldDef.name);
- if (paramValue == null) {
- paramValue = req.getParameter(fieldDef.shortName);
+ if (paramCntr instanceof Map) {
+ Map<String, String> keyValueMap =
+ (Map<String, String>) paramCntr;
+ paramValue = keyValueMap.get(fieldDef.name);
+ if (paramValue == null) {
+ paramValue = keyValueMap.get(fieldDef.shortName);
+ }
+ } else if (paramCntr instanceof HttpServletRequest) {
+ HttpServletRequest req = (HttpServletRequest) paramCntr;
+ paramValue = req.getParameter(fieldDef.name);
+ if (paramValue == null) {
+ paramValue = req.getParameter(fieldDef.shortName);
+ }
+ } else {
+ throw new IllegalArgumentException("Unknown parameter type!");
}
+ return checkStrParamValue(paramValue,
+ fieldDef, required, defValue, sBuffer, result);
+ }
+
+ /**
+ * Check the parameter value
+ *
+ * @param paramValue parameter value
+ * @param fieldDef the parameter field definition
+ * @param required a boolean value represent whether the parameter is must required
+ * @param defValue a default value returned if the field not exist
+ * @param sBuffer string buffer
+ * @param result process result
+ * @return valid result for the parameter value
+ */
+ private static boolean checkStrParamValue(String paramValue, WebFieldDef fieldDef,
+ boolean required, String defValue,
+ StringBuilder sBuffer, ProcessResult result) {
if (TStringUtils.isNotBlank(paramValue)) {
// Cleanup value extra characters
paramValue = escDoubleQuotes(paramValue.trim());
@@ -435,56 +725,188 @@ public class WebParameterUtils {
// Check if the parameter exists
if (TStringUtils.isBlank(paramValue)) {
if (required) {
- result.setFailResult(new StringBuilder(512)
- .append("Parameter ").append(fieldDef.name)
- .append(" is missing or value is null or blank!").toString());
+ result.setFailResult(sBuffer.append("Parameter ").append(fieldDef.name)
+ .append(" is missing or value is blank!").toString());
+ sBuffer.delete(0, sBuffer.length());
} else {
procStringDefValue(fieldDef.isCompFieldType(), defValue, result);
}
- return result.success;
+ return result.isSuccess();
}
// check if value is norm;
if (fieldDef.isCompFieldType()) {
// split original value to items
- Set<String> valItemSet = new HashSet<>();
+ TreeSet<String> valItemSet = new TreeSet<>();
String[] strParamValueItems = paramValue.split(fieldDef.splitToken);
for (String strParamValueItem : strParamValueItems) {
if (TStringUtils.isBlank(strParamValueItem)) {
continue;
}
- if (!checkStrValueNorms(fieldDef, strParamValueItem, result)) {
- return result.success;
+ if (!checkStrValueNorms(fieldDef, strParamValueItem, sBuffer, result)) {
+ return result.isSuccess();
}
valItemSet.add((String) result.retData1);
}
// check if is empty result
if (valItemSet.isEmpty()) {
if (required) {
- result.setFailResult(new StringBuilder(512)
- .append("Parameter ").append(fieldDef.name)
- .append(" is missing or value is null or blank!").toString());
+ result.setFailResult(sBuffer.append("Parameter ").append(fieldDef.name)
+ .append(" is missing or value is blank!").toString());
+ sBuffer.delete(0, sBuffer.length());
} else {
procStringDefValue(fieldDef.isCompFieldType(), defValue, result);
}
- return result.success;
+ return result.isSuccess();
}
// check max item count
if (fieldDef.itemMaxCnt != TBaseConstants.META_VALUE_UNDEFINED) {
if (valItemSet.size() > fieldDef.itemMaxCnt) {
- result.setFailResult(new StringBuilder(512)
- .append("Parameter ").append(fieldDef.name)
+ result.setFailResult(sBuffer.append("Parameter ").append(fieldDef.name)
.append("'s item count over max allowed count (")
.append(fieldDef.itemMaxCnt).append(")!").toString());
+ sBuffer.delete(0, sBuffer.length());
}
}
+ valItemSet.comparator();
result.setSuccResult(valItemSet);
} else {
- if (!checkStrValueNorms(fieldDef, paramValue, result)) {
- return result.success;
+ if (!checkStrValueNorms(fieldDef, paramValue, sBuffer, result)) {
+ return result.isSuccess();
}
result.setSuccResult(paramValue);
}
- return result.success;
+ return result.isSuccess();
+ }
+
+ /**
+ * Get and valid topicName value
+ *
+ * @param req Http Servlet Request
+ * @param confManager configure manager
+ * @param required a boolean value represent whether the parameter is must required
+ * @param defValue a default value returned if the field not exist
+ * @param sBuffer string buffer
+ * @param result process result of parameter value
+ * @return process result
+ */
+ public static boolean getAndValidTopicNameInfo(HttpServletRequest req,
+ MetaDataManager confManager,
+ boolean required,
+ String defValue,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ if (!WebParameterUtils.getStringParamValue(req,
+ WebFieldDef.COMPSTOPICNAME, required, defValue, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ Set<String> topicNameSet = (Set<String>) result.getRetData();
+ Set<String> existedTopicSet =
+ confManager.getTotalConfiguredTopicNames();
+ for (String topic : topicNameSet) {
+ if (!existedTopicSet.contains(topic)) {
+ result.setFailResult(sBuffer.append(WebFieldDef.COMPSTOPICNAME.name)
+ .append(" value ").append(topic)
+ .append(" is not configure, please configure first!").toString());
+ sBuffer.delete(0, sBuffer.length());
+ break;
+ }
+ }
+ return result.isSuccess();
+ }
+
+ /**
+ * check the filter conditions and get them in a String
+ *
+ * @param paramCntr parameter container object
+ * @param required denote whether it translate blank condition
+ * @param transBlank whether to translate condition item
+ * @param sBuffer string buffer
+ * @param result process result of parameter value
+ * @return process result
+ */
+ public static <T> boolean getFilterCondString(T paramCntr, boolean required,
+ boolean transBlank,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ if (!getFilterCondSet(paramCntr, required, false, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ Set<String> filterCondSet = (Set<String>) result.getRetData();
+ if (filterCondSet.isEmpty()) {
+ if (transBlank) {
+ sBuffer.append(TServerConstants.BLANK_FILTER_ITEM_STR);
+ }
+ } else {
+ sBuffer.append(TokenConstants.ARRAY_SEP);
+ for (String filterCond : filterCondSet) {
+ sBuffer.append(filterCond).append(TokenConstants.ARRAY_SEP);
+ }
+ }
+ result.setSuccResult(sBuffer.toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
+ }
+
+ /**
+ * check the filter conditions and get them in a set
+ *
+ * @param paramCntr parameter container object
+ * @param required a boolean value represent whether the parameter is must required
+ * @param transCondItem whether to translate condition item
+ * @param sBuffer string buffer
+ * @param result process result of parameter value
+ * @return process result
+ */
+ public static <T> boolean getFilterCondSet(T paramCntr, boolean required,
+ boolean transCondItem,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ if (!WebParameterUtils.getStringParamValue(paramCntr,
+ WebFieldDef.FILTERCONDS, required, null, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ if (transCondItem) {
+ // translate filter condition item with "''"
+ TreeSet<String> newFilterCondSet = new TreeSet<>();
+ Set<String> filterCondSet = (Set<String>) result.getRetData();
+ if (!filterCondSet.isEmpty()) {
+ for (String filterCond : filterCondSet) {
+ newFilterCondSet.add(sBuffer.append(TokenConstants.ARRAY_SEP)
+ .append(filterCond).append(TokenConstants.ARRAY_SEP).toString());
+ sBuffer.delete(0, sBuffer.length());
+ }
+ newFilterCondSet.comparator();
+ }
+ result.setSuccResult(newFilterCondSet);
+ }
+ return result.isSuccess();
+ }
+
+ /**
+ * Judge whether the query filter item set is completely contained by the target item set
+ *
+ * @param qryFilterSet the query filter item set
+ * @param confFilterStr the target item set
+ * @return true all item are included, false not
+ */
+ public static boolean isFilterSetFullIncluded(
+ Set<String> qryFilterSet, String confFilterStr) {
+ if (qryFilterSet == null || qryFilterSet.isEmpty()) {
+ return true;
+ }
+ if (confFilterStr == null
+ || (confFilterStr.length() == 2
+ && confFilterStr.equals(TServerConstants.BLANK_FILTER_ITEM_STR))) {
+ return false;
+ }
+ boolean allInc = true;
+ for (String filterCond : qryFilterSet) {
+ if (!confFilterStr.contains(filterCond)) {
+ allInc = false;
+ break;
+ }
+ }
+ return allInc;
}
/**
@@ -516,11 +938,11 @@ public class WebParameterUtils {
if (required) {
result.setFailResult(new StringBuilder(512)
.append("Parameter ").append(fieldDef.name)
- .append(" is missing or value is null or blank!").toString());
+ .append(" is missing or value is blank!").toString());
} else {
result.setSuccResult(defValue);
}
- return result.success;
+ return result.isSuccess();
}
try {
paramValue = URLDecoder.decode(paramValue,
@@ -538,7 +960,7 @@ public class WebParameterUtils {
} else {
result.setSuccResult(defValue);
}
- return result.success;
+ return result.isSuccess();
}
if (fieldDef.valMaxLen != TBaseConstants.META_VALUE_UNDEFINED) {
if (paramValue.length() > fieldDef.valMaxLen) {
@@ -546,7 +968,7 @@ public class WebParameterUtils {
.append("Parameter ").append(fieldDef.name)
.append("'s length over max allowed length (")
.append(fieldDef.valMaxLen).append(")!").toString());
- return result.success;
+ return result.isSuccess();
}
}
// parse data
@@ -560,68 +982,146 @@ public class WebParameterUtils {
.append(" value parse failure, error is ")
.append(e.getMessage()).append("!").toString());
}
- return result.success;
+ return result.isSuccess();
}
/**
- * Parse the parameter value from an string value to Date value
+ * Parse the parameter value from an json array
*
* @param req Http Servlet Request
* @param fieldDef the parameter field definition
* @param required a boolean value represent whether the parameter is must required
+ * @param defValue a default value returned if the field not exist
+ * @param result process result
+ * @return valid result for the parameter value
+ */
+ public static boolean getJsonArrayParamValue(HttpServletRequest req,
+ WebFieldDef fieldDef,
+ boolean required,
+ List<Map<String, String>> defValue,
+ ProcessResult result) {
+ // get parameter value
+ String paramValue = req.getParameter(fieldDef.name);
+ if (paramValue == null) {
+ paramValue = req.getParameter(fieldDef.shortName);
+ }
+ if (TStringUtils.isNotBlank(paramValue)) {
+ // Cleanup value extra characters
+ paramValue = escDoubleQuotes(paramValue.trim());
+ }
+ // Check if the parameter exists
+ if (TStringUtils.isBlank(paramValue)) {
+ if (required) {
+ result.setFailResult(new StringBuilder(512)
+ .append("Parameter ").append(fieldDef.name)
+ .append(" is missing or value is blank!").toString());
+ } else {
+ result.setSuccResult(defValue);
+ }
+ return result.isSuccess();
+ }
+ try {
+ paramValue = URLDecoder.decode(paramValue,
+ TBaseConstants.META_DEFAULT_CHARSET_NAME);
+ } catch (UnsupportedEncodingException e) {
+ result.setFailResult(new StringBuilder(512)
+ .append("Parameter ").append(fieldDef.name)
+ .append(" decode error, exception is ")
+ .append(e.toString()).toString());
+ }
+ if (TStringUtils.isBlank(paramValue)) {
+ if (required) {
+ result.setFailResult(new StringBuilder(512).append("Parameter ")
+ .append(fieldDef.name).append("'s value is blank!").toString());
+ } else {
+ result.setSuccResult(defValue);
+ }
+ return result.isSuccess();
+ }
+ if (fieldDef.valMaxLen != TBaseConstants.META_VALUE_UNDEFINED) {
+ if (paramValue.length() > fieldDef.valMaxLen) {
+ result.setFailResult(new StringBuilder(512)
+ .append("Parameter ").append(fieldDef.name)
+ .append("'s length over max allowed length (")
+ .append(fieldDef.valMaxLen).append(")!").toString());
+ return result.isSuccess();
+ }
+ }
+ // parse data
+ try {
+ List<Map<String, String>> arrayValue = new Gson().fromJson(paramValue,
+ new TypeToken<List<Map<String, String>>>(){}.getType());
+ result.setSuccResult(arrayValue);
+ } catch (Throwable e) {
+ result.setFailResult(new StringBuilder(512)
+ .append("Parameter ").append(fieldDef.name)
+ .append(" value parse failure, error is ")
+ .append(e.getMessage()).append("!").toString());
+ }
+ return result.isSuccess();
+ }
+
+ /**
+ * Parse the parameter value from an string value to Date value
+ *
+ * @param paramCntr parameter container object
+ * @param fieldDef the parameter field definition
+ * @param required a boolean value represent whether the parameter is must required
* @param defValue a default value returned if failed to parse value from the given object
+ * @param sBuffer string buffer
* @param result process result
* @return valid result for the parameter value
*/
- public static boolean getDateParameter(HttpServletRequest req,
- WebFieldDef fieldDef,
- boolean required,
- Date defValue,
- ProcessResult result) {
- if (!getStringParamValue(req, fieldDef, required, null, result)) {
- return result.success;
- }
- String paramValue = (String) result.retData1;
+ public static <T> boolean getDateParameter(T paramCntr, WebFieldDef fieldDef,
+ boolean required, Date defValue,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ if (!getStringParamValue(paramCntr, fieldDef,
+ required, null, sBuffer, result)) {
+ return result.isSuccess();
+ }
+ String paramValue = (String) result.getRetData();
if (paramValue == null) {
result.setSuccResult(defValue);
- return result.success;
+ return result.isSuccess();
}
try {
DateFormat sdf = new SimpleDateFormat(TBaseConstants.META_TMP_DATE_VALUE);
Date date = sdf.parse(paramValue);
result.setSuccResult(date);
} catch (Throwable e) {
- result.setFailResult(new StringBuilder(512)
- .append("Parameter ").append(fieldDef.name)
- .append(" parse error: ").append(e.getMessage()).toString());
+ result.setFailResult(sBuffer.append("Parameter ").append(fieldDef.name)
+ .append("'s value ").append(paramValue)
+ .append(" parse error, required value format is ")
+ .append(TBaseConstants.META_TMP_DATE_VALUE).toString());
+ sBuffer.delete(0, sBuffer.length());
}
- return result.success;
+ return result.isSuccess();
}
/**
* Valid execution authorization info
* @param req Http Servlet Request
- * @param fieldDef the parameter field definition
* @param required a boolean value represent whether the parameter is must required
* @param master current master object
+ * @param sBuffer string buffer
* @param result process result
* @return valid result for the parameter value
*/
- public static boolean validReqAuthorizeInfo(HttpServletRequest req,
- WebFieldDef fieldDef,
- boolean required,
- TMaster master,
+ public static boolean validReqAuthorizeInfo(HttpServletRequest req, boolean required,
+ TMaster master, StringBuilder sBuffer,
ProcessResult result) {
- if (!getStringParamValue(req, fieldDef, required, null, result)) {
- return result.success;
+ if (!getStringParamValue(req, WebFieldDef.ADMINAUTHTOKEN,
+ required, null, sBuffer, result)) {
+ return result.isSuccess();
}
- String paramValue = (String) result.retData1;
+ String paramValue = (String) result.getRetData();
if (paramValue != null) {
if (!paramValue.equals(master.getMasterConfig().getConfModAuthToken())) {
- result.setFailResult("Illegal access, unauthorized request!");
+ result.setFailResult("illegal access, unauthorized request!");
}
}
- return result.success;
+ return result.isSuccess();
}
/**
@@ -636,15 +1136,16 @@ public class WebParameterUtils {
String defValue,
ProcessResult result) {
if (isCompFieldType) {
- Set<String> valItemSet = new HashSet<>();
+ TreeSet<String> valItemSet = new TreeSet<>();
if (TStringUtils.isNotBlank(defValue)) {
valItemSet.add(defValue);
}
+ valItemSet.comparator();
result.setSuccResult(valItemSet);
} else {
result.setSuccResult(defValue);
}
- return result.success;
+ return result.isSuccess();
}
/**
@@ -652,12 +1153,12 @@ public class WebParameterUtils {
*
* @param fieldDef the parameter field definition
* @param paramVal the parameter value
+ * @param sBuffer string buffer
* @param result process result
* @return check result for string value of parameter
*/
- private static boolean checkStrValueNorms(WebFieldDef fieldDef,
- String paramVal,
- ProcessResult result) {
+ private static boolean checkStrValueNorms(WebFieldDef fieldDef, String paramVal,
+ StringBuilder sBuffer, ProcessResult result) {
paramVal = paramVal.trim();
if (TStringUtils.isBlank(paramVal)) {
result.setSuccResult(null);
@@ -666,20 +1167,20 @@ public class WebParameterUtils {
// check value's max length
if (fieldDef.valMaxLen != TBaseConstants.META_VALUE_UNDEFINED) {
if (paramVal.length() > fieldDef.valMaxLen) {
- result.setFailResult(new StringBuilder(512)
- .append("over max length for ").append(fieldDef.name)
- .append(", only allow ").append(fieldDef.valMaxLen)
- .append(" length").toString());
+ result.setFailResult(sBuffer.append("over max length for ")
+ .append(fieldDef.name).append(", only allow ")
+ .append(fieldDef.valMaxLen).append(" length").toString());
+ sBuffer.delete(0, sBuffer.length());
return false;
}
}
// check value's pattern
if (fieldDef.regexCheck) {
if (!paramVal.matches(fieldDef.regexDef.getPattern())) {
- result.setFailResult(new StringBuilder(512)
- .append("illegal value for ").append(fieldDef.name)
- .append(", value ").append(fieldDef.regexDef.getErrMsgTemp())
- .toString());
+ result.setFailResult(sBuffer.append("illegal value for ")
+ .append(fieldDef.name).append(", value ")
+ .append(fieldDef.regexDef.getErrMsgTemp()).toString());
+ sBuffer.delete(0, sBuffer.length());
return false;
}
}
@@ -693,222 +1194,42 @@ public class WebParameterUtils {
* @param fieldDef the parameter field definition
* @param paramValue the parameter value
* @param hasMinVal whether there is a minimum
- * @param minValue the parameter min value
+ * param minValue the parameter min value
* @param hasMaxVal whether there is a maximum
- * @param maxValue the parameter max value
+ * param maxValue the parameter max value
+ * @param sBuffer string buffer
* @param result process result
* @return check result for string value of parameter
*/
- private static boolean checkIntValueNorms(WebFieldDef fieldDef,
- String paramValue,
- boolean hasMinVal,
- int minValue,
- boolean hasMaxVal,
- int maxValue,
- ProcessResult result) {
+ private static boolean checkIntValueNorms(WebFieldDef fieldDef, String paramValue,
+ boolean hasMinVal, int minValue,
+ boolean hasMaxVal, int maxValue,
+ StringBuilder sBuffer, ProcessResult result) {
try {
int paramIntVal = Integer.parseInt(paramValue);
if (hasMinVal && paramIntVal < minValue) {
- result.setFailResult(new StringBuilder(512)
- .append("Parameter ").append(fieldDef.name)
+ result.setFailResult(sBuffer.append("Parameter ").append(fieldDef.name)
.append(" value must >= ").append(minValue).toString());
+ sBuffer.delete(0, sBuffer.length());
return false;
}
if (hasMaxVal && paramIntVal > maxValue) {
- result.setFailResult(new StringBuilder(512)
- .append("Parameter ").append(fieldDef.name)
+ result.setFailResult(sBuffer.append("Parameter ").append(fieldDef.name)
.append(" value must <= ").append(maxValue).toString());
+ sBuffer.delete(0, sBuffer.length());
return false;
}
result.setSuccResult(paramIntVal);
} catch (Throwable e) {
- result.setFailResult(new StringBuilder(512)
- .append("Parameter ").append(fieldDef.name)
+ result.setFailResult(sBuffer.append("Parameter ").append(fieldDef.name)
.append(" parse error: ").append(e.getMessage()).toString());
+ sBuffer.delete(0, sBuffer.length());
return false;
}
return true;
}
/**
- * Parse the parameter value from an object value to ip address of string value
- *
- * @param paramName the parameter name
- * @param paramValue the parameter value which is an object for parsing
- * @param paramMaxLen the max length of string to return
- * @param required a boolean value represent whether the parameter is must required
- * @param defaultValue a default value returned if failed to parse value from the given object
- * @return a ip string of parameter
- * @throws Exception if failed to parse the object
- */
- public static String validAddressParameter(String paramName, String paramValue, int paramMaxLen,
- boolean required, String defaultValue) throws Exception {
- String tmpParamValue = checkParamCommonRequires(paramName, paramValue, required);
- if (TStringUtils.isBlank(tmpParamValue)) {
- return defaultValue;
- }
- if (tmpParamValue.length() > paramMaxLen) {
- throw new Exception(new StringBuilder(512).append("the max length of ")
- .append(paramName).append(" parameter is ").append(paramMaxLen)
- .append(" characters").toString());
- }
- if (!tmpParamValue.matches(TBaseConstants.META_TMP_IP_ADDRESS_VALUE)) {
- throw new Exception(new StringBuilder(512)
- .append("the value of ").append(paramName)
- .append(" parameter not matches the regulation :")
- .append(TBaseConstants.META_TMP_IP_ADDRESS_VALUE).toString());
- }
- return tmpParamValue;
- }
-
- /**
- * Decode the parameter value from an object value
- *
- * @param paramName the parameter name
- * @param paramValue the parameter value which is an object for parsing
- * @param paramMaxLen the max length of string to return
- * @param required a boolean value represent whether the parameter is must required
- * @param defaultValue a default value returned if failed to parse value from the given object
- * @return the decoded string of parameter
- * @throws Exception if failed to parse the object
- */
- public static String validDecodeStringParameter(String paramName, String paramValue, int paramMaxLen,
- boolean required, String defaultValue) throws Exception {
- String tmpParamValue = checkParamCommonRequires(paramName, paramValue, required);
- if (TStringUtils.isBlank(tmpParamValue)) {
- return defaultValue;
- }
- String output = null;
- try {
- output = URLDecoder.decode(tmpParamValue, TBaseConstants.META_DEFAULT_CHARSET_NAME);
- } catch (UnsupportedEncodingException e) {
- throw new Exception(new StringBuilder(512).append("Decode ")
- .append(paramName).append("error, exception is ")
- .append(e.toString()).toString());
- }
- if (paramMaxLen != TBaseConstants.META_VALUE_UNDEFINED) {
- if (output.length() > paramMaxLen) {
- throw new Exception(new StringBuilder(512)
- .append("the max length of ").append(paramName)
- .append(" parameter is ").append(paramMaxLen)
- .append(" characters").toString());
- }
- }
- return output;
- }
-
- /**
- * Proceed authorization
- *
- * @param master the service master
- * @param brokerConfManager the broker configuration manager
- * @param reqToken the token for checking
- * @throws Exception if authorization failed
- */
- public static void reqAuthorizeCheck(TMaster master,
- BrokerConfManager brokerConfManager,
- String reqToken) throws Exception {
- if (brokerConfManager.isPrimaryNodeActive()) {
- throw new Exception(
- "Illegal visit: designatedPrimary happened...please check if the other member is down");
- }
- String inPutConfModAuthToken =
- validStringParameter("confModAuthToken", reqToken,
- TServerConstants.CFG_MODAUTHTOKEN_MAX_LENGTH, true, "");
- if (!inPutConfModAuthToken.equals(master.getMasterConfig().getConfModAuthToken())) {
- throw new Exception("Illegal visit: not authorized to process authorization info!");
- }
- }
-
- /**
- * Decode the deletePolicy parameter value from an object value
- * the value must like {method},{digital}[s|m|h]
- *
- * @param paramName the parameter name
- * @param paramValue the parameter value which is an object for parsing
- * @param required a boolean value represent whether the parameter is must required
- * @param defaultValue a default value returned if failed to parse value from the given object
- * @return the decoded string of parameter
- * @throws Exception if failed to parse the object
- */
- public static String validDeletePolicyParameter(String paramName, String paramValue,
- boolean required, String defaultValue) throws Exception {
- int paramMaxLen = TServerConstants.CFG_DELETEPOLICY_MAX_LENGTH;
- String tmpParamValue = checkParamCommonRequires(paramName, paramValue, required);
- if (TStringUtils.isBlank(tmpParamValue)) {
- return defaultValue;
- }
- String inDelPolicy = null;
- try {
- inDelPolicy = URLDecoder.decode(tmpParamValue, TBaseConstants.META_DEFAULT_CHARSET_NAME);
- } catch (UnsupportedEncodingException e) {
- throw new Exception(new StringBuilder(512).append("Decode ")
- .append(paramName).append("error, exception is ")
- .append(e.toString()).toString());
- }
- if (inDelPolicy.length() > paramMaxLen) {
- throw new Exception(new StringBuilder(512)
- .append("the max length of ").append(paramName)
- .append(" parameter is ").append(paramMaxLen)
- .append(" characters").toString());
- }
- String[] tmpStrs = inDelPolicy.split(",");
- if (tmpStrs.length != 2) {
- throw new Exception(new StringBuilder(512)
- .append("Illegal value: must include one and only one comma character,")
- .append(" the format of ").append(paramName)
- .append(" must like {method},{digital}[m|s|h]").toString());
- }
- if (TStringUtils.isBlank(tmpStrs[0])) {
- throw new Exception(new StringBuilder(512)
- .append("Illegal value: method's value must not be blank!")
- .append(" the format of ").append(paramName)
- .append(" must like {method},{digital}[s|m|h]").toString());
- }
- if (!"delete".equalsIgnoreCase(tmpStrs[0].trim())) {
- throw new Exception(new StringBuilder(512)
- .append("Illegal value: only support delete method now!").toString());
- }
- String validValStr = tmpStrs[1];
- String timeUnit = validValStr.substring(validValStr.length() - 1).toLowerCase();
- if (Character.isLetter(timeUnit.charAt(0))) {
- if (!allowedDelUnits.contains(timeUnit)) {
- throw new Exception(new StringBuilder(512)
- .append("Illegal value: only support [s|m|h] unit!").toString());
- }
- }
- long validDuration = 0;
- try {
- if (timeUnit.endsWith("s")) {
- validDuration = Long.parseLong(validValStr.substring(0, validValStr.length() - 1)) * 1000;
- } else if (timeUnit.endsWith("m")) {
- validDuration = Long.parseLong(validValStr.substring(0, validValStr.length() - 1)) * 60000;
- } else if (timeUnit.endsWith("h")) {
- validDuration = Long.parseLong(validValStr.substring(0, validValStr.length() - 1)) * 3600000;
- } else {
- validDuration = Long.parseLong(validValStr) * 3600000;
- }
- } catch (Throwable e) {
- throw new Exception(new StringBuilder(512)
- .append("Illegal value: the value of valid duration must digits!").toString());
- }
- if (validDuration <= 0 || validDuration > DataStoreUtils.MAX_FILE_VALID_DURATION) {
- throw new Exception(new StringBuilder(512)
- .append("Illegal value: the value of valid duration must")
- .append(" be greater than 0 and less than or equal to ")
- .append(DataStoreUtils.MAX_FILE_VALID_DURATION).append(" seconds!").toString());
- }
- if (Character.isLetter(timeUnit.charAt(0))) {
- return new StringBuilder(512).append("delete,")
- .append(validValStr.substring(0, validValStr.length() - 1))
- .append(timeUnit).toString();
- } else {
- return new StringBuilder(512).append("delete,")
- .append(validValStr).append("h").toString();
- }
- }
-
- /**
* check the filter conditions and get them
*
* @param inFilterConds the filter conditions to be decoded
@@ -925,7 +1246,7 @@ public class WebParameterUtils {
}
if (TStringUtils.isBlank(inFilterConds)) {
if (isTransBlank) {
- sb.append(TServerConstants.TOKEN_BLANK_FILTER_CONDITION);
+ sb.append(TServerConstants.BLANK_FILTER_ITEM_STR);
}
} else {
sb.append(TokenConstants.ARRAY_SEP);
@@ -1029,6 +1350,7 @@ public class WebParameterUtils {
* @param inputGroupName the group name string value
* @param checkEmpty whether check data emtpy
* @param checkResToken whether check reserved group token
+ * @param resTokens reserved group name set
* @param sb the string process space
* @return the batched group names
*/
@@ -1188,99 +1510,6 @@ public class WebParameterUtils {
}
/**
- * check and get batched broker ids
- *
- * @param inStrBrokerIds the brokerId string value
- * @param checkEmpty whether check data emtpy
- * @return the batched broker ids
- */
- public static Set<Integer> getBatchBrokerIdSet(String inStrBrokerIds,
- boolean checkEmpty) throws Exception {
- Set<Integer> batchBrokerIdSet = new HashSet<>();
- if (TStringUtils.isNotBlank(inStrBrokerIds)) {
- inStrBrokerIds = escDoubleQuotes(inStrBrokerIds.trim());
- }
- if (TStringUtils.isBlank(inStrBrokerIds)) {
- if (checkEmpty) {
- throw new Exception("Illegal value: required brokerId parameter");
- }
- return batchBrokerIdSet;
- }
- String[] strBrokerIds = inStrBrokerIds.split(TokenConstants.ARRAY_SEP);
- if (strBrokerIds.length > TServerConstants.CFG_BATCH_BROKER_OPERATE_MAX_COUNT) {
- throw new Exception(new StringBuilder(512)
- .append("Illegal value: batch numbers of brokerId's value over max count ")
- .append(TServerConstants.CFG_BATCH_BROKER_OPERATE_MAX_COUNT).toString());
- }
- for (int i = 0; i < strBrokerIds.length; i++) {
- if (TStringUtils.isEmpty(strBrokerIds[i])) {
- continue;
- }
- int brokerId =
- validIntDataParameter("brokerId", strBrokerIds[i], true, 0, 1);
- batchBrokerIdSet.add(brokerId);
- }
- if (batchBrokerIdSet.isEmpty()) {
- if (checkEmpty) {
- throw new Exception("Illegal value: Null value of brokerId parameter");
- }
- }
- return batchBrokerIdSet;
- }
-
- /**
- * check and get batched broker ids
- *
- * @param inStrBrokerIds the brokerId string value
- * @param webMaster the broker configure manager
- * @param checkEmpty whether check data emtpy
- * @param sb the string process space
- * @return the batched broker ids
- */
- public static Set<BdbBrokerConfEntity> getBatchBrokerIdSet(String inStrBrokerIds,
- BrokerConfManager webMaster,
- boolean checkEmpty,
- final StringBuilder sb) throws Exception {
- Set<BdbBrokerConfEntity> batchBrokerIdSet = new HashSet<>();
- if (TStringUtils.isNotBlank(inStrBrokerIds)) {
- inStrBrokerIds = escDoubleQuotes(inStrBrokerIds.trim());
- }
- if (TStringUtils.isBlank(inStrBrokerIds)) {
- if (checkEmpty) {
- throw new Exception("Illegal value: required brokerId parameter");
- }
- return batchBrokerIdSet;
- }
- String[] strBrokerIds = inStrBrokerIds.split(TokenConstants.ARRAY_SEP);
- if (strBrokerIds.length > TServerConstants.CFG_BATCH_BROKER_OPERATE_MAX_COUNT) {
- throw new Exception(sb
- .append("Illegal value: batch numbers of brokerId's value over max count ")
- .append(TServerConstants.CFG_BATCH_BROKER_OPERATE_MAX_COUNT).toString());
- }
- for (int i = 0; i < strBrokerIds.length; i++) {
- if (TStringUtils.isEmpty(strBrokerIds[i])) {
- continue;
- }
- int brokerId =
- validIntDataParameter("brokerId", strBrokerIds[i], true, 0, 1);
- BdbBrokerConfEntity brokerConfEntity =
- webMaster.getBrokerDefaultConfigStoreInfo(brokerId);
- if (brokerConfEntity == null) {
- throw new Exception(sb
- .append("Illegal value: not found broker default configure record by brokerId=")
- .append(brokerId).toString());
- }
- batchBrokerIdSet.add(brokerConfEntity);
- }
- if (batchBrokerIdSet.isEmpty()) {
- if (checkEmpty) {
- throw new Exception("Illegal value: Null value of brokerId parameter");
- }
- }
- return batchBrokerIdSet;
- }
-
- /**
* check and get parameter value with json array
*
* @param paramName the parameter name
@@ -1342,94 +1571,6 @@ public class WebParameterUtils {
return false;
}
- /**
- * check whether the broker is working in progress
- *
- * @param brokerId the id of the broker
- * @param webMaster the broker configuration manager
- * @param sBuilder the string builder used to construct the detail err
- * @return true if the broker is working in progress, false in other cases
- * @throws Exception
- */
- public static boolean checkBrokerInProcessing(int brokerId,
- BrokerConfManager webMaster,
- StringBuilder sBuilder) throws Exception {
- BrokerSyncStatusInfo brokerSyncStatusInfo =
- webMaster.getBrokerRunSyncStatusInfo(brokerId);
- if ((brokerSyncStatusInfo != null) && (brokerSyncStatusInfo.isBrokerRegister())) {
- int status = brokerSyncStatusInfo.getBrokerRunStatus();
- if (!((status == TStatusConstants.STATUS_SERVICE_UNDEFINED)
- || (status == TStatusConstants.STATUS_SERVICE_TOONLINE_WAIT_REGISTER)
- || (status == TStatusConstants.STATUS_SERVICE_TOONLINE_PART_WAIT_REGISTER))) {
- if (sBuilder != null) {
- sBuilder.append("Illegal value: the broker of brokerId=")
- .append(brokerId).append(" is processing event(")
- .append(brokerSyncStatusInfo.getBrokerRunStatus())
- .append("), please try later! ");
- }
- return true;
- }
- }
- return false;
- }
-
- /**
- * Check whether the broker can perform unload operations
- *
- * @param brokerId the broker id
- * @param webMaster the object value of the parameter
- * @param sBuilder the string process space
- * @return whether the broker can perform unload operations
- */
- public static boolean checkBrokerUnLoad(int brokerId,
- BrokerConfManager webMaster,
- StringBuilder sBuilder) {
- BrokerSyncStatusInfo brokerSyncStatusInfo =
- webMaster.getBrokerRunSyncStatusInfo(brokerId);
- if ((brokerSyncStatusInfo != null) && (brokerSyncStatusInfo.isBrokerRegister())) {
- int status = brokerSyncStatusInfo.getBrokerManageStatus();
- if ((status == TStatusConstants.STATUS_MANAGE_ONLINE)
- || (status == TStatusConstants.STATUS_MANAGE_ONLINE_NOT_WRITE)
- || (status == TStatusConstants.STATUS_MANAGE_ONLINE_NOT_READ)) {
- if (!brokerSyncStatusInfo.isBrokerLoaded()) {
- if (sBuilder != null) {
- sBuilder.append("The broker's configure of brokerId=").append(brokerId)
- .append(" changed but not reload in online status, please reload configure first!");
- }
- return true;
- }
- }
- }
- return false;
- }
-
- /**
- * Check whether the broker in offline status
- *
- * @param brokerId the broker id
- * @param webMaster the object value of the parameter
- * @param sBuilder the string process space
- * @return whether the broker in offline status
- */
- public static boolean checkBrokerInOfflining(int brokerId,
- int manageStatus,
- BrokerConfManager webMaster,
- StringBuilder sBuilder) {
- BrokerSyncStatusInfo brokerSyncStatusInfo =
- webMaster.getBrokerRunSyncStatusInfo(brokerId);
- if ((brokerSyncStatusInfo != null)
- && (brokerSyncStatusInfo.isBrokerRegister())) {
- if ((manageStatus == TStatusConstants.STATUS_MANAGE_OFFLINE)
- && (brokerSyncStatusInfo.getBrokerRunStatus() != TStatusConstants.STATUS_SERVICE_UNDEFINED)) {
- if (sBuilder != null) {
- sBuilder.append("Illegal value: the broker is processing offline event by brokerId=")
- .append(brokerId).append(", please wait and try later!");
- }
- return true;
- }
- }
- return false;
- }
/**
* translate broker manage status from int to string value
@@ -1484,11 +1625,31 @@ public class WebParameterUtils {
* @return the yyyyMMddHHmmss format string
*/
public static String date2yyyyMMddHHmmss(Date date) {
- SimpleDateFormat sdf = new SimpleDateFormat("yyyyMMddHHmmss");
+ SimpleDateFormat sdf =
+ new SimpleDateFormat(TBaseConstants.META_TMP_DATE_VALUE);
return sdf.format(date);
}
/**
+ * translate yyyyMMddHHmmss format string to Date value
+ *
+ * @param dateStr date string, format yyyyMMddHHmmss
+ * @return the Date value of string
+ */
+ public static Date yyyyMMddHHmmss2date(String dateStr) {
+ if (dateStr == null) {
+ return null;
+ }
+ SimpleDateFormat sdf =
+ new SimpleDateFormat(TBaseConstants.META_TMP_DATE_VALUE);
+ try {
+ return sdf.parse(dateStr);
+ } catch (Throwable e) {
+ return null;
+ }
+ }
+
+ /**
* check parameter is required
*
* @param paramName the parameter name
@@ -1517,6 +1678,46 @@ public class WebParameterUtils {
return temParamValue;
}
+ /**
+ * translate rule info to json format string
+ *
+ * @param paramCntr the parameter name
+ * @param defValue the default value
+ * @param sBuffer string buffer
+ * @param result process result
+ * @return the count of flow control rule
+ */
+ public static <T> int getAndCheckFlowRules(T paramCntr, String defValue,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ // get parameter value
+ String paramValue;
+ // get parameter value
+ if (paramCntr instanceof Map) {
+ Map<String, String> keyValueMap =
+ (Map<String, String>) paramCntr;
+ paramValue = keyValueMap.get(WebFieldDef.FLOWCTRLSET.name);
+ if (paramValue == null) {
+ paramValue = keyValueMap.get(WebFieldDef.FLOWCTRLSET.shortName);
+ }
+ } else if (paramCntr instanceof HttpServletRequest) {
+ HttpServletRequest req = (HttpServletRequest) paramCntr;
+ paramValue = req.getParameter(WebFieldDef.FLOWCTRLSET.name);
+ if (paramValue == null) {
+ paramValue = req.getParameter(WebFieldDef.FLOWCTRLSET.shortName);
+ }
+ } else {
+ throw new IllegalArgumentException("Unknown parameter type!");
+ }
+ if (TStringUtils.isBlank(paramValue)) {
+ result.setSuccResult(defValue);
+ return 0;
+ }
+ paramValue = paramValue.trim();
+ return validFlowRuleValue(paramValue, sBuffer, result);
+ }
+
+ // remove double quotes in string left and right
private static String escDoubleQuotes(String inPutStr) {
if (TStringUtils.isBlank(inPutStr) || inPutStr.length() < 2) {
return inPutStr;
@@ -1531,4 +1732,52 @@ public class WebParameterUtils {
}
return inPutStr;
}
+
+ // valid flow control rule informations
+ private static int validFlowRuleValue(String paramValue,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ int ruleCnt = 0;
+ paramValue = paramValue.trim();
+ List<Integer> ruleTypes = Arrays.asList(0, 1, 2, 3);
+ FlowCtrlRuleHandler flowCtrlRuleHandler =
+ new FlowCtrlRuleHandler(true);
+ Map<Integer, List<FlowCtrlItem>> flowCtrlItemMap;
+ try {
+ flowCtrlItemMap =
+ flowCtrlRuleHandler.parseFlowCtrlInfo(paramValue);
+ } catch (Throwable e) {
+ result.setFailResult(sBuffer.append("Parse parameter ")
+ .append(WebFieldDef.FLOWCTRLSET.name)
+ .append(" failure: '").append(e.toString()).toString());
+ sBuffer.delete(0, sBuffer.length());
+ return 0;
+ }
+ sBuffer.append("[");
+ for (Integer typeId : ruleTypes) {
+ if (typeId != null) {
+ int rules = 0;
+ List<FlowCtrlItem> flowCtrlItems = flowCtrlItemMap.get(typeId);
+ if (flowCtrlItems != null) {
+ if (ruleCnt++ > 0) {
+ sBuffer.append(",");
+ }
+ sBuffer.append("{\"type\":").append(typeId.intValue()).append(",\"rule\":[");
+ for (FlowCtrlItem flowCtrlItem : flowCtrlItems) {
+ if (flowCtrlItem != null) {
+ if (rules++ > 0) {
+ sBuffer.append(",");
+ }
+ sBuffer = flowCtrlItem.toJsonString(sBuffer);
+ }
+ }
+ sBuffer.append("]}");
+ }
+ }
+ }
+ sBuffer.append("]");
+ result.setSuccResult(sBuffer.toString());
+ sBuffer.delete(0, sBuffer.length());
+ return ruleCnt;
+ }
}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/TMaster.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/TMaster.java
index eecb459..4422486 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/TMaster.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/TMaster.java
@@ -46,7 +46,6 @@ import org.apache.inlong.tubemq.corebase.cluster.NodeAddrInfo;
import org.apache.inlong.tubemq.corebase.cluster.Partition;
import org.apache.inlong.tubemq.corebase.cluster.ProducerInfo;
import org.apache.inlong.tubemq.corebase.cluster.SubscribeInfo;
-import org.apache.inlong.tubemq.corebase.cluster.TopicInfo;
import org.apache.inlong.tubemq.corebase.config.TLSConfig;
import org.apache.inlong.tubemq.corebase.protobuf.generated.ClientMaster;
import org.apache.inlong.tubemq.corebase.protobuf.generated.ClientMaster.CloseRequestB2M;
@@ -82,8 +81,6 @@ import org.apache.inlong.tubemq.corerpc.RpcServiceFactory;
import org.apache.inlong.tubemq.corerpc.exception.StandbyException;
import org.apache.inlong.tubemq.corerpc.service.MasterService;
import org.apache.inlong.tubemq.server.Stoppable;
-import org.apache.inlong.tubemq.server.common.TServerConstants;
-import org.apache.inlong.tubemq.server.common.TStatusConstants;
import org.apache.inlong.tubemq.server.common.aaaserver.CertificateMasterHandler;
import org.apache.inlong.tubemq.server.common.aaaserver.CertifiedResult;
import org.apache.inlong.tubemq.server.common.aaaserver.SimpleCertificateMasterHandler;
@@ -96,18 +93,18 @@ import org.apache.inlong.tubemq.server.common.offsetstorage.ZkOffsetStorage;
import org.apache.inlong.tubemq.server.common.paramcheck.PBParameterUtils;
import org.apache.inlong.tubemq.server.common.paramcheck.ParamCheckResult;
import org.apache.inlong.tubemq.server.common.utils.HasThread;
+import org.apache.inlong.tubemq.server.common.utils.ProcessResult;
import org.apache.inlong.tubemq.server.common.utils.RowLock;
import org.apache.inlong.tubemq.server.common.utils.Sleeper;
import org.apache.inlong.tubemq.server.master.balance.DefaultLoadBalancer;
import org.apache.inlong.tubemq.server.master.balance.LoadBalancer;
-import org.apache.inlong.tubemq.server.master.bdbstore.DefaultBdbStoreService;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbBrokerConfEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbClusterSettingEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbGroupFlowCtrlEntity;
-import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerConfManager;
-import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerInfoHolder;
-import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerSyncStatusInfo;
-import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.TargetValidResult;
+import org.apache.inlong.tubemq.server.master.metamanage.MetaDataManager;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.BrokerConfEntity;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.ClusterSettingEntity;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.GroupResCtrlEntity;
+import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerAbnHolder;
+import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerRunManager;
+import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.DefBrokerRunManager;
import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.TopicPSInfoManager;
import org.apache.inlong.tubemq.server.master.nodemanage.nodeconsumer.ConsumerBandInfo;
import org.apache.inlong.tubemq.server.master.nodemanage.nodeconsumer.ConsumerEventManager;
@@ -129,10 +126,11 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
currentSubInfo = new ConcurrentHashMap<>();
private final RpcServiceFactory rpcServiceFactory = //rpc service factory
new RpcServiceFactory();
+ private final MetaDataManager defMetaDataManager; // meta data manager
+ private final BrokerRunManager brokerRunManager; // broker run status manager
private final ConsumerEventManager consumerEventManager; //consumer event manager
private final TopicPSInfoManager topicPSInfoManager; //topic publish/subscribe info manager
private final ExecutorService executor;
- private final BrokerInfoHolder brokerHolder; //broker holder
private final ProducerInfoHolder producerHolder; //producer holder
private final ConsumerInfoHolder consumerHolder; //consumer holder
private final RowLock masterRowLock; //lock
@@ -143,8 +141,6 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
private final HeartbeatManager heartbeatManager; //heartbeat manager
private final OffsetStorage zkOffsetStorage; //zookeeper offset manager
private final ShutdownHook shutdownHook; //shutdown hook
- private final DefaultBdbStoreService defaultBdbStoreService; //bdb store service
- private final BrokerConfManager defaultBrokerConfManager; //broker config manager
private final CertificateMasterHandler serverAuthHandler; //server auth handler
private AtomicBoolean shutdownHooked = new AtomicBoolean(false);
private AtomicLong idGenerator = new AtomicLong(0); //id generator
@@ -173,14 +169,14 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
this.executor = Executors.newFixedThreadPool(this.masterConfig.getRebalanceParallel());
this.visitTokenManager = new SimpleVisitTokenManager(this.masterConfig);
this.serverAuthHandler = new SimpleCertificateMasterHandler(this.masterConfig);
+ this.heartbeatManager = new HeartbeatManager();
+ this.zkOffsetStorage = new ZkOffsetStorage(this.masterConfig.getZkConfig(),
+ false, TBaseConstants.META_VALUE_UNDEFINED);
this.producerHolder = new ProducerInfoHolder();
this.consumerHolder = new ConsumerInfoHolder();
this.consumerEventManager = new ConsumerEventManager(consumerHolder);
this.topicPSInfoManager = new TopicPSInfoManager(this);
this.loadBalancer = new DefaultLoadBalancer();
- this.zkOffsetStorage = new ZkOffsetStorage(this.masterConfig.getZkConfig(),
- false, TBaseConstants.META_VALUE_UNDEFINED);
- this.heartbeatManager = new HeartbeatManager();
heartbeatManager.regConsumerCheckBusiness(masterConfig.getConsumerHeartbeatTimeoutMs(),
new TimeoutListener() {
@Override
@@ -199,22 +195,9 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
new ReleaseProducer().run(nodeId);
}
});
- heartbeatManager.regBrokerCheckBusiness(masterConfig.getBrokerHeartbeatTimeoutMs(),
- new TimeoutListener() {
- @Override
- public void onTimeout(final String nodeId, TimeoutInfo nodeInfo) throws Exception {
- logger.info(new StringBuilder(512).append("[Broker Timeout] ")
- .append(nodeId).toString());
- new ReleaseBroker().run(nodeId);
- }
- });
- this.defaultBdbStoreService = new DefaultBdbStoreService(masterConfig, this);
- this.defaultBdbStoreService.start();
- this.defaultBrokerConfManager = new BrokerConfManager(this.defaultBdbStoreService);
- this.defaultBrokerConfManager.start();
- this.brokerHolder =
- new BrokerInfoHolder(this.masterConfig.getMaxAutoForbiddenCnt(),
- this.defaultBrokerConfManager);
+ this.defMetaDataManager = new MetaDataManager(this);
+ this.brokerRunManager = new DefBrokerRunManager(this);
+ this.defMetaDataManager.start();
RpcConfig rpcTcpConfig = new RpcConfig();
rpcTcpConfig.put(RpcConstants.REQUEST_TIMEOUT,
masterConfig.getRpcReadTimeoutMs());
@@ -273,13 +256,17 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
return masterConfig;
}
- /**
- * Get master topic manage
- *
- * @return
- */
- public BrokerConfManager getMasterTopicManager() {
- return this.defaultBrokerConfManager;
+
+ public MetaDataManager getDefMetaDataManager() {
+ return defMetaDataManager;
+ }
+
+ public HeartbeatManager getHeartbeatManager() {
+ return heartbeatManager;
+ }
+
+ public BrokerRunManager getBrokerRunManager() {
+ return brokerRunManager;
}
/**
@@ -347,8 +334,10 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
heartbeatManager.regProducerNode(producerId);
producerHolder.setProducerInfo(producerId,
new HashSet<>(transTopicSet), hostName, overtls);
- builder.setBrokerCheckSum(this.defaultBrokerConfManager.getBrokerInfoCheckSum());
- builder.addAllBrokerInfos(this.defaultBrokerConfManager.getBrokersMap(overtls).values());
+ Tuple2<Long, Map<Integer, String>> brokerStaticInfo =
+ brokerRunManager.getBrokerStaticInfo(overtls);
+ builder.setBrokerCheckSum(brokerStaticInfo.getF0());
+ builder.addAllBrokerInfos(brokerStaticInfo.getF1().values());
builder.setAuthorizedInfo(genAuthorizedInfo(certResult.authorizedToken, false).build());
ClientMaster.ApprovedClientConfig.Builder clientConfigBuilder =
buildApprovedClientConfig(request.getAppdConfig());
@@ -438,10 +427,12 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
transTopicSet, hostName, overtls);
Map<String, String> availTopicPartitions = getProducerTopicPartitionInfo(producerId);
builder.addAllTopicInfos(availTopicPartitions.values());
- builder.setBrokerCheckSum(defaultBrokerConfManager.getBrokerInfoCheckSum());
builder.setAuthorizedInfo(genAuthorizedInfo(certResult.authorizedToken, false).build());
- if (defaultBrokerConfManager.getBrokerInfoCheckSum() != inBrokerCheckSum) {
- builder.addAllBrokerInfos(defaultBrokerConfManager.getBrokersMap(overtls).values());
+ Tuple2<Long, Map<Integer, String>> brokerStaticInfo =
+ brokerRunManager.getBrokerStaticInfo(overtls);
+ builder.setBrokerCheckSum(brokerStaticInfo.getF0());
+ if (brokerStaticInfo.getF0() != inBrokerCheckSum) {
+ builder.addAllBrokerInfos(brokerStaticInfo.getF1().values());
}
ClientMaster.ApprovedClientConfig.Builder clientConfigBuilder =
buildApprovedClientConfig(request.getAppdConfig());
@@ -515,6 +506,7 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
final String rmtAddress,
boolean overtls) throws Exception {
// #lizard forgives
+ ProcessResult result = new ProcessResult();
final StringBuilder strBuffer = new StringBuilder(512);
RegisterResponseM2C.Builder builder = RegisterResponseM2C.newBuilder();
builder.setSuccess(false);
@@ -581,7 +573,7 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
sessionKey, sessionTime, sourceCount, requiredPartMap);
paramCheckResult =
PBParameterUtils.checkConsumerInputInfo(inConsumerInfo,
- masterConfig, defaultBrokerConfManager, topicPSInfoManager, strBuffer);
+ masterConfig, defMetaDataManager, brokerRunManager, strBuffer);
if (!paramCheckResult.result) {
builder.setErrCode(paramCheckResult.errCode);
builder.setErrMsg(paramCheckResult.errMsg);
@@ -598,16 +590,14 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
return builder.build();
}
// need removed for authorize center begin
- TargetValidResult validResult =
- this.defaultBrokerConfManager
- .isConsumeTargetAuthorized(consumerId, groupName,
- reqTopicSet, reqTopicConditions, strBuffer);
- if (!validResult.result) {
+ if (!this.defMetaDataManager
+ .isConsumeTargetAuthorized(consumerId, groupName,
+ reqTopicSet, reqTopicConditions, strBuffer, result)) {
if (strBuffer.length() > 0) {
logger.warn(strBuffer.toString());
}
- builder.setErrCode(validResult.errCode);
- builder.setErrMsg(validResult.errInfo);
+ builder.setErrCode(result.getErrCode());
+ builder.setErrMsg(result.getErrInfo());
return builder.build();
}
// need removed for authorize center end
@@ -679,23 +669,22 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
builder.setQryPriorityId(TBaseConstants.META_VALUE_UNDEFINED);
builder.setDefFlowControlInfo(" ");
builder.setGroupFlowControlInfo(" ");
- BdbGroupFlowCtrlEntity defFlowCtrlEntity =
- defaultBrokerConfManager.getBdbDefFlowCtrl();
- BdbGroupFlowCtrlEntity bdbGroupFlowCtrlEntity =
- defaultBrokerConfManager.getBdbGroupFlowCtrl(groupName);
- if (defFlowCtrlEntity != null
- && defFlowCtrlEntity.isValidStatus()) {
- builder.setDefFlowCheckId(defFlowCtrlEntity.getSerialId());
- if (request.getDefFlowCheckId() != defFlowCtrlEntity.getSerialId()) {
- builder.setDefFlowControlInfo(defFlowCtrlEntity.getFlowCtrlInfo());
+ ClusterSettingEntity defSetting =
+ defMetaDataManager.getClusterDefSetting(false);
+ GroupResCtrlEntity groupResCtrlConf =
+ defMetaDataManager.confGetGroupResCtrlConf(groupName);
+ if (defSetting.enableFlowCtrl()) {
+ builder.setDefFlowCheckId(defSetting.getSerialId());
+ if (request.getDefFlowCheckId() != defSetting.getSerialId()) {
+ builder.setDefFlowControlInfo(defSetting.getGloFlowCtrlRuleInfo());
}
}
- if (bdbGroupFlowCtrlEntity != null
- && bdbGroupFlowCtrlEntity.isValidStatus()) {
- builder.setGroupFlowCheckId(bdbGroupFlowCtrlEntity.getSerialId());
- builder.setQryPriorityId(bdbGroupFlowCtrlEntity.getQryPriorityId());
- if (request.getGroupFlowCheckId() != bdbGroupFlowCtrlEntity.getSerialId()) {
- builder.setGroupFlowControlInfo(bdbGroupFlowCtrlEntity.getFlowCtrlInfo());
+ if (groupResCtrlConf != null
+ && groupResCtrlConf.isFlowCtrlEnable()) {
+ builder.setGroupFlowCheckId(groupResCtrlConf.getSerialId());
+ builder.setQryPriorityId(groupResCtrlConf.getQryPriorityId());
+ if (request.getGroupFlowCheckId() != groupResCtrlConf.getSerialId()) {
+ builder.setGroupFlowControlInfo(groupResCtrlConf.getFlowCtrlInfo());
}
}
}
@@ -855,23 +844,22 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
builder.setGroupFlowCheckId(TBaseConstants.META_VALUE_UNDEFINED);
builder.setDefFlowControlInfo(" ");
builder.setGroupFlowControlInfo(" ");
- BdbGroupFlowCtrlEntity defFlowCtrlEntity =
- defaultBrokerConfManager.getBdbDefFlowCtrl();
- BdbGroupFlowCtrlEntity bdbGroupFlowCtrlEntity =
- defaultBrokerConfManager.getBdbGroupFlowCtrl(groupName);
- if (defFlowCtrlEntity != null
- && defFlowCtrlEntity.isValidStatus()) {
- builder.setDefFlowCheckId(defFlowCtrlEntity.getSerialId());
- if (request.getDefFlowCheckId() != defFlowCtrlEntity.getSerialId()) {
- builder.setDefFlowControlInfo(defFlowCtrlEntity.getFlowCtrlInfo());
+ ClusterSettingEntity defSetting =
+ defMetaDataManager.getClusterDefSetting(false);
+ GroupResCtrlEntity groupResCtrlConf =
+ defMetaDataManager.confGetGroupResCtrlConf(groupName);
+ if (defSetting.enableFlowCtrl()) {
+ builder.setDefFlowCheckId(defSetting.getSerialId());
+ if (request.getDefFlowCheckId() != defSetting.getSerialId()) {
+ builder.setDefFlowControlInfo(defSetting.getGloFlowCtrlRuleInfo());
}
}
- if (bdbGroupFlowCtrlEntity != null
- && bdbGroupFlowCtrlEntity.isValidStatus()) {
- builder.setGroupFlowCheckId(bdbGroupFlowCtrlEntity.getSerialId());
- builder.setQryPriorityId(bdbGroupFlowCtrlEntity.getQryPriorityId());
- if (request.getGroupFlowCheckId() != bdbGroupFlowCtrlEntity.getSerialId()) {
- builder.setGroupFlowControlInfo(bdbGroupFlowCtrlEntity.getFlowCtrlInfo());
+ if (groupResCtrlConf != null
+ && groupResCtrlConf.isFlowCtrlEnable()) {
+ builder.setGroupFlowCheckId(groupResCtrlConf.getSerialId());
+ builder.setQryPriorityId(groupResCtrlConf.getQryPriorityId());
+ if (request.getGroupFlowCheckId() != groupResCtrlConf.getSerialId()) {
+ builder.setGroupFlowControlInfo(groupResCtrlConf.getFlowCtrlInfo());
}
}
}
@@ -947,20 +935,21 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
final String rmtAddress,
boolean overtls) throws Exception {
// #lizard forgives
- final StringBuilder strBuffer = new StringBuilder(512);
RegisterResponseM2B.Builder builder = RegisterResponseM2B.newBuilder();
builder.setSuccess(false);
builder.setStopRead(false);
builder.setStopWrite(false);
builder.setTakeConfInfo(false);
// auth
- CertifiedResult result =
+ CertifiedResult cfResult =
serverAuthHandler.identityValidBrokerInfo(request.getAuthInfo());
- if (!result.result) {
- builder.setErrCode(result.errCode);
- builder.setErrMsg(result.errInfo);
+ if (!cfResult.result) {
+ builder.setErrCode(cfResult.errCode);
+ builder.setErrMsg(cfResult.errInfo);
return builder.build();
}
+ ProcessResult result = new ProcessResult();
+ final StringBuilder strBuffer = new StringBuilder(512);
// get clientId and check valid
ParamCheckResult paramCheckResult =
PBParameterUtils.checkClientId(request.getClientId(), strBuffer);
@@ -972,82 +961,29 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
final String clientId = (String) paramCheckResult.checkData;
// check authority
checkNodeStatus(clientId, strBuffer);
- // check broker validity
- //
- BrokerInfo brokerInfo =
- new BrokerInfo(clientId, request.getEnableTls(),
- request.hasTlsPort() ? request.getTlsPort() : TBaseConstants.META_DEFAULT_BROKER_TLS_PORT);
- BdbBrokerConfEntity bdbBrokerConfEntity =
- defaultBrokerConfManager.getBrokerDefaultConfigStoreInfo(brokerInfo.getBrokerId());
- if (bdbBrokerConfEntity == null) {
- builder.setErrCode(TErrCodeConstants.BAD_REQUEST);
- builder.setErrMsg(strBuffer
- .append("No broker configure info, please create first! the connecting client id is:")
- .append(clientId).toString());
- return builder.build();
- }
- if ((!brokerInfo.getHost().equals(bdbBrokerConfEntity.getBrokerIp()))
- || (brokerInfo.getPort() != bdbBrokerConfEntity.getBrokerPort())) {
- builder.setErrCode(TErrCodeConstants.BAD_REQUEST);
- builder.setErrMsg(strBuffer
- .append("Inconsistent broker configure,please confirm first! the connecting client id is:")
- .append(clientId).append(", the configure's broker address by brokerId is:")
- .append(bdbBrokerConfEntity.getBrokerIdAndAddress()).toString());
- return builder.build();
- }
- int confTLSPort = bdbBrokerConfEntity.getBrokerTLSPort();
- if (confTLSPort != brokerInfo.getTlsPort()) {
- builder.setErrCode(TErrCodeConstants.BAD_REQUEST);
- builder.setErrMsg(strBuffer
- .append("Inconsistent TLS configure,please confirm first! the connecting client id is:")
- .append(clientId).append(", the configured TLS port is:")
- .append(confTLSPort).append(", the broker reported TLS port is ")
- .append(brokerInfo.getTlsPort()).toString());
- return builder.build();
- }
- if (bdbBrokerConfEntity.getManageStatus() == TStatusConstants.STATUS_MANAGE_APPLY) {
- builder.setErrCode(TErrCodeConstants.BAD_REQUEST);
- builder.setErrMsg(strBuffer.append("Broker's configure not online, ")
- .append("please online configure first! the connecting client id is:")
- .append(clientId).toString());
- return builder.build();
- }
// get optional filed
- boolean needFastStart = false;
+ ClusterSettingEntity defSetting =
+ defMetaDataManager.getClusterDefSetting(false);
final long reFlowCtrlId = request.hasFlowCheckId()
? request.getFlowCheckId() : TBaseConstants.META_VALUE_UNDEFINED;
final int qryPriorityId = request.hasQryPriorityId()
? request.getQryPriorityId() : TBaseConstants.META_VALUE_UNDEFINED;
- ConcurrentHashMap<Integer, BrokerSyncStatusInfo> brokerSyncStatusMap =
- this.defaultBrokerConfManager.getBrokerRunSyncManageMap();
- // update broker status
- List<String> brokerTopicSetConfInfo =
- this.defaultBrokerConfManager.getBrokerTopicStrConfigInfo(bdbBrokerConfEntity);
- BrokerSyncStatusInfo brokerStatusInfo =
- new BrokerSyncStatusInfo(bdbBrokerConfEntity, brokerTopicSetConfInfo);
- brokerSyncStatusMap.put(bdbBrokerConfEntity.getBrokerId(), brokerStatusInfo);
- brokerStatusInfo.updateCurrBrokerConfInfo(bdbBrokerConfEntity.getManageStatus(),
- bdbBrokerConfEntity.isConfDataUpdated(), bdbBrokerConfEntity.isBrokerLoaded(),
- bdbBrokerConfEntity.getBrokerDefaultConfInfo(), brokerTopicSetConfInfo, false);
- if (brokerTopicSetConfInfo.isEmpty()) {
- needFastStart = true;
- }
- brokerStatusInfo.setFastStart(needFastStart);
- // set broker report info
- if (request.getCurBrokerConfId() <= 0) {
- brokerStatusInfo.setBrokerReportInfo(true, request.getCurBrokerConfId(),
- request.getConfCheckSumId(), true, request.getBrokerDefaultConfInfo(),
- request.getBrokerTopicSetConfInfoList(), true, request.getBrokerOnline(), overtls);
- } else {
- brokerStatusInfo.setBrokerReportInfo(true,
- brokerStatusInfo.getLastPushBrokerConfId(),
- brokerStatusInfo.getLastPushBrokerCheckSumId(), true,
- bdbBrokerConfEntity.getBrokerDefaultConfInfo(), brokerTopicSetConfInfo, true,
- request.getBrokerOnline(), overtls);
- }
- this.defaultBrokerConfManager.removeBrokerRunTopicInfoMap(brokerInfo.getBrokerId());
- brokerHolder.setBrokerInfo(brokerInfo.getBrokerId(), brokerInfo);
- heartbeatManager.regBrokerNode(String.valueOf(brokerInfo.getBrokerId()));
+ int tlsPort = request.hasTlsPort()
+ ? request.getTlsPort() : defSetting.getBrokerTLSPort();
+ // build broker info
+ BrokerInfo brokerInfo =
+ new BrokerInfo(clientId, request.getEnableTls(), tlsPort);
+ // register broker run status info
+ if (!brokerRunManager.brokerRegister2M(clientId, brokerInfo,
+ request.getCurBrokerConfId(), request.getConfCheckSumId(),
+ true, request.getBrokerDefaultConfInfo(),
+ request.getBrokerTopicSetConfInfoList(), request.getBrokerOnline(),
+ overtls, strBuffer, result)) {
+ builder.setErrCode(result.getErrCode());
+ builder.setErrMsg(result.getErrInfo());
+ return builder.build();
+ }
+ // print broker register log
logger.info(strBuffer.append("[Broker Register] ").append(clientId)
.append(" report, configureId=").append(request.getCurBrokerConfId())
.append(",readStatusRpt=").append(request.getReadStatusRpt())
@@ -1058,9 +994,6 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
.append(",qryPriorityId=").append(qryPriorityId)
.append(",checksumId=").append(request.getConfCheckSumId()).toString());
strBuffer.delete(0, strBuffer.length());
- if (request.getCurBrokerConfId() > 0) {
- processBrokerReportConfigureInfo(brokerInfo, strBuffer);
- }
// response
builder.setSuccess(true);
builder.setErrCode(TErrCodeConstants.SUCCESS);
@@ -1076,11 +1009,7 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
enableInfo.setEnableConsumeAuthenticate(masterConfig.isStartConsumeAuthenticate());
enableInfo.setEnableConsumeAuthorize(masterConfig.isStartConsumeAuthorize());
builder.setEnableBrokerInfo(enableInfo);
- builder.setTakeConfInfo(true);
- builder.setCurBrokerConfId(brokerStatusInfo.getLastPushBrokerConfId());
- builder.setConfCheckSumId(brokerStatusInfo.getLastPushBrokerCheckSumId());
- builder.setBrokerDefaultConfInfo(brokerStatusInfo.getLastPushBrokerDefaultConfInfo());
- builder.addAllBrokerTopicSetConfInfo(brokerStatusInfo.getLastPushBrokerTopicSetConfInfo());
+ brokerRunManager.setRegisterDownConfInfo(brokerInfo.getBrokerId(), strBuffer, builder);
builder.setSsdStoreId(TBaseConstants.META_VALUE_UNDEFINED);
ClientMaster.ClusterConfig.Builder clusterConfigBuilder =
buildClusterConfig(request.getClsConfig());
@@ -1088,36 +1017,16 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
builder.setClsConfig(clusterConfigBuilder);
}
if (request.hasFlowCheckId()) {
- BdbGroupFlowCtrlEntity bdbGroupFlowCtrlEntity =
- defaultBrokerConfManager.getBdbDefFlowCtrl();
- if (bdbGroupFlowCtrlEntity == null) {
- builder.setFlowCheckId(TBaseConstants.META_VALUE_UNDEFINED);
- builder.setQryPriorityId(TBaseConstants.META_VALUE_UNDEFINED);
- if (request.getFlowCheckId() != TBaseConstants.META_VALUE_UNDEFINED) {
+ builder.setQryPriorityId(defSetting.getQryPriorityId());
+ builder.setFlowCheckId(defSetting.getSerialId());
+ if (reFlowCtrlId != defSetting.getSerialId()) {
+ if (defSetting.enableFlowCtrl()) {
+ builder.setFlowControlInfo(defSetting.getGloFlowCtrlRuleInfo());
+ } else {
builder.setFlowControlInfo(" ");
}
- } else {
- builder.setQryPriorityId(bdbGroupFlowCtrlEntity.getQryPriorityId());
- builder.setFlowCheckId(bdbGroupFlowCtrlEntity.getSerialId());
- if (reFlowCtrlId != bdbGroupFlowCtrlEntity.getSerialId()) {
- if (bdbGroupFlowCtrlEntity.isValidStatus()) {
- builder.setFlowControlInfo(bdbGroupFlowCtrlEntity.getFlowCtrlInfo());
- } else {
- builder.setFlowControlInfo(" ");
- }
- }
}
}
- logger.info(strBuffer.append("[TMaster sync] push broker configure: brokerId = ")
- .append(brokerStatusInfo.getBrokerId())
- .append(",configureId=").append(brokerStatusInfo.getLastPushBrokerConfId())
- .append(",stopWrite=").append(builder.getStopWrite())
- .append(",stopRead=").append(builder.getStopRead())
- .append(",checksumId=").append(brokerStatusInfo.getLastPushBrokerCheckSumId())
- .append(",default configure is ").append(brokerStatusInfo.getLastPushBrokerDefaultConfInfo())
- .append(",topic configure is ").append(brokerStatusInfo.getLastPushBrokerTopicSetConfInfo())
- .toString());
- strBuffer.delete(0, strBuffer.length());
logger.info(strBuffer.append("[Broker Register] ").append(clientId)
.append(", isOverTLS=").append(overtls).toString());
return builder.build();
@@ -1137,7 +1046,6 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
final String rmtAddress,
boolean overtls) throws Exception {
// #lizard forgives
- final StringBuilder strBuffer = new StringBuilder(512);
// set response field
HeartResponseM2B.Builder builder = HeartResponseM2B.newBuilder();
builder.setSuccess(false);
@@ -1151,13 +1059,15 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
builder.setCurBrokerConfId(TBaseConstants.META_VALUE_UNDEFINED);
builder.setConfCheckSumId(TBaseConstants.META_VALUE_UNDEFINED);
// identity broker info
- CertifiedResult result =
+ CertifiedResult certResult =
serverAuthHandler.identityValidBrokerInfo(request.getAuthInfo());
- if (!result.result) {
- builder.setErrCode(result.errCode);
- builder.setErrMsg(result.errInfo);
+ if (!certResult.result) {
+ builder.setErrCode(certResult.errCode);
+ builder.setErrMsg(certResult.errInfo);
return builder.build();
}
+ ProcessResult result = new ProcessResult();
+ final StringBuilder strBuffer = new StringBuilder(512);
ParamCheckResult paramCheckResult =
PBParameterUtils.checkBrokerId(request.getBrokerId(), strBuffer);
if (!paramCheckResult.result) {
@@ -1165,71 +1075,23 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
builder.setErrMsg(paramCheckResult.errMsg);
return builder.build();
}
- final String brokerId = (String) paramCheckResult.checkData;
- checkNodeStatus(brokerId, strBuffer);
- BrokerInfo brokerInfo = brokerHolder.getBrokerInfo(Integer.parseInt(brokerId));
- if (brokerInfo == null) {
- builder.setErrCode(TErrCodeConstants.HB_NO_NODE);
- builder.setErrMsg(strBuffer
- .append("Please register broker first! the connecting client id is:")
- .append(brokerId).toString());
- return builder.build();
- }
- BdbBrokerConfEntity bdbBrokerConfEntity =
- defaultBrokerConfManager.getBrokerDefaultConfigStoreInfo(brokerInfo.getBrokerId());
- if (bdbBrokerConfEntity == null) {
- builder.setErrCode(TErrCodeConstants.BAD_REQUEST);
- builder.setErrMsg(strBuffer
- .append("No broker configure info, please create first! the connecting client id is:")
- .append(brokerInfo.toString()).toString());
- return builder.build();
- }
- if (bdbBrokerConfEntity.getManageStatus() == TStatusConstants.STATUS_MANAGE_APPLY) {
- builder.setErrCode(TErrCodeConstants.BAD_REQUEST);
- builder.setErrMsg(strBuffer.append("Broker's configure not online, ")
- .append("please online configure first! the connecting client id is:")
- .append(brokerInfo.toString()).toString());
- return builder.build();
- }
- ConcurrentHashMap<Integer, BrokerSyncStatusInfo> brokerSyncStatusMap =
- this.defaultBrokerConfManager.getBrokerRunSyncManageMap();
- BrokerSyncStatusInfo brokerSyncStatusInfo =
- brokerSyncStatusMap.get(brokerInfo.getBrokerId());
- if (brokerSyncStatusInfo == null) {
- builder.setErrCode(TErrCodeConstants.BAD_REQUEST);
- builder.setErrMsg(strBuffer
- .append("Not found Broker run status info,please register first! the connecting client id is:")
- .append(brokerInfo.toString()).toString());
- return builder.build();
- }
- // update heartbeat
- try {
- heartbeatManager.updBrokerNode(brokerId);
- } catch (HeartbeatException e) {
- builder.setErrCode(TErrCodeConstants.HB_NO_NODE);
- builder.setErrMsg(e.getMessage());
- return builder.build();
- }
- // update broker status
- brokerSyncStatusInfo.setBrokerReportInfo(false, request.getCurBrokerConfId(),
- request.getConfCheckSumId(), request.getTakeConfInfo(),
- request.getBrokerDefaultConfInfo(), request.getBrokerTopicSetConfInfoList(),
- true, request.getBrokerOnline(), overtls);
- processBrokerReportConfigureInfo(brokerInfo, strBuffer);
- if (request.getTakeRemovedTopicInfo()) {
- List<String> removedTopics = request.getRemovedTopicsInfoList();
- logger.info(strBuffer.append("[Broker Report] receive broker confirmed removed topic list is ")
- .append(removedTopics.toString()).toString());
- strBuffer.delete(0, strBuffer.length());
- this.defaultBrokerConfManager
- .clearRemovedTopicEntityInfo(bdbBrokerConfEntity.getBrokerId(), removedTopics);
- }
- brokerHolder.updateBrokerReportStatus(brokerInfo.getBrokerId(),
- request.getReadStatusRpt(), request.getWriteStatusRpt());
+ int brokerId = (int) paramCheckResult.checkData;
long reFlowCtrlId = request.hasFlowCheckId()
? request.getFlowCheckId() : TBaseConstants.META_VALUE_UNDEFINED;
int qryPriorityId = request.hasQryPriorityId()
? request.getQryPriorityId() : TBaseConstants.META_VALUE_UNDEFINED;
+ checkNodeStatus(String.valueOf(brokerId), strBuffer);
+ if (!brokerRunManager.brokerHeartBeat2M(brokerId,
+ request.getCurBrokerConfId(), request.getConfCheckSumId(),
+ request.getTakeConfInfo(), request.getBrokerDefaultConfInfo(),
+ request.getBrokerTopicSetConfInfoList(), request.getTakeRemovedTopicInfo(),
+ request.getRemovedTopicsInfoList(), request.getReadStatusRpt(),
+ request.getWriteStatusRpt(), request.getBrokerOnline(),
+ strBuffer, result)) {
+ builder.setErrCode(result.getErrCode());
+ builder.setErrMsg(result.getErrInfo());
+ return builder.build();
+ }
if (request.getTakeConfInfo()) {
strBuffer.append("[Broker Report] heartbeat report: brokerId=")
.append(request.getBrokerId()).append(", configureId=")
@@ -1242,67 +1104,35 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
.append(",qryPriorityId=").append(qryPriorityId)
.append(",brokerOnline=").append(request.getBrokerOnline())
.append(",default broker configure is ").append(request.getBrokerDefaultConfInfo())
- .append(",broker topic configure is ").append(request.getBrokerTopicSetConfInfoList())
- .append(",current brokerSyncStatusInfo is ");
- logger.info(brokerSyncStatusInfo.toJsonString(strBuffer, true).toString());
+ .append(",broker topic configure is ").append(request.getBrokerTopicSetConfInfoList());
strBuffer.delete(0, strBuffer.length());
}
// create response
- builder.setNeedReportData(brokerSyncStatusInfo.needReportData());
- builder.setCurBrokerConfId(brokerSyncStatusInfo.getLastPushBrokerConfId());
- builder.setConfCheckSumId(brokerSyncStatusInfo.getLastPushBrokerCheckSumId());
+ brokerRunManager.setHeatBeatDownConfInfo(brokerId, strBuffer, builder);
+ BrokerConfEntity brokerConfEntity =
+ defMetaDataManager.getBrokerConfByBrokerId(brokerId);
+ builder.setTakeRemoveTopicInfo(true);
+ builder.addAllRemoveTopicConfInfo(defMetaDataManager
+ .getBrokerRemovedTopicStrConfigInfo(brokerConfEntity, strBuffer).values());
builder.setSsdStoreId(TBaseConstants.META_VALUE_UNDEFINED);
if (request.hasFlowCheckId()) {
- BdbGroupFlowCtrlEntity bdbGroupFlowCtrlEntity =
- defaultBrokerConfManager.getBdbDefFlowCtrl();
- if (bdbGroupFlowCtrlEntity == null) {
- builder.setFlowCheckId(TBaseConstants.META_VALUE_UNDEFINED);
- builder.setQryPriorityId(TBaseConstants.META_VALUE_UNDEFINED);
- if (request.getFlowCheckId() != TBaseConstants.META_VALUE_UNDEFINED) {
+ ClusterSettingEntity defSetting =
+ defMetaDataManager.getClusterDefSetting(false);
+ builder.setFlowCheckId(defSetting.getSerialId());
+ builder.setQryPriorityId(defSetting.getQryPriorityId());
+ if (reFlowCtrlId != defSetting.getSerialId()) {
+ if (defSetting.enableFlowCtrl()) {
+ builder.setFlowControlInfo(defSetting.getGloFlowCtrlRuleInfo());
+ } else {
builder.setFlowControlInfo(" ");
}
- } else {
- builder.setFlowCheckId(bdbGroupFlowCtrlEntity.getSerialId());
- builder.setQryPriorityId(bdbGroupFlowCtrlEntity.getQryPriorityId());
- if (reFlowCtrlId != bdbGroupFlowCtrlEntity.getSerialId()) {
- if (bdbGroupFlowCtrlEntity.isValidStatus()) {
- builder.setFlowControlInfo(bdbGroupFlowCtrlEntity.getFlowCtrlInfo());
- } else {
- builder.setFlowControlInfo(" ");
- }
- }
}
}
- brokerHolder.setBrokerHeartBeatReqStatus(brokerInfo.getBrokerId(), builder);
ClientMaster.ClusterConfig.Builder clusterConfigBuilder =
buildClusterConfig(request.getClsConfig());
if (clusterConfigBuilder != null) {
builder.setClsConfig(clusterConfigBuilder);
}
- builder.setTakeRemoveTopicInfo(true);
- builder.addAllRemoveTopicConfInfo(defaultBrokerConfManager
- .getBrokerRemovedTopicStrConfigInfo(bdbBrokerConfEntity));
- if (brokerSyncStatusInfo.needSyncConfDataToBroker()) {
- builder.setTakeConfInfo(true);
- builder.setBrokerDefaultConfInfo(brokerSyncStatusInfo
- .getLastPushBrokerDefaultConfInfo());
- builder.addAllBrokerTopicSetConfInfo(brokerSyncStatusInfo
- .getLastPushBrokerTopicSetConfInfo());
- logger.info(strBuffer
- .append("[Broker Report] heartbeat sync topic config: brokerId=")
- .append(brokerId).append(", configureId=")
- .append(brokerSyncStatusInfo.getLastPushBrokerConfId())
- .append(",set flowCtrlId=").append(builder.getFlowCheckId())
- .append(",stopWrite=").append(builder.getStopWrite())
- .append(",stopRead=").append(builder.getStopRead())
- .append(",qryPriorityId=").append(builder.getQryPriorityId())
- .append(",checksumId=").append(brokerSyncStatusInfo.getLastPushBrokerCheckSumId())
- .append(",default configure is ")
- .append(brokerSyncStatusInfo.getLastPushBrokerDefaultConfInfo())
- .append(",topic configure is ")
- .append(brokerSyncStatusInfo.getLastPushBrokerTopicSetConfInfo())
- .toString());
- }
// begin: deprecated when brokers version equal to current master version
builder.setAuthorizedInfo(genAuthorizedInfo(null, true));
// end deprecated
@@ -1326,14 +1156,15 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
public CloseResponseM2B brokerCloseClientB2M(CloseRequestB2M request,
final String rmtAddress,
boolean overtls) throws Throwable {
+ ProcessResult result = new ProcessResult();
StringBuilder strBuffer = new StringBuilder(512);
CloseResponseM2B.Builder builder = CloseResponseM2B.newBuilder();
builder.setSuccess(false);
- CertifiedResult result =
+ CertifiedResult cfResult =
serverAuthHandler.identityValidBrokerInfo(request.getAuthInfo());
- if (!result.result) {
- builder.setErrCode(result.errCode);
- builder.setErrMsg(result.errInfo);
+ if (!cfResult.result) {
+ builder.setErrCode(cfResult.errCode);
+ builder.setErrMsg(cfResult.errInfo);
return builder.build();
}
ParamCheckResult paramCheckResult =
@@ -1343,12 +1174,13 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
builder.setErrMsg(paramCheckResult.errMsg);
return builder.build();
}
- final String brokerId = (String) paramCheckResult.checkData;
- checkNodeStatus(brokerId, strBuffer);
- logger.info(strBuffer.append("[Broker Closed]").append(brokerId)
- .append(", isOverTLS=").append(overtls).toString());
- new ReleaseBroker().run(brokerId);
- heartbeatManager.unRegBrokerNode(request.getBrokerId());
+ final int brokerId = (int) paramCheckResult.checkData;
+ checkNodeStatus(String.valueOf(brokerId), strBuffer);
+ if (!brokerRunManager.brokerClose2M(brokerId, strBuffer, result)) {
+ builder.setErrCode(result.getErrCode());
+ builder.setErrMsg(result.getErrInfo());
+ return builder.build();
+ }
builder.setSuccess(true);
builder.setErrCode(TErrCodeConstants.SUCCESS);
builder.setErrMsg("OK!");
@@ -1374,481 +1206,18 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
* @return
*/
private Map<String, String> getProducerTopicPartitionInfo(String producerId) {
- Map<String, String> topicPartStrMap = new HashMap<>();
- ProducerInfo producerInfo = producerHolder.getProducerInfo(producerId);
+ ProducerInfo producerInfo =
+ producerHolder.getProducerInfo(producerId);
if (producerInfo == null) {
- return topicPartStrMap;
+ return new HashMap<>();
}
Set<String> producerInfoTopicSet =
producerInfo.getTopicSet();
if ((producerInfoTopicSet == null)
|| (producerInfoTopicSet.isEmpty())) {
- return topicPartStrMap;
- }
- Map<String, StringBuilder> topicPartStrBuilderMap =
- new HashMap<>();
- for (String topic : producerInfoTopicSet) {
- if (topic == null) {
- continue;
- }
- ConcurrentHashMap<BrokerInfo, TopicInfo> topicInfoMap =
- topicPSInfoManager.getBrokerPubInfo(topic);
- if (topicInfoMap == null) {
- continue;
- }
- for (Map.Entry<BrokerInfo, TopicInfo> entry : topicInfoMap.entrySet()) {
- if (entry.getKey() == null || entry.getValue() == null) {
- continue;
- }
- if (entry.getValue().isAcceptPublish()) {
- StringBuilder tmpValue = topicPartStrBuilderMap.get(topic);
- if (tmpValue == null) {
- StringBuilder strBuffer =
- new StringBuilder(512).append(topic)
- .append(TokenConstants.SEGMENT_SEP)
- .append(entry.getValue().getSimpleValue());
- topicPartStrBuilderMap.put(topic, strBuffer);
- } else {
- tmpValue.append(TokenConstants.ARRAY_SEP)
- .append(entry.getValue().getSimpleValue());
- }
- }
- }
- }
- for (Map.Entry<String, StringBuilder> entry : topicPartStrBuilderMap.entrySet()) {
- if (entry.getValue() != null) {
- topicPartStrMap.put(entry.getKey(), entry.getValue().toString());
- }
- }
- topicPartStrBuilderMap.clear();
- return topicPartStrMap;
- }
-
- /**
- * Update topics
- *
- * @param brokerInfo
- * @param strBuffer
- * @param curTopicInfoMap
- * @param newTopicInfoMap
- * @param requirePartUpdate
- * @param requireAcceptPublish
- * @param requireAcceptSubscribe
- */
- private void updateTopics(BrokerInfo brokerInfo, final StringBuilder strBuffer,
- Map<String/* topicName */, TopicInfo> curTopicInfoMap,
- Map<String/* topicName */, TopicInfo> newTopicInfoMap,
- boolean requirePartUpdate, boolean requireAcceptPublish,
- boolean requireAcceptSubscribe) {
- List<TopicInfo> needAddTopicList = new ArrayList<>();
- for (Map.Entry<String, TopicInfo> entry : newTopicInfoMap.entrySet()) {
- TopicInfo newTopicInfo = entry.getValue();
- TopicInfo oldTopicInfo = null;
- if (curTopicInfoMap != null) {
- oldTopicInfo = curTopicInfoMap.get(entry.getKey());
- }
- if (oldTopicInfo == null
- || oldTopicInfo.getPartitionNum() != newTopicInfo.getPartitionNum()
- || oldTopicInfo.getTopicStoreNum() != newTopicInfo.getTopicStoreNum()
- || oldTopicInfo.isAcceptPublish() != newTopicInfo.isAcceptPublish()
- || oldTopicInfo.isAcceptSubscribe() != newTopicInfo.isAcceptSubscribe()) {
- if (requirePartUpdate) {
- if (!requireAcceptPublish) {
- newTopicInfo.setAcceptPublish(false);
- }
- if (!requireAcceptSubscribe) {
- newTopicInfo.setAcceptSubscribe(false);
- }
- }
- needAddTopicList.add(newTopicInfo);
- }
- }
- updateTopicsInternal(brokerInfo, needAddTopicList, EventType.CONNECT);
- logger.info(strBuffer.append("[addedTopicConfigMap] broker:")
- .append(brokerInfo.toString()).append(" add topicInfo list:")
- .append(needAddTopicList).toString());
- strBuffer.delete(0, strBuffer.length());
- }
-
- /**
- * Delete topics
- *
- * @param brokerInfo
- * @param strBuffer
- * @param curTopicInfoMap
- * @param newTopicInfoMap
- */
- private void deleteTopics(BrokerInfo brokerInfo, final StringBuilder strBuffer,
- Map<String/* topicName */, TopicInfo> curTopicInfoMap,
- Map<String/* topicName */, TopicInfo> newTopicInfoMap) {
- List<TopicInfo> needRmvTopicList = new ArrayList<>();
- if (curTopicInfoMap != null) {
- for (Map.Entry<String, TopicInfo> entry : curTopicInfoMap.entrySet()) {
- if (newTopicInfoMap.get(entry.getKey()) == null) {
- needRmvTopicList.add(entry.getValue());
- }
- }
- }
- updateTopicsInternal(brokerInfo, needRmvTopicList, EventType.DISCONNECT);
- logger.info(strBuffer.append("[removedTopicConfigMap] broker:")
- .append(brokerInfo.toString()).append(" removed topicInfo list:")
- .append(needRmvTopicList).toString());
- strBuffer.delete(0, strBuffer.length());
- }
-
- /**
- * Process broker report configure info
- *
- * @param brokerInfo
- * @param strBuffer
- */
- private void processBrokerReportConfigureInfo(BrokerInfo brokerInfo,
- final StringBuilder strBuffer) {
- // #lizard forgives
- BrokerSyncStatusInfo brokerSyncStatusInfo =
- this.defaultBrokerConfManager.getBrokerRunSyncStatusInfo(brokerInfo.getBrokerId());
- if (brokerSyncStatusInfo == null) {
- logger.error(strBuffer
- .append("Fail to find broker run manage configure! broker is ")
- .append(brokerInfo.toString()).toString());
- strBuffer.delete(0, strBuffer.length());
- return;
- }
- boolean requireAcceptPublish = false;
- boolean requireAcceptSubscribe = false;
- boolean requirePartUpdate = false;
- boolean requireSyncClient = false;
- int brokerManageStatus = brokerSyncStatusInfo.getBrokerManageStatus();
- int brokerRunStatus = brokerSyncStatusInfo.getBrokerRunStatus();
- long subStepOpTimeInMills = brokerSyncStatusInfo.getSubStepOpTimeInMills();
- boolean isBrokerRegister = brokerSyncStatusInfo.isBrokerRegister();
- boolean isBrokerOnline = brokerSyncStatusInfo.isBrokerOnline();
- if (!isBrokerRegister) {
- return;
- }
- if (brokerManageStatus == TStatusConstants.STATUS_MANAGE_ONLINE) {
- if (isBrokerOnline) {
- if (brokerRunStatus == TStatusConstants.STATUS_SERVICE_UNDEFINED) {
- return;
- } else if (brokerRunStatus == TStatusConstants.STATUS_SERVICE_TOONLINE_WAIT_REGISTER
- || brokerRunStatus == TStatusConstants.STATUS_SERVICE_TOONLINE_WAIT_ONLINE) {
- brokerSyncStatusInfo.setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_TOONLINE_ONLY_READ);
- requireAcceptSubscribe = true;
- requireSyncClient = true;
- } else if (brokerRunStatus == TStatusConstants.STATUS_SERVICE_TOONLINE_ONLY_READ) {
- if ((brokerSyncStatusInfo.isBrokerConfChanged())
- || (!brokerSyncStatusInfo.isBrokerLoaded())) {
- long waitTime =
- brokerSyncStatusInfo.isFastStart() ? masterConfig.getStepChgWaitPeriodMs()
- : masterConfig.getOnlineOnlyReadToRWPeriodMs();
- if ((System.currentTimeMillis() - subStepOpTimeInMills) > waitTime) {
- brokerSyncStatusInfo
- .setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_TOONLINE_READ_AND_WRITE);
- requireAcceptPublish = true;
- requireAcceptSubscribe = true;
- requireSyncClient = true;
- }
- } else {
- brokerSyncStatusInfo
- .setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_TOONLINE_READ_AND_WRITE);
- requireAcceptPublish = true;
- requireAcceptSubscribe = true;
- requireSyncClient = true;
- }
- } else if (brokerRunStatus == TStatusConstants.STATUS_SERVICE_TOONLINE_READ_AND_WRITE) {
- long waitTime =
- brokerSyncStatusInfo.isFastStart() ? 0 : masterConfig.getStepChgWaitPeriodMs();
- if ((System.currentTimeMillis() - subStepOpTimeInMills) > waitTime) {
- brokerSyncStatusInfo.setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_UNDEFINED);
- brokerSyncStatusInfo.setFastStart(true);
- requireAcceptPublish = true;
- requireAcceptSubscribe = true;
- requireSyncClient = true;
- if ((brokerSyncStatusInfo.isBrokerConfChanged())
- || (!brokerSyncStatusInfo.isBrokerLoaded())) {
- this.defaultBrokerConfManager
- .updateBrokerConfChanged(brokerInfo.getBrokerId(), false, true);
- }
- }
- } else if (brokerRunStatus == TStatusConstants.STATUS_SERVICE_TOONLINE_PART_WAIT_REGISTER) {
- brokerSyncStatusInfo
- .setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_TOONLINE_PART_WAIT_ONLINE);
- requireAcceptSubscribe = true;
- requireSyncClient = true;
- requirePartUpdate = true;
- } else if (brokerRunStatus == TStatusConstants.STATUS_SERVICE_TOONLINE_PART_WAIT_ONLINE) {
- brokerSyncStatusInfo
- .setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_TOONLINE_PART_ONLY_READ);
- requireAcceptSubscribe = true;
- requireSyncClient = true;
- requirePartUpdate = true;
- } else if (brokerRunStatus == TStatusConstants.STATUS_SERVICE_TOONLINE_PART_ONLY_READ) {
- if ((brokerSyncStatusInfo.isBrokerConfChanged())
- || (!brokerSyncStatusInfo.isBrokerLoaded())) {
- long waitTime =
- brokerSyncStatusInfo.isFastStart()
- ? 0 : masterConfig.getOnlineOnlyReadToRWPeriodMs();
- if ((System.currentTimeMillis() - subStepOpTimeInMills) > waitTime) {
- brokerSyncStatusInfo
- .setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_TOONLINE_READ_AND_WRITE);
- requireAcceptPublish = true;
- requireAcceptSubscribe = true;
- requireSyncClient = true;
- requirePartUpdate = true;
- }
- } else {
- brokerSyncStatusInfo
- .setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_TOONLINE_READ_AND_WRITE);
- requireAcceptPublish = true;
- requireAcceptSubscribe = true;
- requireSyncClient = true;
- requirePartUpdate = true;
- }
- }
- } else {
- if (brokerRunStatus != TStatusConstants.STATUS_SERVICE_TOONLINE_WAIT_ONLINE) {
- brokerSyncStatusInfo
- .setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_TOONLINE_WAIT_ONLINE);
- requireSyncClient = true;
- }
- }
- } else {
- if (!isBrokerOnline
- || brokerRunStatus == TStatusConstants.STATUS_SERVICE_UNDEFINED) {
- return;
- }
- if (brokerManageStatus == TStatusConstants.STATUS_MANAGE_ONLINE_NOT_WRITE) {
- if (brokerRunStatus == TStatusConstants.STATUS_SERVICE_TOONLINE_ONLY_READ) {
- long waitTime =
- brokerSyncStatusInfo.isFastStart() ? 0 : masterConfig.getStepChgWaitPeriodMs();
- if ((System.currentTimeMillis() - subStepOpTimeInMills) > waitTime) {
- brokerSyncStatusInfo
- .setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_TOOFFLINE_WAIT_REBALANCE);
- requireAcceptSubscribe = true;
- requireSyncClient = true;
- }
- } else if (brokerRunStatus == TStatusConstants.STATUS_SERVICE_TOOFFLINE_WAIT_REBALANCE) {
- long waitTime =
- brokerSyncStatusInfo.isFastStart()
- ? masterConfig.getStepChgWaitPeriodMs()
- : masterConfig.getOfflineOnlyReadToRWPeriodMs();
- if ((System.currentTimeMillis() - subStepOpTimeInMills) > waitTime) {
- brokerSyncStatusInfo.setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_UNDEFINED);
- brokerSyncStatusInfo.setFastStart(true);
- requireAcceptSubscribe = true;
- requireSyncClient = true;
- }
- }
- } else if (brokerManageStatus == TStatusConstants.STATUS_MANAGE_ONLINE_NOT_READ) {
- if (brokerRunStatus == TStatusConstants.STATUS_SERVICE_TOONLINE_ONLY_WRITE) {
- long waitTime =
- brokerSyncStatusInfo.isFastStart() ? 0 : masterConfig.getStepChgWaitPeriodMs();
- if ((System.currentTimeMillis() - subStepOpTimeInMills) > waitTime) {
- brokerSyncStatusInfo
- .setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_TOOFFLINE_WAIT_REBALANCE);
- requireAcceptPublish = true;
- requireSyncClient = true;
- }
- } else if (brokerRunStatus == TStatusConstants.STATUS_SERVICE_TOOFFLINE_WAIT_REBALANCE) {
- long waitTime =
- brokerSyncStatusInfo.isFastStart() ? masterConfig.getStepChgWaitPeriodMs()
- : masterConfig.getOfflineOnlyReadToRWPeriodMs();
- if ((System.currentTimeMillis() - subStepOpTimeInMills) > waitTime) {
- brokerSyncStatusInfo.setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_UNDEFINED);
- brokerSyncStatusInfo.setFastStart(true);
- requireAcceptPublish = true;
- requireSyncClient = true;
- }
- }
- } else if (brokerManageStatus == TStatusConstants.STATUS_MANAGE_OFFLINE) {
- if (brokerRunStatus == TStatusConstants.STATUS_SERVICE_TOOFFLINE_NOT_WRITE) {
- brokerSyncStatusInfo
- .setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_TOOFFLINE_NOT_READ_WRITE);
- requireAcceptSubscribe = true;
- requireSyncClient = true;
- } else if (brokerRunStatus == TStatusConstants.STATUS_SERVICE_TOOFFLINE_NOT_READ_WRITE) {
- long waitTime =
- brokerSyncStatusInfo.isFastStart() ? 0 : masterConfig.getStepChgWaitPeriodMs();
- if ((System.currentTimeMillis() - subStepOpTimeInMills) > waitTime) {
- brokerSyncStatusInfo
- .setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_TOOFFLINE_WAIT_REBALANCE);
- requireSyncClient = true;
- }
- } else if (brokerRunStatus == TStatusConstants.STATUS_SERVICE_TOOFFLINE_WAIT_REBALANCE) {
- long waitTime = brokerSyncStatusInfo.isFastStart()
- ? masterConfig.getStepChgWaitPeriodMs()
- : masterConfig.getOfflineOnlyReadToRWPeriodMs();
- if ((System.currentTimeMillis() - subStepOpTimeInMills) > waitTime) {
- brokerSyncStatusInfo.setBrokerRunStatus(TStatusConstants.STATUS_SERVICE_UNDEFINED);
- brokerSyncStatusInfo.setFastStart(true);
- requireSyncClient = true;
- }
- }
- }
- }
-
- if (requireSyncClient) {
- updateTopicInfoToClient(brokerInfo, requirePartUpdate,
- requireAcceptPublish, requireAcceptSubscribe, strBuffer);
- }
- }
-
- /**
- * Update topic info to client
- *
- * @param brokerInfo
- * @param requirePartUpdate
- * @param requireAcceptPublish
- * @param requireAcceptSubscribe
- * @param strBuffer
- */
- private void updateTopicInfoToClient(BrokerInfo brokerInfo,
- boolean requirePartUpdate,
- boolean requireAcceptPublish,
- boolean requireAcceptSubscribe,
- final StringBuilder strBuffer) {
- // #lizard forgives
- // check broker status
- BrokerSyncStatusInfo brokerSyncStatusInfo =
- this.defaultBrokerConfManager.getBrokerRunSyncStatusInfo(brokerInfo.getBrokerId());
- if (brokerSyncStatusInfo == null) {
- logger.error(strBuffer
- .append("Fail to find broker run manage configure, not update topic info! broker is ")
- .append(brokerInfo.toString()).toString());
- strBuffer.delete(0, strBuffer.length());
- return;
- }
- // get broker config and then generate topic status record
- String brokerDefaultConfInfo = brokerSyncStatusInfo.getReportedBrokerDefaultConfInfo();
- int brokerManageStatusId = brokerSyncStatusInfo.getBrokerManageStatus();
- if (TStringUtils.isBlank(brokerDefaultConfInfo)) {
- return;
- }
- // get broker status and topic default config
- boolean acceptPublish = false;
- boolean acceptSubscribe = false;
- if (brokerManageStatusId >= TStatusConstants.STATUS_MANAGE_ONLINE) {
- if (brokerManageStatusId == TStatusConstants.STATUS_MANAGE_ONLINE) {
- acceptPublish = true;
- acceptSubscribe = true;
- } else if (brokerManageStatusId == TStatusConstants.STATUS_MANAGE_ONLINE_NOT_WRITE) {
- acceptSubscribe = true;
- } else if (brokerManageStatusId == TStatusConstants.STATUS_MANAGE_ONLINE_NOT_READ) {
- acceptPublish = true;
- }
- }
- List<String> brokerTopicSetConfInfo =
- brokerSyncStatusInfo.getReportedBrokerTopicSetConfInfo();
- String[] brokerDefaultConfInfoArr = brokerDefaultConfInfo.split(TokenConstants.ATTR_SEP);
- int numPartitions = Integer.parseInt(brokerDefaultConfInfoArr[0]);
- boolean cfgAcceptPublish = Boolean.parseBoolean(brokerDefaultConfInfoArr[1]);
- boolean cfgAcceptSubscribe = Boolean.parseBoolean(brokerDefaultConfInfoArr[2]);
- int numTopicStores = 1;
- if (brokerDefaultConfInfoArr.length > 7) {
- if (!TStringUtils.isBlank(brokerDefaultConfInfoArr[7])) {
- numTopicStores = Integer.parseInt(brokerDefaultConfInfoArr[7]);
- }
- }
- int unFlushDataHold = TServerConstants.CFG_DEFAULT_DATA_UNFLUSH_HOLD;
- if (brokerDefaultConfInfoArr.length > 8) {
- if (!TStringUtils.isBlank(brokerDefaultConfInfoArr[8])) {
- unFlushDataHold = Integer.parseInt(brokerDefaultConfInfoArr[8]);
- }
- }
- ConcurrentHashMap<String/* topic */, TopicInfo> newTopicInfoMap =
- new ConcurrentHashMap<>();
- // according to broker status and default config, topic config, make up current status record
- for (String strTopicConfInfo : brokerTopicSetConfInfo) {
- if (TStringUtils.isBlank(strTopicConfInfo)) {
- continue;
- }
- String[] topicConfInfoArr =
- strTopicConfInfo.split(TokenConstants.ATTR_SEP);
- final String tmpTopic = topicConfInfoArr[0];
- int tmpPartNum = numPartitions;
- if (!TStringUtils.isBlank(topicConfInfoArr[1])) {
- tmpPartNum = Integer.parseInt(topicConfInfoArr[1]);
- }
- boolean tmpAcceptPublish = cfgAcceptPublish;
- if (!TStringUtils.isBlank(topicConfInfoArr[2])) {
- tmpAcceptPublish = Boolean.parseBoolean(topicConfInfoArr[2]);
- }
- if (!acceptPublish) {
- tmpAcceptPublish = acceptPublish;
- } else {
- if (!requirePartUpdate) {
- if (!requireAcceptPublish) {
- tmpAcceptPublish = false;
- }
- }
- }
- int tmpNumTopicStores = numTopicStores;
- if (!TStringUtils.isBlank(topicConfInfoArr[8])) {
- tmpNumTopicStores = Integer.parseInt(topicConfInfoArr[8]);
- tmpNumTopicStores = tmpNumTopicStores > 0 ? tmpNumTopicStores : numTopicStores;
- }
- boolean tmpAcceptSubscribe = cfgAcceptSubscribe;
- if (!TStringUtils.isBlank(topicConfInfoArr[3])) {
- tmpAcceptSubscribe = Boolean.parseBoolean(topicConfInfoArr[3]);
- }
- if (!acceptSubscribe) {
- tmpAcceptSubscribe = acceptSubscribe;
- } else {
- if (!requirePartUpdate) {
- if (!requireAcceptSubscribe) {
- tmpAcceptSubscribe = false;
- }
- }
- }
- newTopicInfoMap.put(tmpTopic, new TopicInfo(brokerInfo, tmpTopic,
- tmpPartNum, tmpNumTopicStores, tmpAcceptPublish, tmpAcceptSubscribe));
- }
-
- ConcurrentHashMap<String/* topicName */, TopicInfo> oldTopicInfoMap =
- defaultBrokerConfManager.getBrokerRunTopicInfoMap(brokerInfo.getBrokerId());
- deleteTopics(brokerInfo, strBuffer, oldTopicInfoMap, newTopicInfoMap);
- updateTopics(brokerInfo, strBuffer, oldTopicInfoMap, newTopicInfoMap,
- requirePartUpdate, requireAcceptPublish, requireAcceptSubscribe);
- defaultBrokerConfManager.updateBrokerRunTopicInfoMap(brokerInfo.getBrokerId(), newTopicInfoMap);
- }
-
- /**
- * Update topic internal
- *
- * @param broker
- * @param topicList
- * @param type
- */
- private void updateTopicsInternal(BrokerInfo broker,
- List<TopicInfo> topicList,
- EventType type) {
- List<TopicInfo> cloneTopicList = new ArrayList<>();
- for (TopicInfo topicInfo : topicList) {
- cloneTopicList.add(topicInfo.clone());
- }
- for (TopicInfo topicInfo : cloneTopicList) {
- Integer lid = null;
- try {
- lid = this.masterRowLock.getLock(null, StringUtils.getBytesUtf8(topicInfo.getTopic()), true);
- ConcurrentHashMap<BrokerInfo, TopicInfo> topicInfoMap =
- topicPSInfoManager.getBrokerPubInfo(topicInfo.getTopic());
- if (topicInfoMap == null) {
- topicInfoMap = new ConcurrentHashMap<>();
- topicPSInfoManager.setBrokerPubInfo(topicInfo.getTopic(), topicInfoMap);
- }
- if (EventType.CONNECT == type) {
- topicInfoMap.put(broker, topicInfo);
- } else {
- topicInfoMap.remove(broker);
- }
- } catch (IOException e) {
- logger.error("Get lock error!", e);
- } finally {
- if (lid != null) {
- this.masterRowLock.releaseRowLock(lid);
- }
- }
+ return new HashMap<>();
}
+ return brokerRunManager.getPubBrokerAcceptPubPartInfo(producerInfoTopicSet);
}
@Override
@@ -1878,11 +1247,11 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
private void balance(final TMaster tMaster) {
final StringBuilder strBuffer = new StringBuilder(512);
final long rebalanceId = idGenerator.incrementAndGet();
- if (defaultBdbStoreService != null) {
+ if (defMetaDataManager != null) {
logger.info(strBuffer.append("[Rebalance Start] ").append(rebalanceId)
- .append(", isMaster=").append(defaultBdbStoreService.isMaster())
+ .append(", isMaster=").append(defMetaDataManager.isSelfMaster())
.append(", isPrimaryNodeActive=")
- .append(defaultBdbStoreService.isPrimaryNodeActive()).toString());
+ .append(defMetaDataManager.isPrimaryNodeActive()).toString());
} else {
logger.info(strBuffer.append("[Rebalance Start] ").append(rebalanceId)
.append(", BDB service is null isMaster= false, isPrimaryNodeActive=false").toString());
@@ -1959,13 +1328,12 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
// choose different load balance strategy
if (isFirstReb) {
finalSubInfoMap = this.loadBalancer.bukAssign(consumerHolder,
- topicPSInfoManager, groups, defaultBrokerConfManager,
+ brokerRunManager, groups, defMetaDataManager,
masterConfig.getMaxGroupBrokerConsumeRate(), strBuffer);
} else {
finalSubInfoMap = this.loadBalancer.balanceCluster(currentSubInfo,
- consumerHolder, brokerHolder, topicPSInfoManager, groups,
- defaultBrokerConfManager, masterConfig.getMaxGroupBrokerConsumeRate(),
- strBuffer);
+ consumerHolder, brokerRunManager, groups, defMetaDataManager,
+ masterConfig.getMaxGroupBrokerConsumeRate(), strBuffer);
}
// allocate partitions to consumers
for (Map.Entry<String, Map<String, List<Partition>>> entry : finalSubInfoMap.entrySet()) {
@@ -1980,8 +1348,8 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
|| tupleInfo.getF1() == null) {
continue;
}
- List<String> blackTopicList =
- this.defaultBrokerConfManager.getBdbBlackTopicList(tupleInfo.getF0());
+ Set<String> blackTopicSet =
+ defMetaDataManager.getDisableConsumeTopicByGroupName(tupleInfo.getF0());
Map<String, List<Partition>> topicSubPartMap = entry.getValue();
List<SubscribeInfo> deletedSubInfoList = new ArrayList<>();
List<SubscribeInfo> addedSubInfoList = new ArrayList<>();
@@ -2001,7 +1369,7 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
}
if (tupleInfo.getF1().isOverTLS()) {
for (Partition currentPart : currentPartMap.values()) {
- if (!blackTopicList.contains(currentPart.getTopic())) {
+ if (!blackTopicSet.contains(currentPart.getTopic())) {
boolean found = false;
for (Partition newPart : finalPartList) {
if (newPart.getPartitionFullStr(true)
@@ -2019,7 +1387,7 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
tupleInfo.getF1().isOverTLS(), currentPart));
}
for (Partition finalPart : finalPartList) {
- if (!blackTopicList.contains(finalPart.getTopic())) {
+ if (!blackTopicSet.contains(finalPart.getTopic())) {
boolean found = false;
for (Partition curPart : currentPartMap.values()) {
if (finalPart.getPartitionFullStr(true)
@@ -2037,7 +1405,7 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
}
} else {
for (Partition currentPart : currentPartMap.values()) {
- if ((blackTopicList.contains(currentPart.getTopic()))
+ if ((blackTopicSet.contains(currentPart.getTopic()))
|| (!finalPartList.contains(currentPart))) {
deletedSubInfoList.add(new SubscribeInfo(consumerId,
tupleInfo.getF0(), false, currentPart));
@@ -2045,7 +1413,7 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
}
for (Partition finalPart : finalPartList) {
if ((currentPartMap.get(finalPart.getPartitionKey()) == null)
- && (!blackTopicList.contains(finalPart.getTopic()))) {
+ && (!blackTopicSet.contains(finalPart.getTopic()))) {
addedSubInfoList.add(new SubscribeInfo(consumerId,
tupleInfo.getF0(), false, finalPart));
}
@@ -2093,12 +1461,12 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
// choose different load balance strategy
if (isFirstReb) {
finalSubInfoMap = this.loadBalancer.resetBukAssign(consumerHolder,
- topicPSInfoManager, groups, this.zkOffsetStorage,
- this.defaultBrokerConfManager, strBuffer);
+ brokerRunManager, groups, this.zkOffsetStorage,
+ this.defMetaDataManager, strBuffer);
} else {
finalSubInfoMap = this.loadBalancer.resetBalanceCluster(currentSubInfo,
- consumerHolder, topicPSInfoManager, groups, this.zkOffsetStorage,
- this.defaultBrokerConfManager, strBuffer);
+ consumerHolder, brokerRunManager, groups, this.zkOffsetStorage,
+ this.defMetaDataManager, strBuffer);
}
// filter
for (Map.Entry<String, Map<String, Map<String, Partition>>> entry
@@ -2115,8 +1483,8 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
continue;
}
// allocate partitions to consumers
- List<String> blackTopicList =
- this.defaultBrokerConfManager.getBdbBlackTopicList(tupleInfo.getF0());
+ Set<String> blackTopicSet =
+ defMetaDataManager.getDisableConsumeTopicByGroupName(tupleInfo.getF0());
Map<String, Map<String, Partition>> topicSubPartMap = entry.getValue();
List<SubscribeInfo> deletedSubInfoList = new ArrayList<>();
List<SubscribeInfo> addedSubInfoList = new ArrayList<>();
@@ -2137,7 +1505,7 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
}
// filter
for (Partition currentPart : currentPartMap.values()) {
- if ((blackTopicList.contains(currentPart.getTopic()))
+ if ((blackTopicSet.contains(currentPart.getTopic()))
|| (finalPartMap.get(currentPart.getPartitionKey()) == null)) {
deletedSubInfoList
.add(new SubscribeInfo(consumerId, tupleInfo.getF0(),
@@ -2146,7 +1514,7 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
}
for (Partition finalPart : finalPartMap.values()) {
if ((currentPartMap.get(finalPart.getPartitionKey()) == null)
- && (!blackTopicList.contains(finalPart.getTopic()))) {
+ && (!blackTopicSet.contains(finalPart.getTopic()))) {
addedSubInfoList.add(new SubscribeInfo(consumerId, tupleInfo.getF0(),
tupleInfo.getF1().isOverTLS(), finalPart));
}
@@ -2325,13 +1693,13 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
ClientMaster.ApprovedClientConfig.Builder outClientConfig = null;
if (inClientConfig != null) {
outClientConfig = ClientMaster.ApprovedClientConfig.newBuilder();
- BdbClusterSettingEntity settingEntity =
- this.defaultBrokerConfManager.getBdbClusterSetting();
+ ClusterSettingEntity settingEntity =
+ this.defMetaDataManager.getClusterDefSetting(false);
if (settingEntity == null) {
outClientConfig.setConfigId(TBaseConstants.META_VALUE_UNDEFINED);
} else {
- outClientConfig.setConfigId(settingEntity.getConfigId());
- if (settingEntity.getConfigId() != inClientConfig.getConfigId()) {
+ outClientConfig.setConfigId(settingEntity.getSerialId());
+ if (settingEntity.getSerialId() != inClientConfig.getConfigId()) {
outClientConfig.setMaxMsgSize(settingEntity.getMaxMsgSizeInB());
}
}
@@ -2351,13 +1719,13 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
ClientMaster.ClusterConfig.Builder outClsConfig = null;
if (inClusterConfig != null) {
outClsConfig = ClientMaster.ClusterConfig.newBuilder();
- BdbClusterSettingEntity settingEntity =
- this.defaultBrokerConfManager.getBdbClusterSetting();
+ ClusterSettingEntity settingEntity =
+ this.defMetaDataManager.getClusterDefSetting(false);
if (settingEntity == null) {
outClsConfig.setConfigId(TBaseConstants.META_VALUE_UNDEFINED);
} else {
- outClsConfig.setConfigId(settingEntity.getConfigId());
- if (settingEntity.getConfigId() != inClusterConfig.getConfigId()) {
+ outClsConfig.setConfigId(settingEntity.getSerialId());
+ if (settingEntity.getSerialId() != inClusterConfig.getConfigId()) {
outClsConfig.setMaxMsgSize(settingEntity.getMaxMsgSizeInB());
}
}
@@ -2406,8 +1774,7 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
stopChores();
heartbeatManager.stop();
zkOffsetStorage.close();
- defaultBrokerConfManager.stop();
- defaultBdbStoreService.stop();
+ defMetaDataManager.stop();
visitTokenManager.stop();
if (!shutdownHooked.get()) {
Runtime.getRuntime().removeShutdownHook(shutdownHook);
@@ -2435,8 +1802,8 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
return topicPSInfoManager;
}
- public BrokerInfoHolder getBrokerHolder() {
- return brokerHolder;
+ public BrokerAbnHolder getBrokerAbnHolder() {
+ return brokerRunManager.getBrokerAbnHolder();
}
public ProducerInfoHolder getProducerHolder() {
@@ -2470,7 +1837,7 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
}
private void checkNodeStatus(String clientId, final StringBuilder strBuffer) throws Exception {
- if (!defaultBdbStoreService.isMaster()) {
+ if (!defMetaDataManager.isSelfMaster()) {
throw new StandbyException(strBuffer.append(masterAddInfo.getHostPortStr())
.append(" is not master now. the connecting client id is ")
.append(clientId).toString());
@@ -2513,22 +1880,6 @@ public class TMaster extends HasThread implements MasterService, Stoppable {
}
}
- private class ReleaseBroker extends AbstractReleaseRunner {
- @Override
- void run(String arg) {
- int brokerId = Integer.parseInt(arg);
- BrokerInfo broker = brokerHolder.removeBroker(brokerId);
- if (broker != null) {
- List<TopicInfo> topicInfoList =
- topicPSInfoManager.getBrokerPubInfoList(broker);
- if (topicInfoList != null) {
- updateTopicsInternal(broker, topicInfoList, EventType.DISCONNECT);
- }
- defaultBrokerConfManager.resetBrokerReportInfo(broker.getBrokerId());
- }
- }
- }
-
private class ReleaseProducer extends AbstractReleaseRunner {
@Override
void run(String clientId) {
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/balance/DefaultLoadBalancer.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/balance/DefaultLoadBalancer.java
index e1a554e..1873267 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/balance/DefaultLoadBalancer.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/balance/DefaultLoadBalancer.java
@@ -35,11 +35,10 @@ import org.apache.inlong.tubemq.corebase.cluster.ConsumerInfo;
import org.apache.inlong.tubemq.corebase.cluster.Partition;
import org.apache.inlong.tubemq.server.common.offsetstorage.OffsetStorage;
import org.apache.inlong.tubemq.server.common.offsetstorage.OffsetStorageInfo;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbBrokerConfEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbConsumeGroupSettingEntity;
-import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerConfManager;
-import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerInfoHolder;
-import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.TopicPSInfoManager;
+import org.apache.inlong.tubemq.server.master.metamanage.MetaDataManager;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.BrokerConfEntity;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.GroupResCtrlEntity;
+import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerRunManager;
import org.apache.inlong.tubemq.server.master.nodemanage.nodeconsumer.ConsumerBandInfo;
import org.apache.inlong.tubemq.server.master.nodemanage.nodeconsumer.ConsumerInfoHolder;
import org.apache.inlong.tubemq.server.master.nodemanage.nodeconsumer.NodeRebInfo;
@@ -47,6 +46,7 @@ import org.apache.inlong.tubemq.server.master.nodemanage.nodeconsumer.RebProcess
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+
/* Load balance class for server side load balance, (partition size) mod (consumer size) */
public class DefaultLoadBalancer implements LoadBalancer {
private static final Logger logger = LoggerFactory.getLogger(LoadBalancer.class);
@@ -61,10 +61,9 @@ public class DefaultLoadBalancer implements LoadBalancer {
*
* @param clusterState
* @param consumerHolder
- * @param brokerHolder
- * @param topicPSInfoManager
+ * @param brokerRunManager
* @param groupSet
- * @param brokerConfManager
+ * @param metaDataManager
* @param defAllowBClientRate
* @param strBuffer
* @return
@@ -73,12 +72,11 @@ public class DefaultLoadBalancer implements LoadBalancer {
public Map<String, Map<String, List<Partition>>> balanceCluster(
Map<String, Map<String, Map<String, Partition>>> clusterState,
ConsumerInfoHolder consumerHolder,
- BrokerInfoHolder brokerHolder,
- TopicPSInfoManager topicPSInfoManager,
+ BrokerRunManager brokerRunManager,
List<String> groupSet,
- BrokerConfManager brokerConfManager,
+ MetaDataManager metaDataManager,
int defAllowBClientRate,
- final StringBuilder strBuffer) {
+ StringBuilder strBuffer) {
// #lizard forgives
// load balance according to group
Map<String/* consumer */,
@@ -122,14 +120,15 @@ public class DefaultLoadBalancer implements LoadBalancer {
if (!consumerBandInfo.isBandConsume()
&& consumerBandInfo.getRebalanceCheckStatus() <= 0) {
// check if current client meet minimal requirements
- BdbConsumeGroupSettingEntity offsetResetGroupEntity =
- brokerConfManager.getBdbConsumeGroupSetting(group);
+ GroupResCtrlEntity offsetResetGroupEntity =
+ metaDataManager.confGetGroupResCtrlConf(group);
int confAllowBClientRate = (offsetResetGroupEntity != null
&& offsetResetGroupEntity.getAllowedBrokerClientRate() > 0)
? offsetResetGroupEntity.getAllowedBrokerClientRate() : -2;
int allowRate = confAllowBClientRate > 0
? confAllowBClientRate : defAllowBClientRate;
- int maxBrokerCount = topicPSInfoManager.getTopicMaxBrokerCount(topicSet);
+ int maxBrokerCount =
+ brokerRunManager.getSubTopicMaxBrokerCount(topicSet);
int curBClientRate = (int) Math.floor(maxBrokerCount / newConsumerList.size());
if (curBClientRate > allowRate) {
int minClientCnt = maxBrokerCount / allowRate;
@@ -160,7 +159,8 @@ public class DefaultLoadBalancer implements LoadBalancer {
}
}
List<ConsumerInfo> newConsumerList2 = new ArrayList<>();
- Map<String, Partition> psMap = topicPSInfoManager.getPartitionMap(topicSet);
+ Map<String, Partition> partMap =
+ brokerRunManager.getSubBrokerAcceptSubParts(topicSet);
Map<String, NodeRebInfo> rebProcessInfoMap = consumerBandInfo.getRebalanceMap();
for (ConsumerInfo consumer : newConsumerList) {
Map<String, List<Partition>> partitions = new HashMap<>();
@@ -188,7 +188,7 @@ public class DefaultLoadBalancer implements LoadBalancer {
Map<String, Partition> partitionMap = entry.getValue();
if (partitionMap != null && !partitionMap.isEmpty()) {
for (Partition partition : partitionMap.values()) {
- Partition curPart = psMap.remove(partition.getPartitionKey());
+ Partition curPart = partMap.remove(partition.getPartitionKey());
if (curPart != null) {
ps.add(curPart);
}
@@ -198,10 +198,10 @@ public class DefaultLoadBalancer implements LoadBalancer {
}
}
// random allocate
- if (psMap.size() > 0) {
+ if (partMap.size() > 0) {
onlineOfflineGroupSet.add(group);
if (!newConsumerList2.isEmpty()) {
- this.randomAssign(psMap, newConsumerList2,
+ this.randomAssign(partMap, newConsumerList2,
finalSubInfoMap, clusterState, rebProcessInfo.needProcessList);
}
}
@@ -227,7 +227,7 @@ public class DefaultLoadBalancer implements LoadBalancer {
}
if (groupsNeedToBalance.size() > 0) {
finalSubInfoMap =
- balance(finalSubInfoMap, consumerHolder, topicPSInfoManager,
+ balance(finalSubInfoMap, consumerHolder, brokerRunManager,
groupsNeedToBalance, clusterState, rejGroupClientINfoMap);
}
if (!rejGroupClientINfoMap.isEmpty()) {
@@ -244,7 +244,7 @@ public class DefaultLoadBalancer implements LoadBalancer {
private Map<String, Map<String, List<Partition>>> balance(
Map<String, Map<String, List<Partition>>> clusterState,
ConsumerInfoHolder consumerHolder,
- TopicPSInfoManager topicPSInfoManager,
+ BrokerRunManager brokerRunManager,
List<String> groupSet,
Map<String, Map<String, Map<String, Partition>>> oldClusterState,
Map<String, RebProcessInfo> rejGroupClientInfoMap) {
@@ -287,7 +287,8 @@ public class DefaultLoadBalancer implements LoadBalancer {
}
// sort consumer and partitions, then mod
Set<String> topics = consumerBandInfo.getTopicSet();
- Map<String, Partition> psPartMap = topicPSInfoManager.getPartitionMap(topics);
+ Map<String, Partition> psPartMap =
+ brokerRunManager.getSubBrokerAcceptSubParts(topics);
int min = psPartMap.size() / consumerList.size();
int max = psPartMap.size() % consumerList.size() == 0 ? min : min + 1;
int serverNumToLoadMax = psPartMap.size() % consumerList.size();
@@ -482,9 +483,9 @@ public class DefaultLoadBalancer implements LoadBalancer {
* Assign consumer partitions
*
* @param consumerHolder
- * @param topicPSInfoManager
+ * @param brokerRunManager
* @param groupSet
- * @param brokerConfManager
+ * @param metaDataManager
* @param defAllowBClientRate
* @param strBuffer
* @return
@@ -492,11 +493,11 @@ public class DefaultLoadBalancer implements LoadBalancer {
@Override
public Map<String, Map<String, List<Partition>>> bukAssign(
ConsumerInfoHolder consumerHolder,
- TopicPSInfoManager topicPSInfoManager,
+ BrokerRunManager brokerRunManager,
List<String> groupSet,
- BrokerConfManager brokerConfManager,
+ MetaDataManager metaDataManager,
int defAllowBClientRate,
- final StringBuilder strBuffer) {
+ StringBuilder strBuffer) {
// #lizard forgives
// regular consumer allocate operation
Map<String, Map<String, List<Partition>>> finalSubInfoMap =
@@ -522,14 +523,15 @@ public class DefaultLoadBalancer implements LoadBalancer {
}
// check if current client meet minimal requirements
Set<String> topicSet = consumerBandInfo.getTopicSet();
- BdbConsumeGroupSettingEntity offsetResetGroupEntity =
- brokerConfManager.getBdbConsumeGroupSetting(group);
+ GroupResCtrlEntity offsetResetGroupEntity =
+ metaDataManager.confGetGroupResCtrlConf(group);
int confAllowBClientRate = (offsetResetGroupEntity != null
&& offsetResetGroupEntity.getAllowedBrokerClientRate() > 0)
? offsetResetGroupEntity.getAllowedBrokerClientRate() : -2;
int allowRate = confAllowBClientRate > 0
? confAllowBClientRate : defAllowBClientRate;
- int maxBrokerCount = topicPSInfoManager.getTopicMaxBrokerCount(topicSet);
+ int maxBrokerCount =
+ brokerRunManager.getSubTopicMaxBrokerCount(topicSet);
int curBClientRate = (int) Math.floor(maxBrokerCount / consumerList.size());
if (curBClientRate > allowRate) {
int minClientCnt = maxBrokerCount / allowRate;
@@ -555,7 +557,8 @@ public class DefaultLoadBalancer implements LoadBalancer {
// sort and mod
Collections.sort(consumerList);
for (String topic : topicSet) {
- List<Partition> partPubList = topicPSInfoManager.getPartitionList(topic);
+ List<Partition> partPubList =
+ brokerRunManager.getSubBrokerAcceptSubParts(topic);
Collections.sort(partPubList);
int partsPerConsumer = partPubList.size() / consumerList.size();
int consumersWithExtraPart = partPubList.size() % consumerList.size();
@@ -588,20 +591,20 @@ public class DefaultLoadBalancer implements LoadBalancer {
* Reset
*
* @param consumerHolder
- * @param topicPSInfoManager
+ * @param brokerRunManager
* @param groupSet
* @param zkOffsetStorage
- * @param defaultBrokerConfManager
+ * @param metaDataManager
* @param strBuffer
* @return
*/
@Override
public Map<String, Map<String, Map<String, Partition>>> resetBukAssign(
- ConsumerInfoHolder consumerHolder, TopicPSInfoManager topicPSInfoManager,
+ ConsumerInfoHolder consumerHolder, BrokerRunManager brokerRunManager,
List<String> groupSet, OffsetStorage zkOffsetStorage,
- BrokerConfManager defaultBrokerConfManager, final StringBuilder strBuffer) {
+ MetaDataManager metaDataManager, final StringBuilder strBuffer) {
return inReBalanceCluster(false, consumerHolder,
- topicPSInfoManager, groupSet, zkOffsetStorage, defaultBrokerConfManager, strBuffer);
+ brokerRunManager, groupSet, zkOffsetStorage, metaDataManager, strBuffer);
}
/**
@@ -609,32 +612,32 @@ public class DefaultLoadBalancer implements LoadBalancer {
*
* @param clusterState
* @param consumerHolder
- * @param topicPSInfoManager
+ * @param brokerRunManager
* @param groupSet
* @param zkOffsetStorage
- * @param defaultBrokerConfManager
+ * @param metaDataManager
* @param strBuffer
* @return
*/
@Override
public Map<String, Map<String, Map<String, Partition>>> resetBalanceCluster(
Map<String, Map<String, Map<String, Partition>>> clusterState,
- ConsumerInfoHolder consumerHolder, TopicPSInfoManager topicPSInfoManager,
+ ConsumerInfoHolder consumerHolder, BrokerRunManager brokerRunManager,
List<String> groupSet, OffsetStorage zkOffsetStorage,
- BrokerConfManager defaultBrokerConfManager, final StringBuilder strBuffer) {
+ MetaDataManager metaDataManager, StringBuilder strBuffer) {
return inReBalanceCluster(true, consumerHolder,
- topicPSInfoManager, groupSet, zkOffsetStorage, defaultBrokerConfManager, strBuffer);
+ brokerRunManager, groupSet, zkOffsetStorage, metaDataManager, strBuffer);
}
// #lizard forgives
private Map<String, Map<String, Map<String, Partition>>> inReBalanceCluster(
boolean isResetRebalance,
ConsumerInfoHolder consumerHolder,
- TopicPSInfoManager topicPSInfoManager,
+ BrokerRunManager brokerRunManager,
List<String> groupSet,
OffsetStorage zkOffsetStorage,
- BrokerConfManager defaultBrokerConfManager,
- final StringBuilder strBuffer) {
+ MetaDataManager metaDataManager,
+ StringBuilder strBuffer) {
// band consume reset offset
Map<String, Map<String, Map<String, Partition>>> finalSubInfoMap =
new HashMap<>();
@@ -687,12 +690,8 @@ public class DefaultLoadBalancer implements LoadBalancer {
// actual reset offset
Map<String, Long> partsOffsetMap = consumerBandInfo.getPartOffsetMap();
List<OffsetStorageInfo> offsetInfoList = new ArrayList<>();
- Set<Partition> partPubList =
- topicPSInfoManager.getPartitions(consumerBandInfo.getTopicSet());
- Map<String, Partition> partitionMap = new HashMap<>();
- for (Partition partition : partPubList) {
- partitionMap.put(partition.getPartitionKey(), partition);
- }
+ Map<String, Partition> partitionMap =
+ brokerRunManager.getSubBrokerAcceptSubParts(consumerBandInfo.getTopicSet());
for (Entry<String, String> entry : partsConsumerMap.entrySet()) {
Partition foundPart = partitionMap.get(entry.getKey());
if (foundPart != null) {
@@ -718,12 +717,12 @@ public class DefaultLoadBalancer implements LoadBalancer {
partitionMap.remove(entry.getKey());
} else {
String[] partitionKeyItems = entry.getKey().split(TokenConstants.ATTR_SEP);
- BdbBrokerConfEntity bdbBrokerConfEntity = defaultBrokerConfManager
- .getBrokerDefaultConfigStoreInfo(Integer.parseInt(partitionKeyItems[0]));
- if (bdbBrokerConfEntity != null) {
+ BrokerConfEntity brokerConfEntity =
+ metaDataManager.getBrokerConfByBrokerId(Integer.parseInt(partitionKeyItems[0]));
+ if (brokerConfEntity != null) {
if (partsOffsetMap.get(entry.getKey()) != null) {
offsetInfoList.add(new OffsetStorageInfo(partitionKeyItems[1],
- bdbBrokerConfEntity.getBrokerId(),
+ brokerConfEntity.getBrokerId(),
Integer.parseInt(partitionKeyItems[2]),
partsOffsetMap.get(entry.getKey()), 0));
}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/balance/LoadBalancer.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/balance/LoadBalancer.java
index 207facb..f6ee9a6 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/balance/LoadBalancer.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/balance/LoadBalancer.java
@@ -22,9 +22,8 @@ import java.util.Map;
import org.apache.inlong.tubemq.corebase.cluster.ConsumerInfo;
import org.apache.inlong.tubemq.corebase.cluster.Partition;
import org.apache.inlong.tubemq.server.common.offsetstorage.OffsetStorage;
-import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerConfManager;
-import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerInfoHolder;
-import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.TopicPSInfoManager;
+import org.apache.inlong.tubemq.server.master.metamanage.MetaDataManager;
+import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerRunManager;
import org.apache.inlong.tubemq.server.master.nodemanage.nodeconsumer.ConsumerInfoHolder;
@@ -33,34 +32,33 @@ public interface LoadBalancer {
Map<String, Map<String, List<Partition>>> balanceCluster(
Map<String, Map<String, Map<String, Partition>>> clusterState,
ConsumerInfoHolder consumerHolder,
- BrokerInfoHolder brokerHolder,
- TopicPSInfoManager topicPSInfoManager,
+ BrokerRunManager brokerRunManager,
List<String> groups,
- BrokerConfManager brokerConfManager,
+ MetaDataManager metaDataManager,
int defAllowBClientRate,
StringBuilder sBuilder);
Map<String, Map<String, Map<String, Partition>>> resetBalanceCluster(
Map<String, Map<String, Map<String, Partition>>> clusterState,
ConsumerInfoHolder consumerHolder,
- TopicPSInfoManager topicPSInfoManager,
+ BrokerRunManager brokerRunManager,
List<String> groups,
OffsetStorage zkOffsetStorage,
- BrokerConfManager defaultBrokerConfManager,
- StringBuilder sBuilder);
+ MetaDataManager metaDataManager,
+ final StringBuilder sBuilder);
Map<String, Map<String, List<Partition>>> bukAssign(ConsumerInfoHolder consumerHolder,
- TopicPSInfoManager topicPSInfoManager,
+ BrokerRunManager brokerRunManager,
List<String> groups,
- BrokerConfManager brokerConfManager,
+ MetaDataManager metaDataManager,
int defAllowBClientRate,
StringBuilder sBuilder);
Map<String, Map<String, Map<String, Partition>>> resetBukAssign(ConsumerInfoHolder consumerHolder,
- TopicPSInfoManager topicPSInfoManager,
+ BrokerRunManager brokerRunManager,
List<String> groups,
OffsetStorage zkOffsetStorage,
- BrokerConfManager defaultBrokerConfManager,
+ MetaDataManager metaDataManager,
StringBuilder sBuilder);
Map<String, List<Partition>> roundRobinAssignment(List<Partition> partitions,
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/BdbStoreService.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/BdbStoreService.java
deleted file mode 100644
index 8fc9c9f..0000000
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/BdbStoreService.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.tubemq.server.master.bdbstore;
-
-import java.net.InetSocketAddress;
-import java.util.concurrent.ConcurrentHashMap;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbBlackGroupEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbBrokerConfEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbClusterSettingEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbConsumeGroupSettingEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbConsumerGroupEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbGroupFilterCondEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbGroupFlowCtrlEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbTopicAuthControlEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbTopicConfEntity;
-
-
-public interface BdbStoreService {
-
- void cleanData();
-
- boolean isMaster();
-
- long getMasterStartTime();
-
- InetSocketAddress getMasterAddress();
-
- ConcurrentHashMap<String/* nodeName */, MasterNodeInfo> getMasterGroupNodeInfo();
-
- boolean putBdbBrokerConfEntity(BdbBrokerConfEntity bdbBrokerConfEntity, boolean isNew);
-
- boolean delBdbBrokerConfEntity(int brokerId);
-
- ConcurrentHashMap<Integer/* brokerId */, BdbBrokerConfEntity> getBrokerConfigMap();
-
- boolean putBdbTopicConfEntity(BdbTopicConfEntity bdbTopicConfEntity, boolean isNew);
-
- boolean delBdbTopicConfEntity(String recordKey, String topicName);
-
- ConcurrentHashMap<Integer/* brokerId */,
- ConcurrentHashMap<String/* topicName */, BdbTopicConfEntity>> getBrokerTopicEntityMap();
-
- boolean putBdbTopicAuthControlEntity(BdbTopicAuthControlEntity bdbTopicAuthControlEntity,
- boolean isNew);
-
- boolean delBdbTopicAuthControlEntity(String topicName);
-
- ConcurrentHashMap<String, BdbTopicAuthControlEntity> getTopicAuthControlMap();
-
- boolean putBdbConsumerGroupConfEntity(BdbConsumerGroupEntity bdbConsumerGroupEntity,
- boolean isNew);
-
- boolean delBdbConsumerGroupEntity(String recordKey);
-
- ConcurrentHashMap<String,
- ConcurrentHashMap<String, BdbConsumerGroupEntity>> getConsumerGroupNameAccControlMap();
-
- MasterGroupStatus getMasterGroupStatus(boolean isFromHeartbeat);
-
- boolean isPrimaryNodeActive();
-
- void transferMaster() throws Exception;
-
- boolean putBdbBlackGroupConfEntity(BdbBlackGroupEntity bdbBlackGroupEntity, boolean isNew);
-
- boolean delBdbBlackGroupEntity(String recordKey);
-
- ConcurrentHashMap<String,
- ConcurrentHashMap<String, BdbBlackGroupEntity>> getBlackGroupNameAccControlMap();
-
- boolean delBdbGroupFilterCondEntity(String recordKey);
-
- boolean putBdbGroupFilterCondConfEntity(BdbGroupFilterCondEntity bdbGroupFilterCondEntity,
- boolean isNew);
-
- ConcurrentHashMap<String,
- ConcurrentHashMap<String, BdbGroupFilterCondEntity>> getGroupFilterCondAccControlMap();
-
- boolean delBdbGroupFlowCtrlStoreEntity(String groupName);
-
- boolean putBdbGroupFlowCtrlConfEntity(BdbGroupFlowCtrlEntity bdbGroupFlowCtrlEntity,
- boolean isNew);
-
- ConcurrentHashMap<String, BdbGroupFlowCtrlEntity> getGroupFlowCtrlMap();
-
- boolean delBdbConsumeGroupSettingEntity(String consumeGroupName);
-
- boolean putBdbConsumeGroupSettingEntity(BdbConsumeGroupSettingEntity offsetResetGroupEntity,
- boolean isNew);
-
- ConcurrentHashMap<String, BdbConsumeGroupSettingEntity> getConsumeGroupSettingMap();
-
- boolean putBdbClusterConfEntity(BdbClusterSettingEntity clusterConfEntity, boolean isNew);
-
- boolean delBdbClusterConfEntity();
-
- ConcurrentHashMap<String, BdbClusterSettingEntity> getClusterDefSettingMap();
-}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/DefaultBdbStoreService.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/DefaultBdbStoreService.java
deleted file mode 100644
index ccf264b..0000000
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/DefaultBdbStoreService.java
+++ /dev/null
@@ -1,1560 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.inlong.tubemq.server.master.bdbstore;
-
-import com.sleepycat.je.DatabaseException;
-import com.sleepycat.je.Durability;
-import com.sleepycat.je.EnvironmentConfig;
-import com.sleepycat.je.EnvironmentFailureException;
-import com.sleepycat.je.rep.InsufficientLogException;
-import com.sleepycat.je.rep.NetworkRestore;
-import com.sleepycat.je.rep.NetworkRestoreConfig;
-import com.sleepycat.je.rep.NodeState;
-import com.sleepycat.je.rep.ReplicatedEnvironment;
-import com.sleepycat.je.rep.ReplicationConfig;
-import com.sleepycat.je.rep.ReplicationGroup;
-import com.sleepycat.je.rep.ReplicationMutableConfig;
-import com.sleepycat.je.rep.ReplicationNode;
-import com.sleepycat.je.rep.StateChangeEvent;
-import com.sleepycat.je.rep.StateChangeListener;
-import com.sleepycat.je.rep.TimeConsistencyPolicy;
-import com.sleepycat.je.rep.UnknownMasterException;
-import com.sleepycat.je.rep.util.ReplicationGroupAdmin;
-import com.sleepycat.je.rep.utilint.ServiceDispatcher;
-import com.sleepycat.persist.EntityCursor;
-import com.sleepycat.persist.EntityStore;
-import com.sleepycat.persist.PrimaryIndex;
-import com.sleepycat.persist.StoreConfig;
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.inlong.tubemq.corebase.TokenConstants;
-import org.apache.inlong.tubemq.corebase.utils.TStringUtils;
-import org.apache.inlong.tubemq.server.Server;
-import org.apache.inlong.tubemq.server.common.TServerConstants;
-import org.apache.inlong.tubemq.server.common.fileconfig.MasterReplicationConfig;
-import org.apache.inlong.tubemq.server.master.MasterConfig;
-import org.apache.inlong.tubemq.server.master.TMaster;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbBlackGroupEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbBrokerConfEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbClusterSettingEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbConsumeGroupSettingEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbConsumerGroupEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbGroupFilterCondEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbGroupFlowCtrlEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbTopicAuthControlEntity;
-import org.apache.inlong.tubemq.server.master.bdbstore.bdbentitys.BdbTopicConfEntity;
-import org.apache.inlong.tubemq.server.master.utils.BdbStoreSamplePrint;
-import org.apache.inlong.tubemq.server.master.web.model.ClusterGroupVO;
-import org.apache.inlong.tubemq.server.master.web.model.ClusterNodeVO;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * Bdb store service
- * like a local database manager, according to database table name, store instance, primary key, memory cache
- * organize table structure
- */
-public class DefaultBdbStoreService implements BdbStoreService, Server {
- private static final Logger logger = LoggerFactory.getLogger(DefaultBdbStoreService.class);
-
- private static final String BDB_CLUSTER_SETTING_STORE_NAME = "bdbClusterSetting";
- private static final String BDB_TOPIC_CONFIG_STORE_NAME = "bdbTopicConfig";
- private static final String BDB_BROKER_CONFIG_STORE_NAME = "bdbBrokerConfig";
- private static final String BDB_CONSUMER_GROUP_STORE_NAME = "bdbConsumerGroup";
- private static final String BDB_TOPIC_AUTH_CONTROL_STORE_NAME = "bdbTopicAuthControl";
- private static final String BDB_BLACK_GROUP_STORE_NAME = "bdbBlackGroup";
- private static final String BDB_GROUP_FILTER_COND_STORE_NAME = "bdbGroupFilterCond";
- private static final String BDB_GROUP_FLOW_CONTROL_STORE_NAME = "bdbGroupFlowCtrlCfg";
- private static final String BDB_CONSUME_GROUP_SETTING_STORE_NAME = "bdbConsumeGroupSetting";
- private static final int REP_HANDLE_RETRY_MAX = 1;
- private final TMaster tMaster;
- // simple log print
- private final BdbStoreSamplePrint bdbStoreSamplePrint =
- new BdbStoreSamplePrint(logger);
- private Set<String> replicas4Transfer = new HashSet<>();
- private String masterNodeName;
- private int connectNodeFailCount = 0;
- private long masterStartTime = Long.MAX_VALUE;
- private File envHome;
- private EnvironmentConfig envConfig;
- private ReplicationConfig repConfig;
- private ReplicatedEnvironment repEnv;
- private ReplicationGroupAdmin replicationGroupAdmin;
- private StoreConfig storeConfig = new StoreConfig();
- // broker config store
- private EntityStore brokerConfStore;
- private PrimaryIndex<Integer/* brokerId */, BdbBrokerConfEntity> brokerConfIndex;
- private ConcurrentHashMap<Integer/* brokerId */, BdbBrokerConfEntity> brokerConfigMap =
- new ConcurrentHashMap<>();
- // topic config store
- private EntityStore topicConfStore;
- private PrimaryIndex<String/* recordKey */, BdbTopicConfEntity> topicConfIndex;
- private ConcurrentHashMap<Integer/* brokerId */, ConcurrentHashMap<String/* topicName */, BdbTopicConfEntity>>
- brokerIdTopicEntityMap = new ConcurrentHashMap<>();
- // consumer group store
- private EntityStore consumerGroupStore;
- private PrimaryIndex<String/* recordKey */, BdbConsumerGroupEntity> consumerGroupIndex;
- private ConcurrentHashMap<
- String/* topicName */,
- ConcurrentHashMap<String /* consumerGroup */, BdbConsumerGroupEntity>>
- consumerGroupTopicMap =
- new ConcurrentHashMap<>();
- //consumer group black list store
- private EntityStore blackGroupStore;
- private PrimaryIndex<String/* recordKey */, BdbBlackGroupEntity> blackGroupIndex;
- private ConcurrentHashMap<
- String/* consumerGroup */,
- ConcurrentHashMap<String /* topicName */, BdbBlackGroupEntity>>
- blackGroupTopicMap =
- new ConcurrentHashMap<>();
- // topic auth config store
- private EntityStore topicAuthControlStore;
- private PrimaryIndex<String/* recordKey */, BdbTopicAuthControlEntity> topicAuthControlIndex;
- private ConcurrentHashMap<String/* topicName */, BdbTopicAuthControlEntity> topicAuthControlMap =
- new ConcurrentHashMap<>();
- // consumer group filter condition store
- private EntityStore groupFilterCondStore;
- private PrimaryIndex<String/* recordKey */, BdbGroupFilterCondEntity> groupFilterCondIndex;
- private ConcurrentHashMap<
- String/* topicName */,
- ConcurrentHashMap<String /* consumerGroup */, BdbGroupFilterCondEntity>>
- groupFilterCondMap =
- new ConcurrentHashMap<>();
- // consumer group flow control store
- private EntityStore groupFlowCtrlStore;
- private PrimaryIndex<String/* groupName */, BdbGroupFlowCtrlEntity> groupFlowCtrlIndex;
- private ConcurrentHashMap<String/* groupName */, BdbGroupFlowCtrlEntity> groupFlowCtrlMap =
- new ConcurrentHashMap<>();
- // consumer group setting store
- private EntityStore consumeGroupSettingStore;
- private PrimaryIndex<String/* recordKey */, BdbConsumeGroupSettingEntity> consumeGroupSettingIndex;
- private ConcurrentHashMap<String/* consumeGroup */, BdbConsumeGroupSettingEntity> consumeGroupSettingMap =
- new ConcurrentHashMap<>();
- // cluster default setting store
- private EntityStore clusterDefSettingStore;
- private PrimaryIndex<String/* recordKey */, BdbClusterSettingEntity> clusterDefSettingIndex;
- private ConcurrentHashMap<String/* recordKey */, BdbClusterSettingEntity> clusterDefSettingMap =
- new ConcurrentHashMap<>();
- // service status
- private AtomicBoolean isStarted = new AtomicBoolean(false);
- // master role flag
- private volatile boolean isMaster;
- // master node list
- private ConcurrentHashMap<String/* nodeName */, MasterNodeInfo> masterNodeInfoMap =
- new ConcurrentHashMap<>();
- private String nodeHost;
- private MasterConfig masterConfig;
- private MasterReplicationConfig replicationConfig;
- private Listener listener = new Listener();
- private ExecutorService executorService = null;
-
- public DefaultBdbStoreService(MasterConfig masterConfig, final TMaster tMaster) {
- this.tMaster = tMaster;
- this.masterConfig = masterConfig;
- this.nodeHost = masterConfig.getHostName();
- this.replicationConfig = masterConfig.getReplicationConfig();
- Set<InetSocketAddress> helpers = new HashSet<>();
- for (int i = 1; i <= 3; i++) {
- InetSocketAddress helper = new InetSocketAddress(this.nodeHost, replicationConfig.getRepNodePort() + i);
- helpers.add(helper);
- }
- this.replicationGroupAdmin =
- new ReplicationGroupAdmin(this.replicationConfig.getRepGroupName(), helpers);
- }
-
- @Override
- public boolean isMaster() {
- return isMaster;
- }
-
- /**
- * Get current master address
- *
- * @return
- */
- @Override
- public InetSocketAddress getMasterAddress() {
- ReplicationGroup replicationGroup = null;
- try {
- replicationGroup = repEnv.getGroup();
- } catch (Throwable e) {
- logger.error("[BDB Error] GetMasterGroup info error", e);
- return null;
- }
- if (replicationGroup == null) {
- logger.info("[BDB Error] ReplicationGroup is null...please check the status of the group!");
- return null;
- }
- for (ReplicationNode node : replicationGroup.getNodes()) {
- try {
- NodeState nodeState = replicationGroupAdmin.getNodeState(node, 2000);
- if (nodeState != null) {
- if (nodeState.getNodeState().isMaster()) {
- return node.getSocketAddress();
- }
- }
- } catch (IOException e) {
- logger.error("[BDB Error] Get nodeState IOException error", e);
- continue;
- } catch (ServiceDispatcher.ServiceConnectFailedException e) {
- logger.error("[BDB Error] Get nodeState ServiceConnectFailedException error", e);
- continue;
- } catch (Throwable e2) {
- logger.error("[BDB Error] Get nodeState Throwable error", e2);
- continue;
- }
- }
- return null;
- }
-
- /**
- * Get replication config
- *
- * @return
- */
- public MasterReplicationConfig getReplicationConfig() {
- return this.replicationConfig;
- }
-
- /**
- * Get group address info
- *
- * @return
- */
- public ClusterGroupVO getGroupAddressStrInfo() {
- ClusterGroupVO clusterGroupVO = new ClusterGroupVO();
- ReplicationGroup replicationGroup = null;
- try {
- clusterGroupVO.setGroupStatus("Abnormal");
- clusterGroupVO.setGroupName(replicationGroupAdmin.getGroupName());
- replicationGroup = repEnv.getGroup();
- } catch (Throwable e) {
- logger.error("[BDB Error] getGroupAddressStrInfo error", e);
- return null;
- }
- if (replicationGroup != null) {
- clusterGroupVO.setPrimaryNodeActive(isPrimaryNodeActive());
- int count = 0;
- boolean hasMaster = false;
- List<ClusterNodeVO> clusterNodeVOList = new ArrayList<>();
- for (ReplicationNode node : replicationGroup.getNodes()) {
- ClusterNodeVO clusterNodeVO = new ClusterNodeVO();
- clusterNodeVO.setHostName(node.getHostName());
- clusterNodeVO.setNodeName(node.getName());
- clusterNodeVO.setPort(node.getPort());
- try {
- NodeState nodeState = replicationGroupAdmin.getNodeState(node, 2000);
- if (nodeState != null) {
- if (nodeState.getNodeState() == ReplicatedEnvironment.State.MASTER) {
- hasMaster = true;
- }
- clusterNodeVO.setNodeStatus(nodeState.getNodeState().toString());
- clusterNodeVO.setJoinTime(nodeState.getJoinTime());
- } else {
- clusterNodeVO.setNodeStatus("Not-found");
- clusterNodeVO.setJoinTime(0);
- }
- } catch (IOException e) {
- clusterNodeVO.setNodeStatus("Error");
- clusterNodeVO.setJoinTime(0);
- } catch (ServiceDispatcher.ServiceConnectFailedException e) {
- clusterNodeVO.setNodeStatus("Unconnected");
- clusterNodeVO.setJoinTime(0);
- }
- clusterNodeVOList.add(clusterNodeVO);
- }
- clusterGroupVO.setNodeData(clusterNodeVOList);
- if (hasMaster) {
- if (isPrimaryNodeActive()) {
- clusterGroupVO.setGroupStatus("Running-ReadOnly");
- } else {
- clusterGroupVO.setGroupStatus("Running-ReadWrite");
- }
- }
- }
- return clusterGroupVO;
- }
-
-
- @Override
- public void start() throws Exception {
- if (!isStarted.compareAndSet(false, true)) {
- return;
- }
- try {
- if (executorService != null) {
- executorService.shutdownNow();
- executorService = null;
- }
- executorService = Executors.newSingleThreadExecutor();
- initEnvConfig();
- repEnv = getEnvironment();
- initMetaStore();
- repEnv.setStateChangeListener(listener);
- } catch (Throwable ee) {
- logger.error("[BDB Error] start StoreManagerService failure, error", ee);
- return;
- }
- logger.info("[BDB Status] start StoreManagerService success");
- }
-
- @Override
- public void stop() throws Exception {
- // #lizard forgives
- if (!isStarted.compareAndSet(true, false)) {
- return;
- }
- logger.info("[BDB Status] Stopping StoreManagerService...");
- if (brokerConfStore != null) {
- try {
- brokerConfStore.close();
- brokerConfStore = null;
- } catch (Throwable e) {
- logger.error("[BDB Error] Close brokerConfStore error ", e);
- }
- }
- if (topicConfStore != null) {
- try {
- topicConfStore.close();
- topicConfStore = null;
- } catch (Throwable e) {
- logger.error("[BDB Error] Close topicConfigStore error ", e);
- }
- }
- if (blackGroupStore != null) {
- try {
- blackGroupStore.close();
- blackGroupStore = null;
- } catch (Throwable e) {
- logger.error("[BDB Error] Close blackGroupStore error ", e);
- }
- }
- if (consumeGroupSettingStore != null) {
- try {
- consumeGroupSettingStore.close();
- consumeGroupSettingStore = null;
- } catch (Throwable e) {
- logger.error("[BDB Error] Close consumeGroupSettingStore error ", e);
- }
- }
- if (consumerGroupStore != null) {
- try {
- consumerGroupStore.close();
- consumerGroupStore = null;
- } catch (DatabaseException e) {
- logger.error("[BDB Error] Close consumerGroupStore error ", e);
- }
- }
- if (groupFilterCondStore != null) {
- try {
- groupFilterCondStore.close();
- groupFilterCondStore = null;
- } catch (Throwable e) {
- logger.error("[BDB Error] Close groupFilterCondStore error ", e);
- }
- }
- if (topicAuthControlStore != null) {
- try {
- topicAuthControlStore.close();
- topicAuthControlStore = null;
- } catch (Throwable e) {
- logger.error("[BDB Error] Close topicFlowControlStore error ", e);
- }
- }
- if (groupFlowCtrlStore != null) {
- try {
- groupFlowCtrlStore.close();
- groupFlowCtrlStore = null;
- } catch (Throwable e) {
- logger.error("[BDB Error] Close groupFlowCtrlStore error ", e);
- }
- }
- if (clusterDefSettingStore != null) {
- try {
- clusterDefSettingStore.close();
- clusterDefSettingStore = null;
- } catch (Throwable e) {
- logger.error("[BDB Error] Close clusterDefSettingStore error ", e);
- }
- }
- /* evn close */
- if (repEnv != null) {
- try {
- repEnv.close();
- repEnv = null;
- } catch (Throwable ee) {
- logger.error("[BDB Error] Close repEnv throw error ", ee);
- }
- }
- if (executorService != null) {
- executorService.shutdownNow();
- executorService = null;
- }
- isStarted.set(false);
- logger.info("[BDB Status] Stopping StoreManagerService successfully...");
- }
-
- @Override
- public void cleanData() {
- // TODO
- }
-
- /**
- * Get master start time
- *
- * @return
- */
- @Override
- public long getMasterStartTime() {
- return masterStartTime;
- }
-
- /**
- * Get broker config bdb entity
- *
- * @param bdbBrokerConfEntity
- * @param isNew
- * @return
- */
- @Override
- public boolean putBdbBrokerConfEntity(BdbBrokerConfEntity bdbBrokerConfEntity, boolean isNew) {
- BdbBrokerConfEntity result = null;
- try {
- result = brokerConfIndex.put(bdbBrokerConfEntity);
- } catch (Throwable e) {
- logger.error("[BDB Error] PutTopicConf Error ", e);
- return false;
- }
- if (isNew) {
- return result == null;
- }
- return result != null;
- }
-
- /**
- * Delete broker config bdb entity
- *
- * @param brokerId
- * @return
- */
- @Override
- public boolean delBdbBrokerConfEntity(int brokerId) {
- try {
- brokerConfIndex.delete(brokerId);
- } catch (Throwable e) {
- logger.error("[BDB Error] delBdbBrokerConfEntity Error ", e);
- return false;
- }
- return true;
- }
-
- /**
- * Get broker config map
- *
- * @return
- */
- @Override
- public ConcurrentHashMap<Integer/* brokerId */, BdbBrokerConfEntity> getBrokerConfigMap() {
- return this.brokerConfigMap;
- }
-
-
- /**
- * Put topic config bdb entity
- *
- * @param bdbTopicConfEntity
- * @param isNew
- * @return
- */
- @Override
- public boolean putBdbTopicConfEntity(BdbTopicConfEntity bdbTopicConfEntity, boolean isNew) {
- BdbTopicConfEntity result = null;
- try {
- result = topicConfIndex.put(bdbTopicConfEntity);
- } catch (Throwable e) {
- logger.error("[BDB Error] PutTopicConf Error ", e);
- return false;
- }
- if (isNew) {
- return result == null;
- }
- return result != null;
- }
-
- /**
- * Delete topic config bdb entity
- *
- * @param recordKey
- * @param topicName
- * @return
- */
- @Override
- public boolean delBdbTopicConfEntity(String recordKey, String topicName) {
- try {
- topicConfIndex.delete(recordKey);
- } catch (Throwable e) {
- logger.error("[BDB Error] delBdbTopicConfEntity Error ", e);
- return false;
- }
- return true;
- }
-
- /**
- * Get topic entity map
- *
- * @return
- */
- @Override
- public ConcurrentHashMap<Integer, ConcurrentHashMap<String, BdbTopicConfEntity>> getBrokerTopicEntityMap() {
- return this.brokerIdTopicEntityMap;
- }
-
- /**
- * Get master group info
- *
- * @return
- */
- @Override
- public ConcurrentHashMap<String/* nodeName */, MasterNodeInfo> getMasterGroupNodeInfo() {
- getMasterGroupStatus(false);
- return this.masterNodeInfoMap;
- }
-
- /**
- * Put topic auth control bdb entity
- *
- * @param bdbTopicAuthControlEntity
- * @param isNew
- * @return
- */
- @Override
- public boolean putBdbTopicAuthControlEntity(BdbTopicAuthControlEntity bdbTopicAuthControlEntity,
- boolean isNew) {
- BdbTopicAuthControlEntity result = null;
- try {
- result = topicAuthControlIndex.put(bdbTopicAuthControlEntity);
- } catch (Throwable e) {
- logger.error("[BDB Error] PutTopicAuthControl Error ", e);
- return false;
- }
- if (isNew) {
- return result == null;
- }
- return result != null;
- }
-
- /**
- * Delete topic auth control bdb entity
- *
- * @param topicName
- * @return
- */
- @Override
- public boolean delBdbTopicAuthControlEntity(String topicName) {
- try {
- topicAuthControlIndex.delete(topicName);
- } catch (Throwable e) {
- logger.error("[BDB Error] delTopicAuthControl Error ", e);
- return false;
- }
- return true;
- }
-
- /**
- * Get topic auth control map
- *
- * @return
- */
- @Override
- public ConcurrentHashMap<String, BdbTopicAuthControlEntity> getTopicAuthControlMap() {
- return this.topicAuthControlMap;
- }
-
- /**
- * Put consumer group config bdb entity
- *
- * @param bdbConsumerGroupEntity
- * @param isNew
- * @return
- */
- @Override
- public boolean putBdbConsumerGroupConfEntity(BdbConsumerGroupEntity bdbConsumerGroupEntity,
- boolean isNew) {
- BdbConsumerGroupEntity result = null;
- try {
- result = consumerGroupIndex.put(bdbConsumerGroupEntity);
- } catch (Throwable e) {
- logger.error("[BDB Error] PutConsumerGroup Error ", e);
- return false;
- }
- if (isNew) {
- return result == null;
- }
- return result != null;
- }
-
- /**
- * Put group filter condition bdb entity
- *
- * @param bdbGroupFilterCondEntity
- * @param isNew
- * @return
- */
- @Override
- public boolean putBdbGroupFilterCondConfEntity(BdbGroupFilterCondEntity bdbGroupFilterCondEntity, boolean isNew) {
- BdbGroupFilterCondEntity result = null;
- try {
- result = groupFilterCondIndex.put(bdbGroupFilterCondEntity);
- } catch (Throwable e) {
- logger.error("[BDB Error] PutGroupFilterCond Error ", e);
- return false;
- }
- if (isNew) {
- return result == null;
- }
- return result != null;
- }
-
- /**
- * Put group flow control config bdb entity
- *
- * @param bdbGroupFlowCtrlEntity
- * @param isNew
- * @return
- */
- @Override
- public boolean putBdbGroupFlowCtrlConfEntity(BdbGroupFlowCtrlEntity bdbGroupFlowCtrlEntity, boolean isNew) {
- BdbGroupFlowCtrlEntity result = null;
- try {
- result = groupFlowCtrlIndex.put(bdbGroupFlowCtrlEntity);
- } catch (Throwable e) {
- logger.error("[BDB Error] putBdbGroupFlowCtrlConfEntity Error ", e);
- return false;
- }
- if (isNew) {
- return result == null;
- }
- return result != null;
- }
-
- /**
- * Put black group config bdb entity
- *
- * @param bdbBlackGroupEntity
- * @param isNew
- * @return
- */
- @Override
- public boolean putBdbBlackGroupConfEntity(BdbBlackGroupEntity bdbBlackGroupEntity, boolean isNew) {
- BdbBlackGroupEntity result = null;
- try {
- result = blackGroupIndex.put(bdbBlackGroupEntity);
- } catch (Throwable e) {
- logger.error("[BDB Error] PutBlackGroup Error ", e);
- return false;
- }
- if (isNew) {
- return result == null;
- }
- return result != null;
- }
-
- /**
- * Put consumer group setting bdb entity
- *
- * @param offsetResetGroupEntity
- * @param isNew
- * @return
- */
- @Override
- public boolean putBdbConsumeGroupSettingEntity(BdbConsumeGroupSettingEntity offsetResetGroupEntity, boolean isNew) {
- BdbConsumeGroupSettingEntity result = null;
- try {
- result = consumeGroupSettingIndex.put(offsetResetGroupEntity);
- } catch (Throwable e) {
- logger.error("[BDB Error] Put ConsumeGroupSetting Error ", e);
- return false;
- }
- if (isNew) {
- return result == null;
- }
- return result != null;
- }
-
- /**
- * Delete consumer group bdb entity
- *
- * @param recordKey
- * @return
- */
- @Override
- public boolean delBdbConsumerGroupEntity(String recordKey) {
- try {
- consumerGroupIndex.delete(recordKey);
- } catch (Throwable e) {
- logger.error("[BDB Error] delBdbConsumerGroupEntity Error ", e);
- return false;
- }
- return true;
- }
-
- /**
- * Delete group filter condition bdb entity
- *
- * @param recordKey
- * @return
- */
- @Override
- public boolean delBdbGroupFilterCondEntity(String recordKey) {
- try {
- groupFilterCondIndex.delete(recordKey);
- } catch (Throwable e) {
- logger.error("[BDB Error] delBdbGroupFilterCondEntity Error ", e);
- return false;
- }
- return true;
- }
-
- /**
- * Delete group flow control store entity
- *
- * @param groupName
- * @return
- */
- @Override
- public boolean delBdbGroupFlowCtrlStoreEntity(String groupName) {
- try {
- groupFlowCtrlIndex.delete(groupName);
- } catch (Throwable e) {
- logger.error("[BDB Error] delBdbGroupFlowCtrlStoreEntity Error ", e);
- return false;
- }
- return true;
- }
-
- /**
- * Delete bdb black group entity
- *
- * @param recordKey
- * @return
- */
- @Override
- public boolean delBdbBlackGroupEntity(String recordKey) {
- try {
- blackGroupIndex.delete(recordKey);
- } catch (Throwable e) {
- logger.error("[BDB Error] delBdbBlackGroupEntity Error ", e);
- return false;
- }
- return true;
- }
-
- @Override
- public boolean delBdbConsumeGroupSettingEntity(String consumeGroupName) {
- try {
- consumeGroupSettingIndex.delete(consumeGroupName);
- } catch (Throwable e) {
- logger.error("[BDB Error] delBdbConsumeGroupSettingEntity Error ", e);
- return false;
- }
- return true;
- }
-
- /**
- * Put cluster default setting bdb entity
- *
- * @param clusterConfEntity
- * @param isNew
- * @return
- */
- @Override
- public boolean putBdbClusterConfEntity(BdbClusterSettingEntity clusterConfEntity, boolean isNew) {
- BdbClusterSettingEntity result = null;
- try {
- result = clusterDefSettingIndex.put(clusterConfEntity);
- } catch (Throwable e) {
- logger.error("[BDB Error] Put ClusterConfEntity Error ", e);
- return false;
- }
- if (isNew) {
- return result == null;
- }
- return result != null;
- }
-
- @Override
- public boolean delBdbClusterConfEntity() {
- try {
- clusterDefSettingIndex.delete(TServerConstants.TOKEN_DEFAULT_CLUSTER_SETTING);
- } catch (Throwable e) {
- logger.error("[BDB Error] delBdbClusterConfEntity Error ", e);
- return false;
- }
- return true;
- }
-
- @Override
- public ConcurrentHashMap<String,
- ConcurrentHashMap<String, BdbConsumerGroupEntity>> getConsumerGroupNameAccControlMap() {
- return this.consumerGroupTopicMap;
- }
-
- @Override
- public ConcurrentHashMap<String,
- ConcurrentHashMap<String, BdbBlackGroupEntity>> getBlackGroupNameAccControlMap() {
- return this.blackGroupTopicMap;
- }
-
- @Override
- public ConcurrentHashMap<String,
- ConcurrentHashMap<String, BdbGroupFilterCondEntity>> getGroupFilterCondAccControlMap() {
- return this.groupFilterCondMap;
- }
-
- @Override
- public ConcurrentHashMap<String, BdbGroupFlowCtrlEntity> getGroupFlowCtrlMap() {
- return this.groupFlowCtrlMap;
- }
-
- @Override
- public ConcurrentHashMap<String, BdbConsumeGroupSettingEntity> getConsumeGroupSettingMap() {
- return this.consumeGroupSettingMap;
- }
-
- @Override
- public ConcurrentHashMap<String, BdbClusterSettingEntity> getClusterDefSettingMap() {
- return this.clusterDefSettingMap;
- }
-
- /**
- * Get master group status
- *
- * @param isFromHeartbeat
- * @return
- */
- @Override
- public MasterGroupStatus getMasterGroupStatus(boolean isFromHeartbeat) {
- // #lizard forgives
- if (repEnv == null) {
- return null;
- }
- ReplicationGroup replicationGroup = null;
- try {
- replicationGroup = repEnv.getGroup();
- } catch (DatabaseException e) {
- if (e instanceof EnvironmentFailureException) {
- if (isFromHeartbeat) {
- logger.error("[BDB Error] Check found EnvironmentFailureException", e);
- try {
- stop();
- start();
- replicationGroup = repEnv.getGroup();
- } catch (Throwable e1) {
- logger.error("[BDB Error] close and reopen storeManager error", e1);
- }
- } else {
- logger.error(
- "[BDB Error] Get EnvironmentFailureException error while non heartBeat request", e);
- }
- } else {
- logger.error("[BDB Error] Get replication group info error", e);
- }
- } catch (Throwable ee) {
- logger.error("[BDB Error] Get replication group throw error", ee);
- }
- if (replicationGroup == null) {
- logger.error(
- "[BDB Error] ReplicationGroup is null...please check the status of the group!");
- return null;
- }
- int activeNodes = 0;
- boolean isMasterActive = false;
- Set<String> tmp = new HashSet<>();
- for (ReplicationNode node : replicationGroup.getNodes()) {
- MasterNodeInfo masterNodeInfo =
- new MasterNodeInfo(replicationGroup.getName(),
- node.getName(), node.getHostName(), node.getPort());
- if (!masterNodeInfoMap.containsKey(masterNodeInfo.getNodeName())) {
- masterNodeInfoMap.put(masterNodeInfo.getNodeName(), masterNodeInfo);
- }
- try {
- NodeState nodeState = replicationGroupAdmin.getNodeState(node, 2000);
- if (nodeState != null) {
- if (nodeState.getNodeState().isActive()) {
- activeNodes++;
- if (nodeState.getNodeName().equals(masterNodeName)) {
- isMasterActive = true;
- masterNodeInfo.setNodeStatus(1);
- }
- }
- if (nodeState.getNodeState().isReplica()) {
- tmp.add(nodeState.getNodeName());
- replicas4Transfer = tmp;
- masterNodeInfo.setNodeStatus(0);
- }
- }
- } catch (IOException e) {
- connectNodeFailCount++;
- masterNodeInfo.setNodeStatus(-1);
- bdbStoreSamplePrint.printExceptionCaught(e, node.getHostName(), node.getName());
- continue;
- } catch (ServiceDispatcher.ServiceConnectFailedException e) {
- masterNodeInfo.setNodeStatus(-2);
- bdbStoreSamplePrint.printExceptionCaught(e, node.getHostName(), node.getName());
- continue;
- } catch (Throwable ee) {
- masterNodeInfo.setNodeStatus(-3);
- bdbStoreSamplePrint.printExceptionCaught(ee, node.getHostName(), node.getName());
- continue;
- }
- }
- MasterGroupStatus masterGroupStatus = new MasterGroupStatus(isMasterActive);
- int groupSize = replicationGroup.getElectableNodes().size();
- int majoritySize = groupSize / 2 + 1;
- if ((activeNodes >= majoritySize) && isMasterActive) {
- masterGroupStatus.setMasterGroupStatus(true, true, true);
- connectNodeFailCount = 0;
- if (isPrimaryNodeActive()) {
- repEnv.setRepMutableConfig(repEnv.getRepMutableConfig().setDesignatedPrimary(false));
- }
- }
- if (groupSize == 2 && connectNodeFailCount >= 3) {
- masterGroupStatus.setMasterGroupStatus(true, false, true);
- if (connectNodeFailCount > 1000) {
- connectNodeFailCount = 3;
- }
- if (!isPrimaryNodeActive()) {
- logger.error("[BDB Error] DesignatedPrimary happened...please check if the other member is down");
- repEnv.setRepMutableConfig(repEnv.getRepMutableConfig().setDesignatedPrimary(true));
- }
- }
- return masterGroupStatus;
- }
-
- /**
- * Check if primary node is active
- *
- * @return
- */
- @Override
- public boolean isPrimaryNodeActive() {
- if (repEnv == null) {
- return false;
- }
- ReplicationMutableConfig tmpConfig = repEnv.getRepMutableConfig();
- return tmpConfig != null && tmpConfig.getDesignatedPrimary();
- }
-
- /**
- * Transfer master role to other replica node
- *
- * @throws Exception
- */
- @Override
- public void transferMaster() throws Exception {
- if (!this.isStarted.get()) {
- throw new Exception("The BDB store StoreService is reboot now!");
- }
- if (isMaster()) {
- if (!isPrimaryNodeActive()) {
- if ((replicas4Transfer != null) && (!replicas4Transfer.isEmpty())) {
- logger.info("start transferMaster to replicas: " + replicas4Transfer);
- repEnv.transferMaster(replicas4Transfer, 5, TimeUnit.MINUTES);
- logger.info("transferMaster end...");
- } else {
- throw new Exception("The replicate nodes is empty!");
- }
- } else {
- throw new Exception("DesignatedPrimary happened...please check if the other member is down!");
- }
- } else {
- throw new Exception("Please send your request to the master Node!");
- }
- }
-
- /* initial metadata */
- private void initMetaStore() {
- brokerConfStore =
- new EntityStore(repEnv, BDB_BROKER_CONFIG_STORE_NAME, storeConfig);
- brokerConfIndex =
- brokerConfStore.getPrimaryIndex(Integer.class, BdbBrokerConfEntity.class);
- topicConfStore =
- new EntityStore(repEnv, BDB_TOPIC_CONFIG_STORE_NAME, storeConfig);
- topicConfIndex =
- topicConfStore.getPrimaryIndex(String.class, BdbTopicConfEntity.class);
- consumerGroupStore =
- new EntityStore(repEnv, BDB_CONSUMER_GROUP_STORE_NAME, storeConfig);
- consumerGroupIndex =
- consumerGroupStore.getPrimaryIndex(String.class, BdbConsumerGroupEntity.class);
- topicAuthControlStore =
- new EntityStore(repEnv, BDB_TOPIC_AUTH_CONTROL_STORE_NAME, storeConfig);
- topicAuthControlIndex =
- topicAuthControlStore.getPrimaryIndex(String.class, BdbTopicAuthControlEntity.class);
- blackGroupStore =
- new EntityStore(repEnv, BDB_BLACK_GROUP_STORE_NAME, storeConfig);
- blackGroupIndex =
- blackGroupStore.getPrimaryIndex(String.class, BdbBlackGroupEntity.class);
- groupFilterCondStore =
- new EntityStore(repEnv, BDB_GROUP_FILTER_COND_STORE_NAME, storeConfig);
- groupFilterCondIndex =
- groupFilterCondStore.getPrimaryIndex(String.class, BdbGroupFilterCondEntity.class);
- groupFlowCtrlStore =
- new EntityStore(repEnv, BDB_GROUP_FLOW_CONTROL_STORE_NAME, storeConfig);
- groupFlowCtrlIndex =
- groupFlowCtrlStore.getPrimaryIndex(String.class, BdbGroupFlowCtrlEntity.class);
- consumeGroupSettingStore =
- new EntityStore(repEnv, BDB_CONSUME_GROUP_SETTING_STORE_NAME, storeConfig);
- consumeGroupSettingIndex =
- consumeGroupSettingStore.getPrimaryIndex(String.class, BdbConsumeGroupSettingEntity.class);
- clusterDefSettingStore =
- new EntityStore(repEnv, BDB_CLUSTER_SETTING_STORE_NAME, storeConfig);
- clusterDefSettingIndex =
- clusterDefSettingStore.getPrimaryIndex(String.class, BdbClusterSettingEntity.class);
- }
-
- /**
- * Initialize configuration for BDB-JE replication environment.
- *
- * */
- private void initEnvConfig() throws InterruptedException {
-
- //Set envHome and generate a ReplicationConfig. Note that ReplicationConfig and
- //EnvironmentConfig values could all be specified in the je.properties file, as is shown in the
- //properties file included in the example.
- repConfig = new ReplicationConfig();
- // Set consistency policy for replica.
- TimeConsistencyPolicy consistencyPolicy = new TimeConsistencyPolicy(3, TimeUnit.SECONDS,
- 3, TimeUnit.SECONDS);
- repConfig.setConsistencyPolicy(consistencyPolicy);
- // Wait up to 3 seconds for commitConsumed acknowledgments.
- repConfig.setReplicaAckTimeout(3, TimeUnit.SECONDS);
- repConfig.setConfigParam(ReplicationConfig.TXN_ROLLBACK_LIMIT, "1000");
- repConfig.setGroupName(replicationConfig.getRepGroupName());
- repConfig.setNodeName(replicationConfig.getRepNodeName());
- repConfig.setNodeHostPort(this.nodeHost + TokenConstants.ATTR_SEP
- + replicationConfig.getRepNodePort());
- if (TStringUtils.isNotEmpty(replicationConfig.getRepHelperHost())) {
- logger.info("ADD HELP HOST");
- repConfig.setHelperHosts(replicationConfig.getRepHelperHost());
- }
-
- //A replicated environment must be opened with transactions enabled. Environments on a master
- //must be read/write, while environments on a client can be read/write or read/only. Since the
- //master's identity may change, it's most convenient to open the environment in the default
- //read/write mode. All write operations will be refused on the client though.
- envConfig = new EnvironmentConfig();
- envConfig.setTransactional(true);
- Durability durability =
- new Durability(replicationConfig.getMetaLocalSyncPolicy(), replicationConfig.getMetaReplicaSyncPolicy(),
- replicationConfig.getRepReplicaAckPolicy());
- envConfig.setDurability(durability);
- envConfig.setAllowCreate(true);
-
- envHome = new File(masterConfig.getMetaDataPath());
-
- // An Entity Store in a replicated environment must be transactional.
- storeConfig.setTransactional(true);
- // Note that both Master and Replica open the store for write.
- storeConfig.setReadOnly(false);
- storeConfig.setAllowCreate(true);
- }
-
- /**
- * Creates the replicated environment handle and returns it. It will retry indefinitely if a
- * master could not be established because a sufficient number of nodes were not available, or
- * there were networking issues, etc.
- *
- * @return the newly created replicated environment handle
- * @throws InterruptedException if the operation was interrupted
- */
- private ReplicatedEnvironment getEnvironment() throws InterruptedException {
- DatabaseException exception = null;
-
- //In this example we retry REP_HANDLE_RETRY_MAX times, but a production HA application may
- //retry indefinitely.
- for (int i = 0; i < REP_HANDLE_RETRY_MAX; i++) {
- try {
- return new ReplicatedEnvironment(envHome, repConfig, envConfig);
- } catch (UnknownMasterException unknownMaster) {
- exception = unknownMaster;
- //Indicates there is a group level problem: insufficient nodes for an election, network
- //connectivity issues, etc. Wait and retry to allow the problem to be resolved.
- logger.error("Master could not be established. " + "Exception message:"
- + unknownMaster.getMessage() + " Will retry after 5 seconds.");
- Thread.sleep(5 * 1000);
- continue;
- } catch (InsufficientLogException insufficientLogEx) {
- logger.error("[Restoring data please wait....] "
- + "Obtains logger files for a Replica from other members of the replication group. "
- + "A Replica may need to do so if it has been offline for some time, "
- + "and has fallen behind in its execution of the replication stream.");
- NetworkRestore restore = new NetworkRestore();
- NetworkRestoreConfig config = new NetworkRestoreConfig();
- // delete obsolete logger files.
- config.setRetainLogFiles(false);
- restore.execute(insufficientLogEx, config);
- // retry
- return new ReplicatedEnvironment(envHome, repConfig, envConfig);
- }
- }
- // Failed despite retries.
- if (exception != null) {
- throw exception;
- }
- // Don't expect to get here.
- throw new IllegalStateException("Failed despite retries");
- }
-
- private void clearCachedRunData() {
- if (tMaster != null && tMaster.getMasterTopicManager() != null) {
- tMaster.getMasterTopicManager().clearBrokerRunSyncManageData();
- }
- }
-
- private void loadBrokerConfUnits() throws Exception {
- long count = 0L;
- EntityCursor<BdbBrokerConfEntity> cursor = null;
- logger.info("loadBrokerConfUnits start...");
- try {
- cursor = brokerConfIndex.entities();
- brokerConfigMap.clear();
- StringBuilder sBuilder = logger.isDebugEnabled() ? new StringBuilder(512) : null;
- logger.debug("[loadBrokerConfUnits] Load broker default configure start:");
- for (BdbBrokerConfEntity bdbEntity : cursor) {
- if (bdbEntity == null) {
- logger.warn("[BDB Error] Found Null data while loading from brokerConfIndex!");
- continue;
- }
- BdbBrokerConfEntity tmpBdbEntity = brokerConfigMap.get(bdbEntity.getBrokerId());
- if (tmpBdbEntity == null) {
- brokerConfigMap.put(bdbEntity.getBrokerId(), bdbEntity);
- if (tMaster != null && tMaster.getMasterTopicManager() != null) {
- tMaster.getMasterTopicManager().updateBrokerMaps(bdbEntity);
- }
- }
- count++;
- if (logger.isDebugEnabled()) {
- logger.debug(bdbEntity.toJsonString(sBuilder).toString());
- sBuilder.delete(0, sBuilder.length());
- }
- }
- logger.debug("[loadBrokerConfUnits] Load broker default configure finished.");
- logger.info("[loadBrokerConfUnits] total load records are {} ", count);
- } catch (Exception e) {
- logger.error("[loadBrokerConfUnits error] ", e);
- throw e;
- } finally {
- if (cursor != null) {
- cursor.close();
- }
- }
- logger.info("loadBrokerConfUnits successfully...");
- }
-
- private void loadTopicConfUnits() throws Exception {
- long count = 0L;
- EntityCursor<BdbTopicConfEntity> cursor = null;
- logger.info("LoadTopicConfUnits start...");
- try {
- cursor = topicConfIndex.entities();
- brokerIdTopicEntityMap.clear();
- StringBuilder sBuilder = logger.isDebugEnabled() ? new StringBuilder(512) : null;
- for (BdbTopicConfEntity bdbEntity : cursor) {
- if (bdbEntity == null) {
- logger.warn("[BDB Error] Found Null data while loading from topicConfIndex!");
- continue;
- }
- ConcurrentHashMap<String/* topicName */, BdbTopicConfEntity> brokerTopicMap =
- brokerIdTopicEntityMap.get(bdbEntity.getBrokerId());
- if (brokerTopicMap == null) {
- brokerTopicMap =
- new ConcurrentHashMap<>();
- brokerIdTopicEntityMap.put(bdbEntity.getBrokerId(), brokerTopicMap);
- }
- brokerTopicMap.put(bdbEntity.getTopicName(), bdbEntity);
- count++;
- if (logger.isDebugEnabled()) {
- logger.debug(bdbEntity.toJsonString(sBuilder).toString());
- sBuilder.delete(0, sBuilder.length());
- }
- }
- logger.debug("[Load topic config] load broker topic record finished!");
- logger.info("[loadTopicConfUnits] total load records are {}", count);
- } catch (Exception e) {
- logger.error("[loadTopicConfUnits error] ", e);
- throw e;
- } finally {
- if (cursor != null) {
- cursor.close();
- }
- }
- logger.info("loadTopicConfUnits successfully...");
- }
-
- private void loadConsumerGroupUnits() throws Exception {
- long count = 0L;
- EntityCursor<BdbConsumerGroupEntity> cursor = null;
- logger.info("loadConsumerGroupUnits start...");
- try {
- cursor = consumerGroupIndex.entities();
- consumerGroupTopicMap.clear();
- StringBuilder sBuilder = logger.isDebugEnabled() ? new StringBuilder(512) : null;
- logger.debug("[loadConsumerGroupUnits] Load consumer group begin:");
- for (BdbConsumerGroupEntity bdbEntity : cursor) {
- if (bdbEntity == null) {
- logger.warn("[BDB Error] Found Null data while loading from consumerGroupIndex!");
- continue;
- }
- String topicName = bdbEntity.getGroupTopicName();
- String consumerGroupName = bdbEntity.getConsumerGroupName();
- ConcurrentHashMap<String/* groupName */, BdbConsumerGroupEntity> consumerGroupMap =
- consumerGroupTopicMap.get(topicName);
- if (consumerGroupMap == null) {
- consumerGroupMap =
- new ConcurrentHashMap<>();
- consumerGroupTopicMap.put(topicName, consumerGroupMap);
- }
- consumerGroupMap.put(consumerGroupName, bdbEntity);
- count++;
- if (logger.isDebugEnabled()) {
- logger.debug(bdbEntity.toJsonString(sBuilder).toString());
- sBuilder.delete(0, sBuilder.length());
- }
- }
- logger.debug("[loadConsumerGroupUnits] Load consumer group finished!");
- logger.info("[loadConsumerGroupUnits] total load records are {}", count);
- } catch (Exception e) {
- logger.error("[loadConsumerGroupUnits error] ", e);
- throw e;
- } finally {
- if (cursor != null) {
- cursor.close();
- }
- }
- logger.info("loadConsumerGroupUnits successfully...");
- }
-
- private void loadGroupFilterCondUnits() throws Exception {
- long count = 0L;
- EntityCursor<BdbGroupFilterCondEntity> cursor = null;
- logger.info("loadGroupFilterCondUnits start...");
- try {
- cursor = groupFilterCondIndex.entities();
- groupFilterCondMap.clear();
- StringBuilder sBuilder = logger.isDebugEnabled() ? new StringBuilder(512) : null;
- logger.debug("[loadGroupFilterCondUnits] Load consumer group start:");
- for (BdbGroupFilterCondEntity bdbEntity : cursor) {
- if (bdbEntity == null) {
- logger.warn("[BDB Error] Found Null data while loading from groupFilterCondIndex!");
- continue;
- }
- String topicName = bdbEntity.getTopicName();
- String consumerGroupName = bdbEntity.getConsumerGroupName();
- ConcurrentHashMap<String, BdbGroupFilterCondEntity> filterCondMap =
- groupFilterCondMap.get(topicName);
- if (filterCondMap == null) {
- filterCondMap =
- new ConcurrentHashMap<>();
- groupFilterCondMap.put(topicName, filterCondMap);
- }
- filterCondMap.put(consumerGroupName, bdbEntity);
- count++;
- if (logger.isDebugEnabled()) {
- logger.debug(bdbEntity.toJsonString(sBuilder).toString());
- sBuilder.delete(0, sBuilder.length());
- }
- }
- logger.debug("[loadGroupFilterCondUnits] Load consumer group finished!");
- logger.info("[loadGroupFilterCondUnits] total load records are {}", count);
- } catch (Exception e) {
- logger.error("[loadGroupFilterCondUnits error] ", e);
- throw e;
- } finally {
- if (cursor != null) {
- cursor.close();
- }
- }
- logger.info("loadGroupFilterCondUnits successfully...");
- }
-
- private void loadGroupFlowCtrlUnits() throws Exception {
- long count = 0L;
- EntityCursor<BdbGroupFlowCtrlEntity> cursor = null;
- logger.info("loadGroupFlowCtrlUnits start...");
- try {
- cursor = groupFlowCtrlIndex.entities();
- groupFlowCtrlMap.clear();
- StringBuilder sBuilder = logger.isDebugEnabled() ? new StringBuilder(512) : null;
- logger.debug("[loadGroupFlowCtrlUnits] Load consumer group start:");
- for (BdbGroupFlowCtrlEntity bdbEntity : cursor) {
- if (bdbEntity == null) {
- logger.warn("[BDB Error] Found Null data while loading from groupFilterCondIndex!");
- continue;
- }
- String groupName = bdbEntity.getGroupName();
- groupFlowCtrlMap.put(groupName, bdbEntity);
- count++;
- if (logger.isDebugEnabled()) {
- logger.debug(bdbEntity.toJsonString(sBuilder).toString());
- sBuilder.delete(0, sBuilder.length());
- }
- }
- logger.debug("[loadGroupFlowCtrlUnits] Load consumer group finished!");
- logger.info("[loadGroupFlowCtrlUnits] total load records are {}", count);
- } catch (Exception e) {
- logger.error("[loadGroupFlowCtrlUnits error] ", e);
- throw e;
- } finally {
- if (cursor != null) {
- cursor.close();
- }
- }
- logger.info("loadGroupFlowCtrlUnits successfully...");
- }
-
- private void loadBlackGroupUnits() throws Exception {
- long count = 0L;
- EntityCursor<BdbBlackGroupEntity> cursor = null;
- logger.info("loadBlackGroupUnits start...");
- try {
- cursor = blackGroupIndex.entities();
- blackGroupTopicMap.clear();
- StringBuilder sBuilder = logger.isDebugEnabled() ? new StringBuilder(512) : null;
- logger.debug("[loadBlackGroupUnits] Load consumer group start:");
- for (BdbBlackGroupEntity bdbEntity : cursor) {
- if (bdbEntity == null) {
- logger.warn("[BDB Error] Found Null data while loading from blackGroupIndex!");
- continue;
- }
- String topicName = bdbEntity.getTopicName();
- String consumerGroupName = bdbEntity.getBlackGroupName();
- ConcurrentHashMap<String/* topicName */, BdbBlackGroupEntity> blackGroupMap =
- blackGroupTopicMap.get(consumerGroupName);
- if (blackGroupMap == null) {
- blackGroupMap =
- new ConcurrentHashMap<>();
- blackGroupTopicMap.put(consumerGroupName, blackGroupMap);
- }
- blackGroupMap.put(topicName, bdbEntity);
- count++;
- if (logger.isDebugEnabled()) {
- logger.debug(bdbEntity.toJsonString(sBuilder).toString());
- sBuilder.delete(0, sBuilder.length());
- }
- }
- logger.debug("[loadBlackGroupUnits] Load consumer group finished!");
- logger.info("[loadBlackGroupUnits] total load records are {}", count);
- } catch (Exception e) {
- logger.error("[loadBlackGroupUnits error] ", e);
- throw e;
- } finally {
- if (cursor != null) {
- cursor.close();
- }
- }
- logger.info("loadBlackGroupUnits successfully...");
- }
-
- private void loadTopicAuthControlUnits() throws Exception {
- long count = 0L;
- EntityCursor<BdbTopicAuthControlEntity> cursor = null;
- logger.info("loadTopicAuthControlUnits start...");
- try {
- cursor = topicAuthControlIndex.entities();
- topicAuthControlMap.clear();
- StringBuilder sBuilder = logger.isDebugEnabled() ? new StringBuilder(512) : null;
- logger.debug("[loadTopicAuthControlUnits] Load topic authorized control start:");
- for (BdbTopicAuthControlEntity bdbEntity : cursor) {
- if (bdbEntity == null) {
- logger.warn("[BDB Error] Found Null data while loading from topicAuthControlIndex!");
- continue;
- }
- String topicName = bdbEntity.getTopicName();
- BdbTopicAuthControlEntity tmpBdbEntity = topicAuthControlMap.get(topicName);
- if (tmpBdbEntity == null) {
- topicAuthControlMap.put(topicName, bdbEntity);
- }
- count++;
- if (logger.isDebugEnabled()) {
- logger.debug(bdbEntity.toJsonString(sBuilder).toString());
- sBuilder.delete(0, sBuilder.length());
- }
- }
- logger.debug("[loadTopicAuthControlUnits] Load topic authorized control finished!");
- logger.info("[loadTopicAuthControlUnits] total load records are {}", count);
- } catch (Exception e) {
- logger.error("[loadTopicAuthControlUnits error] ", e);
- throw e;
- } finally {
- if (cursor != null) {
- cursor.close();
- }
- }
- logger.info("loadTopicAuthControlUnits successfully...");
- }
-
- private void loadConsumeGroupSettingUnits() throws Exception {
- long count = 0L;
- EntityCursor<BdbConsumeGroupSettingEntity> cursor = null;
- logger.info("loadConsumeGroupSettingUnits start...");
- try {
- cursor = consumeGroupSettingIndex.entities();
- consumeGroupSettingMap.clear();
- StringBuilder sBuilder = logger.isDebugEnabled() ? new StringBuilder(512) : null;
- logger.debug("[loadConsumeGroupSettingUnits] Load consumer group begin:");
- for (BdbConsumeGroupSettingEntity bdbEntity : cursor) {
- if (bdbEntity == null) {
- logger.warn("[BDB Error] Found Null data while loading from offsetResetGroupIndex!");
- continue;
- }
- consumeGroupSettingMap.put(bdbEntity.getConsumeGroupName(), bdbEntity);
- count++;
- if (logger.isDebugEnabled()) {
- logger.debug(bdbEntity.toJsonString(sBuilder).toString());
- sBuilder.delete(0, sBuilder.length());
- }
- }
- logger.debug("[loadConsumeGroupSettingUnits] Load consumer group finished!");
- logger.info("[loadConsumeGroupSettingUnits] total load records are {}", count);
- } catch (Exception e) {
- logger.error("[loadConsumeGroupSettingUnits error] ", e);
- throw e;
- } finally {
- if (cursor != null) {
- cursor.close();
- }
- }
- logger.info("loadConsumeGroupSettingUnits successfully...");
- }
-
-
- private void loadClusterDefSettingUnits() throws Exception {
- long count = 0L;
- EntityCursor<BdbClusterSettingEntity> cursor = null;
- logger.info("loadClusterDefSettingUnits start...");
- try {
- cursor = clusterDefSettingIndex.entities();
- clusterDefSettingMap.clear();
- StringBuilder sBuilder = logger.isDebugEnabled() ? new StringBuilder(512) : null;
- logger.debug("[loadClusterDefSettingUnits] Load consumer group begin:");
- for (BdbClusterSettingEntity bdbEntity : cursor) {
- if (bdbEntity == null) {
- logger.warn("[BDB Error] Found Null data while loading from clusterDefSettingIndex!");
- continue;
- }
- clusterDefSettingMap.put(bdbEntity.getRecordKey(), bdbEntity);
- count++;
- if (logger.isDebugEnabled()) {
- logger.debug(bdbEntity.toJsonString(sBuilder).toString());
- sBuilder.delete(0, sBuilder.length());
- }
- }
- logger.debug("[loadClusterDefSettingUnits] Load consumer group finished!");
- logger.info("[loadClusterDefSettingUnits] total load records are {}", count);
- } catch (Exception e) {
- logger.error("[loadClusterDefSettingUnits error] ", e);
- throw e;
- } finally {
- if (cursor != null) {
- cursor.close();
- }
- }
- logger.info("loadClusterDefSettingUnits successfully...");
- }
-
- public class Listener implements StateChangeListener {
- @Override
- public void stateChange(StateChangeEvent stateChangeEvent) throws RuntimeException {
- if (repConfig != null) {
- logger.warn("[" + repConfig.getGroupName()
- + "Receive a group status changed event]...stateChangeEventTime:"
- + stateChangeEvent.getEventTime());
- }
- doWork(stateChangeEvent);
- }
-
- public void doWork(final StateChangeEvent stateChangeEvent) {
-
- final String currentNode = new StringBuilder(512)
- .append("GroupName:").append(repConfig.getGroupName())
- .append(",nodeName:").append(repConfig.getNodeName())
- .append(",hostName:").append(repConfig.getNodeHostPort()).toString();
- if (executorService == null) {
- logger.error("[BDB Error] Found executorService is null while doWork!");
- return;
- }
- executorService.submit(new Runnable() {
- @Override
- public void run() {
- StringBuilder sBuilder = new StringBuilder(512);
- switch (stateChangeEvent.getState()) {
- case MASTER:
- if (!isMaster) {
- try {
- clearCachedRunData();
- loadClusterDefSettingUnits();
- loadBrokerConfUnits();
- loadTopicConfUnits();
- loadGroupFlowCtrlUnits();
- loadGroupFilterCondUnits();
- loadConsumerGroupUnits();
- loadTopicAuthControlUnits();
- loadBlackGroupUnits();
- loadConsumeGroupSettingUnits();
- isMaster = true;
- masterStartTime = System.currentTimeMillis();
- masterNodeName = stateChangeEvent.getMasterNodeName();
- logger.info(sBuilder.append("[BDB Status] ")
- .append(currentNode)
- .append(" is a master.").toString());
- } catch (Throwable e) {
- isMaster = false;
- logger.error("[BDB Error] Fatal error when Reloading Info ", e);
- }
- }
- break;
- case REPLICA:
- isMaster = false;
- masterNodeName = stateChangeEvent.getMasterNodeName();
- logger.info(sBuilder.append("[BDB Status] ")
- .append(currentNode).append(" is a slave.").toString());
- break;
- default:
- isMaster = false;
- logger.info(sBuilder.append("[BDB Status] ")
- .append(currentNode).append(" is Unknown state ")
- .append(stateChangeEvent.getState().name()).toString());
- break;
- }
- }
- });
- }
- }
-
-}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbBlackGroupEntity.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbBlackGroupEntity.java
index a24ed21..ddc7bc1 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbBlackGroupEntity.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbBlackGroupEntity.java
@@ -22,8 +22,11 @@ import com.sleepycat.persist.model.PrimaryKey;
import java.io.Serializable;
import java.util.Date;
import org.apache.commons.lang.builder.ToStringBuilder;
+import org.apache.inlong.tubemq.corebase.TBaseConstants;
import org.apache.inlong.tubemq.corebase.TokenConstants;
+import org.apache.inlong.tubemq.corebase.utils.TStringUtils;
import org.apache.inlong.tubemq.server.common.utils.WebParameterUtils;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.TStoreConstants;
@Entity
@@ -52,6 +55,19 @@ public class BdbBlackGroupEntity implements Serializable {
this.createDate = createDate;
}
+ public BdbBlackGroupEntity(String topicName, String groupName,
+ String attributes, String createUser,
+ Date createDate) {
+ this.recordKey = new StringBuilder(512).append(topicName)
+ .append(TokenConstants.ATTR_SEP).append(consumerGroupName).toString();
+ this.topicName = topicName;
+ this.consumerGroupName = groupName;
+ this.attributes = attributes;
+ this.createUser = createUser;
+ this.createDate = createDate;
+ }
+
+
public String getAttributes() {
return attributes;
}
@@ -96,6 +112,35 @@ public class BdbBlackGroupEntity implements Serializable {
this.createDate = createDate;
}
+ public long getDataVerId() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_VERSION_ID);
+ if (atrVal != null) {
+ return Long.parseLong(atrVal);
+ }
+ return TBaseConstants.META_VALUE_UNDEFINED;
+ }
+
+ public void setDataVerId(long dataVerId) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_VERSION_ID,
+ String.valueOf(dataVerId));
+ }
+
+ public void setReason(String reason) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_BLK_REASON, reason);
+ }
+
+ public String getReason() {
+ return TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_BLK_REASON);
+ }
+
+
public StringBuilder toJsonString(final StringBuilder sBuilder) {
return sBuilder.append("{\"type\":\"BdbBlackGroupEntity\",")
.append("\"recordKey\":\"").append(recordKey)
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbBrokerConfEntity.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbBrokerConfEntity.java
index 8221619..fc29774 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbBrokerConfEntity.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbBrokerConfEntity.java
@@ -27,6 +27,7 @@ import org.apache.inlong.tubemq.corebase.TokenConstants;
import org.apache.inlong.tubemq.corebase.utils.TStringUtils;
import org.apache.inlong.tubemq.server.common.TServerConstants;
import org.apache.inlong.tubemq.server.common.utils.WebParameterUtils;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.TStoreConstants;
@Entity
@@ -259,17 +260,25 @@ public class BdbBrokerConfEntity implements Serializable {
public int getNumTopicStores() {
String atrVal =
TStringUtils.getAttrValFrmAttributes(this.attributes,
- TokenConstants.TOKEN_STORE_NUM);
+ TStoreConstants.TOKEN_STORE_NUM);
if (atrVal != null) {
return Integer.parseInt(atrVal);
}
return 1;
}
+ public BdbBrokerConfEntity setNumTopicStores(int numTopicStores) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_STORE_NUM,
+ String.valueOf(numTopicStores));
+ return this;
+ }
+
public int getDftMemCacheMsgCntInK() {
String atrVal =
TStringUtils.getAttrValFrmAttributes(this.attributes,
- TokenConstants.TOKEN_MCACHE_MSG_CNT);
+ TStoreConstants.TOKEN_MCACHE_MSG_CNT);
if (atrVal != null) {
return Integer.parseInt(atrVal);
}
@@ -279,14 +288,14 @@ public class BdbBrokerConfEntity implements Serializable {
public void setDftMemCacheMsgCntInK(final int memCacheMsgCntInK) {
this.attributes =
TStringUtils.setAttrValToAttributes(this.attributes,
- TokenConstants.TOKEN_MCACHE_MSG_CNT,
+ TStoreConstants.TOKEN_MCACHE_MSG_CNT,
String.valueOf(memCacheMsgCntInK));
}
public int getDftMemCacheMsgSizeInMB() {
String atrVal =
TStringUtils.getAttrValFrmAttributes(this.attributes,
- TokenConstants.TOKEN_MCACHE_MSG_SIZE);
+ TStoreConstants.TOKEN_MCACHE_MSG_SIZE);
if (atrVal != null) {
return Integer.parseInt(atrVal);
}
@@ -296,14 +305,14 @@ public class BdbBrokerConfEntity implements Serializable {
public void setDftMemCacheMsgSizeInMB(final int memCacheMsgSizeInMB) {
this.attributes =
TStringUtils.setAttrValToAttributes(this.attributes,
- TokenConstants.TOKEN_MCACHE_MSG_SIZE,
+ TStoreConstants.TOKEN_MCACHE_MSG_SIZE,
String.valueOf(memCacheMsgSizeInMB));
}
public int getDftMemCacheFlushIntvl() {
String atrVal =
TStringUtils.getAttrValFrmAttributes(this.attributes,
- TokenConstants.TOKEN_MCACHE_FLUSH_INTVL);
+ TStoreConstants.TOKEN_MCACHE_FLUSH_INTVL);
if (atrVal != null) {
return Integer.parseInt(atrVal);
}
@@ -313,14 +322,14 @@ public class BdbBrokerConfEntity implements Serializable {
public void setDftMemCacheFlushIntvl(final int memCacheFlushIntvl) {
this.attributes =
TStringUtils.setAttrValToAttributes(this.attributes,
- TokenConstants.TOKEN_MCACHE_FLUSH_INTVL,
+ TStoreConstants.TOKEN_MCACHE_FLUSH_INTVL,
String.valueOf(memCacheFlushIntvl));
}
public int getDftUnFlushDataHold() {
String atrVal =
TStringUtils.getAttrValFrmAttributes(this.attributes,
- TokenConstants.TOKEN_DATA_UNFLUSHHOLD);
+ TStoreConstants.TOKEN_DATA_UNFLUSHHOLD);
if (atrVal != null) {
return Integer.parseInt(atrVal);
}
@@ -330,14 +339,14 @@ public class BdbBrokerConfEntity implements Serializable {
public void setDftUnFlushDataHold(final int unFlushDataHold) {
this.attributes =
TStringUtils.setAttrValToAttributes(this.attributes,
- TokenConstants.TOKEN_DATA_UNFLUSHHOLD,
+ TStoreConstants.TOKEN_DATA_UNFLUSHHOLD,
String.valueOf(unFlushDataHold));
}
public int getBrokerTLSPort() {
String atrVal =
TStringUtils.getAttrValFrmAttributes(this.attributes,
- TokenConstants.TOKEN_TLS_PORT);
+ TStoreConstants.TOKEN_TLS_PORT);
if (atrVal != null) {
return Integer.parseInt(atrVal);
}
@@ -347,10 +356,19 @@ public class BdbBrokerConfEntity implements Serializable {
public void setBrokerTLSPort(final int brokerTLSPort) {
this.attributes =
TStringUtils.setAttrValToAttributes(this.attributes,
- TokenConstants.TOKEN_TLS_PORT,
+ TStoreConstants.TOKEN_TLS_PORT,
String.valueOf(brokerTLSPort));
}
+ public int getRegionId() {
+ return regionId;
+ }
+
+ public BdbBrokerConfEntity setRegionId(int regionId) {
+ this.regionId = regionId;
+ return this;
+ }
+
public String getBrokerAddress() {
return brokerAddress;
}
@@ -434,6 +452,58 @@ public class BdbBrokerConfEntity implements Serializable {
TStringUtils.setAttrValToAttributes(this.attributes, attrKey, attrVal);
}
+ public long getDataVerId() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_VERSION_ID);
+ if (atrVal != null) {
+ return Long.parseLong(atrVal);
+ }
+ return TBaseConstants.META_VALUE_UNDEFINED;
+ }
+
+ public BdbBrokerConfEntity setDataVerId(long dataVerId) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_VERSION_ID,
+ String.valueOf(dataVerId));
+ return this;
+ }
+
+ public int getBrokerGroupId() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_BROKER_GROUP_ID);
+ if (atrVal != null) {
+ return Integer.parseInt(atrVal);
+ }
+ return TBaseConstants.META_VALUE_UNDEFINED;
+ }
+
+ public void setBrokerGroupId(long brokerGroupId) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_BROKER_GROUP_ID,
+ String.valueOf(brokerGroupId));
+ }
+
+ public int getBrokerWebPort() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_BROKER_WEBPORT);
+ if (atrVal != null) {
+ return Integer.parseInt(atrVal);
+ }
+ return TBaseConstants.META_VALUE_UNDEFINED;
+ }
+
+ public void setBrokerWebPort(int brokerWebPort) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_BROKER_WEBPORT,
+ String.valueOf(brokerWebPort));
+ }
+
private void buildStrInfo() {
StringBuilder sBuilder = new StringBuilder(512);
this.brokerAddress = sBuilder.append(this.brokerIp)
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbClusterSettingEntity.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbClusterSettingEntity.java
index 267da44..d2e60b6 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbClusterSettingEntity.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbClusterSettingEntity.java
@@ -23,7 +23,9 @@ import java.io.Serializable;
import java.util.Date;
import org.apache.commons.lang.builder.ToStringBuilder;
import org.apache.inlong.tubemq.corebase.TBaseConstants;
+import org.apache.inlong.tubemq.corebase.utils.TStringUtils;
import org.apache.inlong.tubemq.server.common.utils.WebParameterUtils;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.TStoreConstants;
/*
@@ -268,6 +270,111 @@ public class BdbClusterSettingEntity implements Serializable {
return modifyDate;
}
+ public void setDefDataPath(String dataPath) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_PATH, dataPath);
+ }
+
+ public String getDefDataPath() {
+ return TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_DATA_PATH);
+ }
+
+ public void setDefDataType(int dataType) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_TYPE, String.valueOf(dataType));
+ }
+
+ public int getDefDataType() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_TYPE);
+ if (atrVal != null) {
+ return Integer.parseInt(atrVal);
+ }
+ return TBaseConstants.META_VALUE_UNDEFINED;
+ }
+
+ public void setEnableGloFlowCtrl(Boolean enableGloFlowCtrl) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_ENABLE_FLOW_CTRL,
+ String.valueOf(enableGloFlowCtrl));
+ }
+
+ public Boolean getEnableGloFlowCtrl() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_ENABLE_FLOW_CTRL);
+ if (atrVal != null) {
+ return Boolean.parseBoolean(atrVal);
+ }
+ return null;
+ }
+
+ public void setGloFlowCtrlCnt(int flowCtrlCnt) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_FLOW_CTRL_CNT, String.valueOf(flowCtrlCnt));
+ }
+
+ public int getGloFlowCtrlCnt() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_FLOW_CTRL_CNT);
+ if (atrVal != null) {
+ return Integer.parseInt(atrVal);
+ }
+ return TBaseConstants.META_VALUE_UNDEFINED;
+ }
+
+ public void setGloFlowCtrlInfo(String flowCtrlInfo) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_FLOW_CTRL_INFO, flowCtrlInfo);
+ }
+
+ public String getGloFlowCtrlInfo() {
+ return TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_FLOW_CTRL_INFO);
+ }
+
+ public void setCreateInfo(String creater, Date createDate) {
+ if (TStringUtils.isNotBlank(creater)) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_CREATE_USER, creater);
+ }
+ if (createDate != null) {
+ String dataStr = WebParameterUtils.date2yyyyMMddHHmmss(createDate);
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_CREATE_DATE, dataStr);
+ }
+ }
+
+ public String getCreateUser() {
+ return TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_CREATE_USER);
+ }
+
+ public Date getCreateDate() {
+ String dateStr = TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_CREATE_DATE);
+ return WebParameterUtils.yyyyMMddHHmmss2date(dateStr);
+ }
+
+ public String getStrCreateDate() {
+ return TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_CREATE_DATE);
+ }
+
+ public String getStrModifyDate() {
+ return WebParameterUtils.date2yyyyMMddHHmmss(modifyDate);
+ }
+
/**
* Serialize field to json format
*
@@ -300,9 +407,11 @@ public class BdbClusterSettingEntity implements Serializable {
}
return sBuilder.append(",\"qryPriorityId\":").append(qryPriorityId)
.append(",\"attributes\":\"").append(attributes).append("\"")
+ .append(",\"createUser\":\"").append(getCreateUser()).append("\"")
+ .append(",\"createDate\":\"").append(getStrCreateDate()).append("\"")
.append(",\"modifyUser\":\"").append(modifyUser).append("\"")
.append(",\"modifyDate\":\"")
- .append(WebParameterUtils.date2yyyyMMddHHmmss(modifyDate))
+ .append(getStrModifyDate())
.append("\"}");
}
@@ -333,8 +442,10 @@ public class BdbClusterSettingEntity implements Serializable {
}
return sBuilder.append("qryPriorityId", qryPriorityId)
.append("attributes", attributes)
+ .append("createUser", getCreateUser())
+ .append("createDate", getStrCreateDate())
.append("modifyUser", modifyUser)
- .append("modifyDate", modifyDate)
+ .append("modifyDate", getStrModifyDate())
.toString();
}
}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbConsumeGroupSettingEntity.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbConsumeGroupSettingEntity.java
index 789fa17..12ad37c 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbConsumeGroupSettingEntity.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbConsumeGroupSettingEntity.java
@@ -22,7 +22,10 @@ import com.sleepycat.persist.model.PrimaryKey;
import java.io.Serializable;
import java.util.Date;
import org.apache.commons.lang.builder.ToStringBuilder;
+import org.apache.inlong.tubemq.corebase.TBaseConstants;
+import org.apache.inlong.tubemq.corebase.utils.TStringUtils;
import org.apache.inlong.tubemq.server.common.utils.WebParameterUtils;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.TStoreConstants;
@Entity
@@ -124,6 +127,23 @@ public class BdbConsumeGroupSettingEntity implements Serializable {
this.lastBindUsedDate = new Date();
}
+ public long getDataVerId() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_VERSION_ID);
+ if (atrVal != null) {
+ return Long.parseLong(atrVal);
+ }
+ return TBaseConstants.META_VALUE_UNDEFINED;
+ }
+
+ public void setDataVerId(long dataVerId) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_VERSION_ID,
+ String.valueOf(dataVerId));
+ }
+
@Override
public String toString() {
return new ToStringBuilder(this)
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbConsumerGroupEntity.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbConsumerGroupEntity.java
index a2310ca..875ed24 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbConsumerGroupEntity.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbConsumerGroupEntity.java
@@ -71,6 +71,11 @@ public class BdbConsumerGroupEntity implements Serializable {
this.topicName = topicName;
}
+ public void setGroupAndTopicName(String groupName, String topicName) {
+ this.consumerGroupName = groupName;
+ this.topicName = topicName;
+ }
+
public String getConsumerGroupName() {
return consumerGroupName;
}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbGroupFilterCondEntity.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbGroupFilterCondEntity.java
index 4533757..0fcea04 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbGroupFilterCondEntity.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbGroupFilterCondEntity.java
@@ -22,8 +22,12 @@ import com.sleepycat.persist.model.PrimaryKey;
import java.io.Serializable;
import java.util.Date;
import org.apache.commons.lang.builder.ToStringBuilder;
+import org.apache.inlong.tubemq.corebase.TBaseConstants;
import org.apache.inlong.tubemq.corebase.TokenConstants;
+import org.apache.inlong.tubemq.corebase.utils.TStringUtils;
+import org.apache.inlong.tubemq.server.common.statusdef.EnableStatus;
import org.apache.inlong.tubemq.server.common.utils.WebParameterUtils;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.TStoreConstants;
@Entity
@@ -36,6 +40,9 @@ public class BdbGroupFilterCondEntity implements Serializable {
private String consumerGroupName;
private int controlStatus = -2; // -2: undefine; 0: not started; 1:started, not limited; 2: started, limited
private String attributes;
+ // ** Based on the data compatibility consideration of the original version:
+ // the creation information in this example is the last modified information,
+ // and the modified information is the creation information
private String createUser;
private Date createDate;
@@ -44,8 +51,8 @@ public class BdbGroupFilterCondEntity implements Serializable {
}
public BdbGroupFilterCondEntity(String topicName, String consumerGroupName,
- int controlStatus, String attributes,
- String createUser, Date createDate) {
+ int controlStatus, String filterCondStr,
+ String modifyUser, Date modifyDate) {
this.recordKey =
new StringBuilder(512)
.append(topicName)
@@ -54,17 +61,75 @@ public class BdbGroupFilterCondEntity implements Serializable {
this.topicName = topicName;
this.consumerGroupName = consumerGroupName;
this.controlStatus = controlStatus;
+ setFilterCondStr(filterCondStr);
+ this.createUser = modifyUser;
+ this.createDate = modifyDate;
+ }
+
+ public BdbGroupFilterCondEntity(String topicName, String consumerGroupName,
+ int controlStatus, String filterCondStr,
+ String attributes, String modifyUser, Date modifyDate) {
+ this.recordKey =
+ new StringBuilder(512)
+ .append(topicName)
+ .append(TokenConstants.ATTR_SEP)
+ .append(consumerGroupName).toString();
+ this.topicName = topicName;
+ this.consumerGroupName = consumerGroupName;
+ this.controlStatus = controlStatus;
+ this.createUser = modifyUser;
+ this.createDate = modifyDate;
this.attributes = attributes;
- this.createUser = createUser;
- this.createDate = createDate;
+ setFilterCondStr(filterCondStr);
}
- public String getAttributes() {
- return attributes;
+ public String getFilterCondStr() {
+ if (TStringUtils.isNotBlank(attributes)
+ && attributes.contains(TokenConstants.EQ)) {
+ return TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_FILTER_COND_STR);
+ } else {
+ return attributes;
+ }
}
- public void setAttributes(String attributes) {
- this.attributes = attributes;
+ public void setFilterCondStr(String filterCondStr) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_FILTER_COND_STR, filterCondStr);
+ }
+
+ public EnableStatus getConsumeEnable() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_ENABLE_CONSUME);
+ if (atrVal != null) {
+ return EnableStatus.valueOf(Integer.parseInt(atrVal));
+ }
+ return EnableStatus.STATUS_ENABLE;
+ }
+
+ public void setConsumeEnable(EnableStatus enableConsume) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_ENABLE_CONSUME,
+ String.valueOf(enableConsume.getCode()));
+ }
+
+ public String getDisableConsumeReason() {
+ if (TStringUtils.isNotBlank(attributes)
+ && attributes.contains(TokenConstants.EQ)) {
+ return TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_BLK_REASON);
+ } else {
+ return "";
+ }
+ }
+
+ public void setDisableConsumeReason(String disableConsumeReason) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_BLK_REASON, disableConsumeReason);
}
public String getRecordKey() {
@@ -95,20 +160,84 @@ public class BdbGroupFilterCondEntity implements Serializable {
this.controlStatus = controlStatus;
}
- public String getCreateUser() {
+ public String getModifyUser() {
return createUser;
}
- public void setCreateUser(String createUser) {
- this.createUser = createUser;
+ public Date getModifyDate() {
+ return createDate;
+ }
+
+ public String getAttributes() {
+ if (TStringUtils.isNotBlank(attributes)
+ && !attributes.contains(TokenConstants.EQ)) {
+ return attributes;
+ } else {
+ return "";
+ }
+ }
+
+ public void setAttributes(String attributes) {
+ this.attributes = attributes;
+ }
+
+ public long getDataVerId() {
+ if (TStringUtils.isNotBlank(attributes)
+ && attributes.contains(TokenConstants.EQ)) {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_VERSION_ID);
+ if (atrVal != null) {
+ return Long.parseLong(atrVal);
+ }
+ }
+ return TBaseConstants.META_VALUE_UNDEFINED;
+ }
+
+ public void setDataVerId(long dataVerId) {
+ if (TStringUtils.isNotBlank(attributes)
+ && !attributes.contains(TokenConstants.EQ)) {
+ setFilterCondStr(attributes);
+ }
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_VERSION_ID,
+ String.valueOf(dataVerId));
+ }
+
+ // for
+ public void setCreateInfo(String createUser, Date createDate) {
+ if (TStringUtils.isNotBlank(createUser)) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_CREATE_USER, createUser);
+ }
+ if (createDate != null) {
+ String dataStr = WebParameterUtils.date2yyyyMMddHHmmss(createDate);
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_CREATE_DATE, dataStr);
+ }
+ }
+
+ public String getCreateUser() {
+ return TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_CREATE_USER);
}
public Date getCreateDate() {
- return createDate;
+ String dateStr = TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_CREATE_DATE);
+ return WebParameterUtils.yyyyMMddHHmmss2date(dateStr);
+ }
+
+ public String getStrModifyDate() {
+ return WebParameterUtils.date2yyyyMMddHHmmss(createDate);
}
- public void setCreateDate(Date createDate) {
- this.createDate = createDate;
+ public String getStrCreateDate() {
+ return TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_CREATE_DATE);
}
@Override
@@ -119,8 +248,10 @@ public class BdbGroupFilterCondEntity implements Serializable {
.append("consumerGroupName", consumerGroupName)
.append("controlStatus", controlStatus)
.append("attributes", attributes)
- .append("createUser", createUser)
- .append("createDate", createDate)
+ .append("createUser", getCreateUser())
+ .append("createDate", getStrCreateDate())
+ .append("modifyUser", createUser)
+ .append("modifyDate", getStrModifyDate())
.toString();
}
@@ -131,9 +262,10 @@ public class BdbGroupFilterCondEntity implements Serializable {
.append("\",\"consumerGroupName\":\"").append(consumerGroupName)
.append("\",\"filterConds\":\"").append(attributes)
.append("\",\"condStatus\":").append(controlStatus)
- .append(",\"createUser\":\"").append(createUser)
- .append("\",\"createDate\":\"")
- .append(WebParameterUtils.date2yyyyMMddHHmmss(createDate))
+ .append(",\"createUser\":\"").append(getCreateUser())
+ .append("\",\"createDate\":\"").append(getStrCreateDate())
+ .append("\",\"modifyUser\":\"").append(createUser)
+ .append("\",\"modifyDate\":\"").append(getStrModifyDate())
.append("\"}");
}
}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbGroupFlowCtrlEntity.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbGroupFlowCtrlEntity.java
index 83cb24e..0c6b066 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbGroupFlowCtrlEntity.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbGroupFlowCtrlEntity.java
@@ -26,7 +26,9 @@ import org.apache.inlong.tubemq.corebase.TBaseConstants;
import org.apache.inlong.tubemq.corebase.TokenConstants;
import org.apache.inlong.tubemq.corebase.utils.TStringUtils;
import org.apache.inlong.tubemq.server.common.TServerConstants;
+import org.apache.inlong.tubemq.server.common.statusdef.EnableStatus;
import org.apache.inlong.tubemq.server.common.utils.WebParameterUtils;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.TStoreConstants;
@Entity
@@ -41,6 +43,9 @@ public class BdbGroupFlowCtrlEntity implements Serializable {
private long ssdTranslateId = System.currentTimeMillis();
private boolean needSSDProc = false; //ssd
private String attributes; //extra attributes
+ // ** Based on the data compatibility consideration of the original version:
+ // the creation information in this example is the last modified information,
+ // and the modified information is the creation information
private String createUser; //create user
private Date createDate; //create date
@@ -52,7 +57,7 @@ public class BdbGroupFlowCtrlEntity implements Serializable {
public BdbGroupFlowCtrlEntity(final String flowCtrlInfo, final int statusId,
final int ruleCnt, final int qryPriorityId,
final String attributes, final boolean curNeedSSDProc,
- final String createUser, final Date createDate) {
+ final String modifyUser, final Date modifyDate) {
this.statusId = statusId;
this.groupName = TServerConstants.TOKEN_DEFAULT_FLOW_CONTROL;
this.serialId = System.currentTimeMillis();
@@ -61,8 +66,8 @@ public class BdbGroupFlowCtrlEntity implements Serializable {
this.ruleCnt = ruleCnt;
this.ssdTranslateId = System.currentTimeMillis();
this.needSSDProc = curNeedSSDProc;
- this.createUser = createUser;
- this.createDate = createDate;
+ this.createUser = modifyUser;
+ this.createDate = modifyDate;
this.setQryPriorityId(qryPriorityId);
}
@@ -70,16 +75,16 @@ public class BdbGroupFlowCtrlEntity implements Serializable {
public BdbGroupFlowCtrlEntity(final String groupName, final String flowCtrlInfo,
final int statusId, final int ruleCnt,
final int qryPriorityId, final String attributes,
- final boolean needSSDProc, final String createUser,
- final Date createDate) {
+ final boolean needSSDProc, final String modifyUser,
+ final Date modifyDate) {
this.groupName = groupName;
this.serialId = System.currentTimeMillis();
this.statusId = statusId;
this.flowCtrlInfo = flowCtrlInfo;
this.attributes = attributes;
this.ruleCnt = ruleCnt;
- this.createUser = createUser;
- this.createDate = createDate;
+ this.createUser = modifyUser;
+ this.createDate = modifyDate;
this.needSSDProc = needSSDProc;
this.ssdTranslateId = TBaseConstants.META_VALUE_UNDEFINED;
this.setQryPriorityId(qryPriorityId);
@@ -89,20 +94,39 @@ public class BdbGroupFlowCtrlEntity implements Serializable {
public BdbGroupFlowCtrlEntity(final String groupName, final String flowCtrlInfo,
final int statusId, final int ruleCnt,
final String attributes, final long ssdTranslateId,
- final boolean needSSDProc, final String createUser,
- final Date createDate) {
+ final boolean needSSDProc, final String modifyUser,
+ final Date modifyDate) {
this.groupName = groupName;
this.serialId = System.currentTimeMillis();
this.statusId = statusId;
this.flowCtrlInfo = flowCtrlInfo;
this.attributes = attributes;
this.ruleCnt = ruleCnt;
- this.createUser = createUser;
- this.createDate = createDate;
+ this.createUser = modifyUser;
+ this.createDate = modifyDate;
this.needSSDProc = needSSDProc;
this.ssdTranslateId = ssdTranslateId;
}
+ //Constructor
+ public BdbGroupFlowCtrlEntity(long serialId, String groupName, String flowCtrlInfo,
+ int statusId, int ruleCnt, int qryPriorityId,
+ String attributes, String modifyUser,
+ Date modifyDate) {
+ this.groupName = groupName;
+ this.serialId = serialId;
+ this.statusId = statusId;
+ this.flowCtrlInfo = flowCtrlInfo;
+ this.attributes = attributes;
+ this.ruleCnt = ruleCnt;
+ this.createUser = modifyUser;
+ this.createDate = modifyDate;
+ this.needSSDProc = false;
+ this.ssdTranslateId = TBaseConstants.META_VALUE_UNDEFINED;
+ this.setQryPriorityId(qryPriorityId);
+
+ }
+
public long getSsdTranslateId() {
return ssdTranslateId;
}
@@ -135,14 +159,10 @@ public class BdbGroupFlowCtrlEntity implements Serializable {
this.attributes = attributes;
}
- public String getCreateUser() {
+ public String getModifyUser() {
return createUser;
}
- public void setCreateUser(String createUser) {
- this.createUser = createUser;
- }
-
public boolean isNeedSSDProc() {
return needSSDProc;
}
@@ -151,14 +171,10 @@ public class BdbGroupFlowCtrlEntity implements Serializable {
this.needSSDProc = needSSDProc;
}
- public Date getCreateDate() {
+ public Date getModifyDate() {
return createDate;
}
- public void setCreateDate(Date createDate) {
- this.createDate = createDate;
- }
-
public String getFlowCtrlInfo() {
return flowCtrlInfo;
}
@@ -185,7 +201,7 @@ public class BdbGroupFlowCtrlEntity implements Serializable {
public int getQryPriorityId() {
String atrVal =
TStringUtils.getAttrValFrmAttributes(this.attributes,
- TokenConstants.TOKEN_QRY_PRIORITY_ID);
+ TStoreConstants.TOKEN_QRY_PRIORITY_ID);
if (atrVal != null) {
return Integer.parseInt(atrVal);
}
@@ -195,13 +211,76 @@ public class BdbGroupFlowCtrlEntity implements Serializable {
public void setQryPriorityId(int qryPriorityId) {
this.attributes =
TStringUtils.setAttrValToAttributes(this.attributes,
- TokenConstants.TOKEN_QRY_PRIORITY_ID,
+ TStoreConstants.TOKEN_QRY_PRIORITY_ID,
String.valueOf(qryPriorityId));
}
- public void setModifyInfo(String modifyUser, Date modifyDate) {
- this.createUser = modifyUser;
- this.createDate = modifyDate;
+ public EnableStatus getResCheckStatus() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_RES_CHECK_STATUS);
+ if (atrVal != null) {
+ return EnableStatus.valueOf(Integer.parseInt(atrVal));
+ }
+ return EnableStatus.STATUS_UNDEFINE;
+ }
+
+ public void setResCheckStatus(EnableStatus resCheckStatus) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_RES_CHECK_STATUS,
+ String.valueOf(resCheckStatus.getCode()));
+ }
+
+ public int getAllowedBrokerClientRate() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_BROKER_CLIENT_RATE);
+ if (atrVal != null) {
+ return Integer.parseInt(atrVal);
+ }
+ return TBaseConstants.META_VALUE_UNDEFINED;
+ }
+
+ public void setAllowedBrokerClientRate(int allowedBrokerClientRate) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_BROKER_CLIENT_RATE,
+ String.valueOf(allowedBrokerClientRate));
+ }
+
+ public void setCreateInfo(String createUser, Date createDate) {
+ if (TStringUtils.isNotBlank(createUser)) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_CREATE_USER, createUser);
+ }
+ if (createDate != null) {
+ String dataStr = WebParameterUtils.date2yyyyMMddHHmmss(createDate);
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_CREATE_DATE, dataStr);
+ }
+ }
+
+ public String getCreateUser() {
+ return TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_CREATE_USER);
+ }
+
+ public Date getCreateDate() {
+ String dateStr = TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_CREATE_DATE);
+ return WebParameterUtils.yyyyMMddHHmmss2date(dateStr);
+ }
+
+ public String getStrModifyDate() {
+ return WebParameterUtils.date2yyyyMMddHHmmss(createDate);
+ }
+
+ public String getStrCreateDate() {
+ return TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_CREATE_DATE);
}
@Override
@@ -215,8 +294,10 @@ public class BdbGroupFlowCtrlEntity implements Serializable {
.append("ssdTranslateId", ssdTranslateId)
.append("needSSDProc", needSSDProc)
.append("attributes", attributes)
- .append("createUser", createUser)
- .append("createDate", createDate)
+ .append("createUser", getCreateUser())
+ .append("createDate", getCreateUser())
+ .append("modifyUser", createUser)
+ .append("modifyDate", getStrModifyDate())
.toString();
}
@@ -237,9 +318,10 @@ public class BdbGroupFlowCtrlEntity implements Serializable {
.append(",\"qryPriorityId\":").append(getQryPriorityId())
.append(",\"flowCtrlInfo\":").append(flowCtrlInfo)
.append(", \"attributes\":\"").append(attributes)
- .append("\", \"createUser\":\"").append(createUser)
- .append("\",\"createDate\":\"")
- .append(WebParameterUtils.date2yyyyMMddHHmmss(createDate))
+ .append(",\"createUser\":\"").append(getCreateUser())
+ .append("\",\"createDate\":\"").append(getStrCreateDate())
+ .append("\",\"modifyUser\":\"").append(createUser)
+ .append("\",\"modifyDate\":\"").append(getStrModifyDate())
.append("\"}");
}
}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbTopicAuthControlEntity.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbTopicAuthControlEntity.java
index 093dbfb..59fb806 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbTopicAuthControlEntity.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbTopicAuthControlEntity.java
@@ -22,7 +22,10 @@ import com.sleepycat.persist.model.PrimaryKey;
import java.io.Serializable;
import java.util.Date;
import org.apache.commons.lang.builder.ToStringBuilder;
+import org.apache.inlong.tubemq.corebase.TBaseConstants;
+import org.apache.inlong.tubemq.corebase.utils.TStringUtils;
import org.apache.inlong.tubemq.server.common.utils.WebParameterUtils;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.TStoreConstants;
@Entity
@@ -32,6 +35,9 @@ public class BdbTopicAuthControlEntity implements Serializable {
@PrimaryKey
private String topicName;
private int enableAuthControl = -1; // -1 : undefine; 0:disable, 1:enable
+ // ** Based on the data compatibility consideration of the original version:
+ // the creation information in this example is the last modified information,
+ // and the modified information is the creation information
private String createUser;
private Date createDate;
private String attributes;
@@ -41,15 +47,28 @@ public class BdbTopicAuthControlEntity implements Serializable {
}
public BdbTopicAuthControlEntity(String topicName, boolean enableAuthControl,
- String createUser, Date createDate) {
+ String modifyUser, Date modifyDate) {
this.topicName = topicName;
if (enableAuthControl) {
this.enableAuthControl = 1;
} else {
this.enableAuthControl = 0;
}
- this.createUser = createUser;
- this.createDate = createDate;
+ this.createUser = modifyUser;
+ this.createDate = modifyDate;
+ }
+
+ public BdbTopicAuthControlEntity(String topicName, boolean enableAuthControl,
+ String attributes, String modifyUser, Date modifyDate) {
+ this.topicName = topicName;
+ if (enableAuthControl) {
+ this.enableAuthControl = 1;
+ } else {
+ this.enableAuthControl = 0;
+ }
+ this.attributes = attributes;
+ this.createUser = modifyUser;
+ this.createDate = modifyDate;
}
public String getAttributes() {
@@ -84,29 +103,109 @@ public class BdbTopicAuthControlEntity implements Serializable {
}
}
- public String getCreateUser() {
+ public String getModifyUser() {
return createUser;
}
- public void setCreateUser(String createUser) {
- this.createUser = createUser;
+ public Date getModifyDate() {
+ return createDate;
+ }
+
+ public long getDataVerId() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_VERSION_ID);
+ if (atrVal != null) {
+ return Long.parseLong(atrVal);
+ }
+ return TBaseConstants.META_VALUE_UNDEFINED;
+ }
+
+ public void setDataVerId(long dataVerId) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_VERSION_ID,
+ String.valueOf(dataVerId));
+ }
+
+ public int getTopicId() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_TOPICNAME_ID);
+ if (atrVal != null) {
+ return Integer.parseInt(atrVal);
+ }
+ return TBaseConstants.META_VALUE_UNDEFINED;
+ }
+
+ public void setTopicId(int topicId) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_TOPICNAME_ID,
+ String.valueOf(topicId));
+ }
+
+
+ public int getMaxMsgSize() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_MAX_MSG_SIZE);
+ if (atrVal != null) {
+ return Integer.parseInt(atrVal);
+ }
+ return TBaseConstants.META_VALUE_UNDEFINED;
+ }
+
+ public void setMaxMsgSize(int maxMsgSize) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_MAX_MSG_SIZE,
+ String.valueOf(maxMsgSize));
+ }
+
+
+ public void setCreateInfo(String createUser, Date createDate) {
+ if (TStringUtils.isNotBlank(createUser)) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_CREATE_USER, createUser);
+ }
+ if (createDate != null) {
+ String dataStr = WebParameterUtils.date2yyyyMMddHHmmss(createDate);
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_CREATE_DATE, dataStr);
+ }
+ }
+
+ public String getCreateUser() {
+ return TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_CREATE_USER);
}
public Date getCreateDate() {
- return createDate;
+ String dateStr = TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_CREATE_DATE);
+ return WebParameterUtils.yyyyMMddHHmmss2date(dateStr);
+ }
+
+ public String getStrModifyDate() {
+ return WebParameterUtils.date2yyyyMMddHHmmss(createDate);
}
- public void setCreateDate(Date createDate) {
- this.createDate = createDate;
+ public String getStrCreateDate() {
+ return TStringUtils.getAttrValFrmAttributes(
+ this.attributes, TStoreConstants.TOKEN_CREATE_DATE);
}
public StringBuilder toJsonString(final StringBuilder sBuilder) {
return sBuilder.append("{\"type\":\"BdbConsumerGroupEntity\",")
.append("\"topicName\":\"").append(topicName)
.append("\",\"enableAuthControl\":\"").append(enableAuthControl)
- .append("\",\"createUser\":\"").append(createUser)
- .append("\",\"createDate\":\"")
- .append(WebParameterUtils.date2yyyyMMddHHmmss(createDate))
+ .append("\",\"createUser\":\"").append(getCreateUser())
+ .append("\",\"createDate\":\"").append(getStrCreateDate())
+ .append("\",\"modifyUser\":\"").append(createUser)
+ .append("\",\"modifyDate\":\"").append(getStrModifyDate())
.append("\",\"attributes\":\"").append(attributes).append("\"}");
}
@@ -115,8 +214,10 @@ public class BdbTopicAuthControlEntity implements Serializable {
return new ToStringBuilder(this)
.append("topicName", topicName)
.append("enableAuthControl", enableAuthControl)
- .append("createUser", createUser)
- .append("createDate", createDate)
+ .append("createUser", getCreateUser())
+ .append("createDate", getCreateUser())
+ .append("modifyUser", createUser)
+ .append("modifyDate", getStrModifyDate())
.append("attributes", attributes)
.toString();
}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbTopicConfEntity.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbTopicConfEntity.java
index 66ab889..0c6e9eb 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbTopicConfEntity.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/bdbstore/bdbentitys/BdbTopicConfEntity.java
@@ -27,6 +27,7 @@ import org.apache.inlong.tubemq.corebase.TokenConstants;
import org.apache.inlong.tubemq.corebase.utils.TStringUtils;
import org.apache.inlong.tubemq.server.common.TServerConstants;
import org.apache.inlong.tubemq.server.common.utils.WebParameterUtils;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.TStoreConstants;
@Entity
@@ -134,7 +135,7 @@ public class BdbTopicConfEntity implements Serializable {
public void setUnflushDataHold(final int unFlushDataHold) {
this.attributes =
TStringUtils.setAttrValToAttributes(this.attributes,
- TokenConstants.TOKEN_DATA_UNFLUSHHOLD,
+ TStoreConstants.TOKEN_DATA_UNFLUSHHOLD,
String.valueOf(unFlushDataHold));
}
@@ -287,7 +288,7 @@ public class BdbTopicConfEntity implements Serializable {
public int getUnflushDataHold() {
String atrVal =
TStringUtils.getAttrValFrmAttributes(this.attributes,
- TokenConstants.TOKEN_DATA_UNFLUSHHOLD);
+ TStoreConstants.TOKEN_DATA_UNFLUSHHOLD);
if (atrVal != null) {
return Integer.parseInt(atrVal);
}
@@ -297,7 +298,7 @@ public class BdbTopicConfEntity implements Serializable {
public int getMemCacheMsgCntInK() {
String atrVal =
TStringUtils.getAttrValFrmAttributes(this.attributes,
- TokenConstants.TOKEN_MCACHE_MSG_CNT);
+ TStoreConstants.TOKEN_MCACHE_MSG_CNT);
if (atrVal != null) {
return Integer.parseInt(atrVal);
}
@@ -307,14 +308,14 @@ public class BdbTopicConfEntity implements Serializable {
public void setMemCacheMsgCntInK(final int memCacheMsgCntInK) {
this.attributes =
TStringUtils.setAttrValToAttributes(this.attributes,
- TokenConstants.TOKEN_MCACHE_MSG_CNT,
+ TStoreConstants.TOKEN_MCACHE_MSG_CNT,
String.valueOf(memCacheMsgCntInK));
}
public int getMemCacheMsgSizeInMB() {
String atrVal =
TStringUtils.getAttrValFrmAttributes(this.attributes,
- TokenConstants.TOKEN_MCACHE_MSG_SIZE);
+ TStoreConstants.TOKEN_MCACHE_MSG_SIZE);
if (atrVal != null) {
return Integer.parseInt(atrVal);
}
@@ -324,14 +325,14 @@ public class BdbTopicConfEntity implements Serializable {
public void setMemCacheMsgSizeInMB(final int memCacheMsgSizeInMB) {
this.attributes =
TStringUtils.setAttrValToAttributes(this.attributes,
- TokenConstants.TOKEN_MCACHE_MSG_SIZE,
+ TStoreConstants.TOKEN_MCACHE_MSG_SIZE,
String.valueOf(memCacheMsgSizeInMB));
}
public int getMemCacheFlushIntvl() {
String atrVal =
TStringUtils.getAttrValFrmAttributes(this.attributes,
- TokenConstants.TOKEN_MCACHE_FLUSH_INTVL);
+ TStoreConstants.TOKEN_MCACHE_FLUSH_INTVL);
if (atrVal != null) {
return Integer.parseInt(atrVal);
}
@@ -341,14 +342,14 @@ public class BdbTopicConfEntity implements Serializable {
public void setMemCacheFlushIntvl(final int memCacheFlushIntvl) {
this.attributes =
TStringUtils.setAttrValToAttributes(this.attributes,
- TokenConstants.TOKEN_MCACHE_FLUSH_INTVL,
+ TStoreConstants.TOKEN_MCACHE_FLUSH_INTVL,
String.valueOf(memCacheFlushIntvl));
}
public int getMaxMsgSize() {
String atrVal =
TStringUtils.getAttrValFrmAttributes(this.attributes,
- TServerConstants.TOKEN_MAX_MSG_SIZE);
+ TStoreConstants.TOKEN_MAX_MSG_SIZE);
if (atrVal != null) {
return Integer.parseInt(atrVal);
}
@@ -358,10 +359,44 @@ public class BdbTopicConfEntity implements Serializable {
public void setMaxMsgSize(int maxMsgSize) {
this.attributes =
TStringUtils.setAttrValToAttributes(this.attributes,
- TServerConstants.TOKEN_MAX_MSG_SIZE,
+ TStoreConstants.TOKEN_MAX_MSG_SIZE,
String.valueOf(maxMsgSize));
}
+ public long getDataVerId() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_VERSION_ID);
+ if (atrVal != null) {
+ return Long.parseLong(atrVal);
+ }
+ return TBaseConstants.META_VALUE_UNDEFINED;
+ }
+
+ public void setDataVerId(long dataVerId) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_DATA_VERSION_ID,
+ String.valueOf(dataVerId));
+ }
+
+ public int getTopicId() {
+ String atrVal =
+ TStringUtils.getAttrValFrmAttributes(this.attributes,
+ TStoreConstants.TOKEN_TOPICNAME_ID);
+ if (atrVal != null) {
+ return Integer.parseInt(atrVal);
+ }
+ return TBaseConstants.META_VALUE_UNDEFINED;
+ }
+
+ public void setTopicId(int topicId) {
+ this.attributes =
+ TStringUtils.setAttrValToAttributes(this.attributes,
+ TStoreConstants.TOKEN_TOPICNAME_ID,
+ String.valueOf(topicId));
+ }
+
public void appendAttributes(String attrKey, String attrVal) {
this.attributes =
TStringUtils.setAttrValToAttributes(this.attributes, attrKey, attrVal);
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/metamanage/DataOpErrCode.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/metamanage/DataOpErrCode.java
new file mode 100644
index 0000000..64b6282
--- /dev/null
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/metamanage/DataOpErrCode.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.server.master.metamanage;
+
+
+public enum DataOpErrCode {
+ DERR_SUCCESS(200, "Success."),
+ DERR_SUCCESS_UNCHANGED(201, "Success, but unchanged"),
+ DERR_NOT_EXIST(401, "Record not exist."),
+ DERR_EXISTED(402, "Record has existed."),
+ DERR_UNCHANGED(403, "Record not changed."),
+ DERR_UNCLEANED(404, "Related configuration is not cleaned up."),
+ DERR_CONDITION_LACK(405, "The preconditions are not met"),
+ DERR_ILLEGAL_STATUS(406, "Illegal operate status"),
+ DERR_ILLEGAL_VALUE(407, "Illegal data format or value"),
+ DERR_STORE_ABNORMAL(501, "Store layer throw exception."),
+ DERR_UPD_NOT_EXIST(502, "Record updated but not exist."),
+ DERR_STORE_STOPPED(510, "Store stopped."),
+ DERR_STORE_NOT_MASTER(511, "Store not active master."),
+ DERR_MASTER_UNKNOWN(599, "Unknown error.");
+
+ private int code;
+ private String description;
+
+
+ DataOpErrCode(int code, String description) {
+ this.code = code;
+ this.description = description;
+ }
+
+ public int getCode() {
+ return code;
+ }
+
+ public String getDescription() {
+ return description;
+ }
+
+ public static DataOpErrCode valueOf(int code) {
+ for (DataOpErrCode status : DataOpErrCode.values()) {
+ if (status.getCode() == code) {
+ return status;
+ }
+ }
+ throw new IllegalArgumentException(String.format("unknown data operate error code %s", code));
+ }
+
+}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/metamanage/MetaDataManager.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/metamanage/MetaDataManager.java
new file mode 100644
index 0000000..d7a8ca5
--- /dev/null
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/master/metamanage/MetaDataManager.java
@@ -0,0 +1,2072 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.server.master.metamanage;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import org.apache.inlong.tubemq.corebase.TBaseConstants;
+import org.apache.inlong.tubemq.corebase.TErrCodeConstants;
+import org.apache.inlong.tubemq.corebase.TokenConstants;
+import org.apache.inlong.tubemq.corebase.utils.KeyBuilderUtils;
+import org.apache.inlong.tubemq.corebase.utils.TStringUtils;
+import org.apache.inlong.tubemq.corebase.utils.Tuple2;
+import org.apache.inlong.tubemq.server.Server;
+import org.apache.inlong.tubemq.server.common.TServerConstants;
+import org.apache.inlong.tubemq.server.common.fileconfig.MasterReplicationConfig;
+import org.apache.inlong.tubemq.server.common.statusdef.ManageStatus;
+import org.apache.inlong.tubemq.server.common.statusdef.TopicStatus;
+import org.apache.inlong.tubemq.server.common.statusdef.TopicStsChgType;
+import org.apache.inlong.tubemq.server.common.utils.ProcessResult;
+import org.apache.inlong.tubemq.server.master.MasterConfig;
+import org.apache.inlong.tubemq.server.master.TMaster;
+import org.apache.inlong.tubemq.server.master.bdbstore.MasterGroupStatus;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.BdbMetaStoreServiceImpl;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.MetaStoreService;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.BaseEntity;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.BrokerConfEntity;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.ClusterSettingEntity;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.GroupConsumeCtrlEntity;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.GroupResCtrlEntity;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.TopicCtrlEntity;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.TopicDeployEntity;
+import org.apache.inlong.tubemq.server.master.metamanage.metastore.dao.entity.TopicPropGroup;
+import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerRunManager;
+import org.apache.inlong.tubemq.server.master.nodemanage.nodebroker.BrokerRunStatusInfo;
+import org.apache.inlong.tubemq.server.master.web.handler.BrokerProcessResult;
+import org.apache.inlong.tubemq.server.master.web.handler.GroupProcessResult;
+import org.apache.inlong.tubemq.server.master.web.handler.TopicProcessResult;
+import org.apache.inlong.tubemq.server.master.web.model.ClusterGroupVO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+
+public class MetaDataManager implements Server {
+
+ private static final Logger logger =
+ LoggerFactory.getLogger(MetaDataManager.class);
+ private final TMaster tMaster;
+ private static final ClusterSettingEntity defClusterSetting =
+ new ClusterSettingEntity().fillDefaultValue();
+ private final MasterReplicationConfig replicationConfig;
+ private final ScheduledExecutorService scheduledExecutorService;
+ private final MasterGroupStatus masterGroupStatus = new MasterGroupStatus();
+
+ private volatile boolean isStarted = false;
+ private volatile boolean isStopped = false;
+ private MetaStoreService metaStoreService;
+ private long serviceStartTime = System.currentTimeMillis();
+
+
+
+ public MetaDataManager(TMaster tMaster) {
+ this.tMaster = tMaster;
+ MasterConfig masterConfig = this.tMaster.getMasterConfig();
+ this.replicationConfig = masterConfig.getReplicationConfig();
+ this.metaStoreService =
+ new BdbMetaStoreServiceImpl(masterConfig.getHostName(),
+ masterConfig.getMetaDataPath(), this.replicationConfig);
+
+ this.scheduledExecutorService =
+ Executors.newSingleThreadScheduledExecutor(new ThreadFactory() {
+ @Override
+ public Thread newThread(Runnable r) {
+ return new Thread(r, "Master Status Check");
+ }
+ });
+ }
+
+ @Override
+ public void start() throws Exception {
+ if (isStarted) {
+ return;
+ }
+ // start meta store service
+ this.metaStoreService.start();
+ this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ MasterGroupStatus tmpGroupStatus =
+ metaStoreService.getMasterGroupStatus(true);
+ if (tmpGroupStatus == null) {
+ masterGroupStatus.setMasterGroupStatus(false, false, false);
+ } else {
+ masterGroupStatus.setMasterGroupStatus(metaStoreService.isMasterNow(),
+ tmpGroupStatus.isWritable(), tmpGroupStatus.isReadable());
+ }
+ } catch (Throwable e) {
+ logger.error(new StringBuilder(512)
+ .append("BDBGroupStatus Check exception, wait ")
+ .append(replicationConfig.getRepStatusCheckTimeoutMs())
+ .append(" ms to try again.").append(e.getMessage()).toString());
+ }
+ }
+ }, 0, replicationConfig.getRepStatusCheckTimeoutMs(), TimeUnit.MILLISECONDS);
+ // initial running data
+ BrokerRunManager brokerRunManager = this.tMaster.getBrokerRunManager();
+ brokerRunManager.updBrokerStaticInfo(this.metaStoreService.getBrokerConfInfo(null));
+ isStarted = true;
+ serviceStartTime = System.currentTimeMillis();
+ logger.info("BrokerConfManager StoreService Started");
+ }
+
+ @Override
+ public void stop() throws Exception {
+ if (isStopped) {
+ return;
+ }
+ this.scheduledExecutorService.shutdownNow();
+ isStopped = true;
+ logger.info("BrokerConfManager StoreService stopped");
+ }
+
+
+ /**
+ * If this node is the master role
+ *
+ * @return true if is master role or else
+ */
+ public boolean isSelfMaster() {
+ return metaStoreService.isMasterNow();
+ }
+
+ public boolean isPrimaryNodeActive() {
+ return metaStoreService.isPrimaryNodeActive();
+ }
+
+ /**
+ * Transfer master role to replica node
+ *
+ * @throws Exception
+ */
+ public void transferMaster() throws Exception {
+ if (metaStoreService.isMasterNow()
+ && !metaStoreService.isPrimaryNodeActive()) {
+ metaStoreService.transferMaster();
+ }
+ }
+
+ public InetSocketAddress getMasterAddress() {
+ return metaStoreService.getMasterAddress();
+ }
+
+ public ClusterGroupVO getGroupAddressStrInfo() {
+ return metaStoreService.getGroupAddressStrInfo();
+ }
+
+ /**
+ * Check if consume target is authorization or not
+ *
+ * @param consumerId checked consume id
+ * @param groupName checked group name
+ * @param reqTopicSet consumer request topic set
+ * @param reqTopicCondMap consumer request filter items
+ * @param sBuffer the print information string buffer
+ * @param result the process result return
+ * @return true is authorized, false not
+ */
+ public boolean isConsumeTargetAuthorized(String consumerId, String groupName,
+ Set<String> reqTopicSet,
+ Map<String, TreeSet<String>> reqTopicCondMap,
+ StringBuilder sBuffer, ProcessResult result) {
+ // check topic set
+ if ((reqTopicSet == null) || (reqTopicSet.isEmpty())) {
+ result.setFailResult(TErrCodeConstants.BAD_REQUEST,
+ "Request miss necessary subscribed topic data");
+ return result.isSuccess();
+ }
+ if ((reqTopicCondMap != null) && (!reqTopicCondMap.isEmpty())) {
+ // check if request topic set is all in the filter topic set
+ Set<String> condTopics = reqTopicCondMap.keySet();
+ List<String> unSetTopic = new ArrayList<>();
+ for (String topic : condTopics) {
+ if (!reqTopicSet.contains(topic)) {
+ unSetTopic.add(topic);
+ }
+ }
+ if (!unSetTopic.isEmpty()) {
+ result.setFailResult(TErrCodeConstants.BAD_REQUEST,
+ sBuffer.append("Filter's Topic not subscribed :")
+ .append(unSetTopic).toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
+ }
+ }
+ // check if group enable consume
+ Set<String> disableCsmTopicSet = new HashSet<>();
+ Set<String> enableFltCsmTopicSet = new HashSet<>();
+ for (String topicItem : reqTopicSet) {
+ if (TStringUtils.isBlank(topicItem)) {
+ continue;
+ }
+ TopicCtrlEntity topicEntity = metaStoreService.getTopicCtrlConf(topicItem);
+ if (topicEntity == null) {
+ continue;
+ }
+ if (topicEntity.isAuthCtrlEnable()) {
+ // check if consume group is allowed to consume
+ GroupConsumeCtrlEntity ctrlEntity =
+ metaStoreService.getConsumeCtrlByGroupAndTopic(groupName, topicItem);
+ if (ctrlEntity == null || !ctrlEntity.isEnableConsume()) {
+ disableCsmTopicSet.add(topicItem);
+ }
+ // check if consume group is required filter consume
+ if (ctrlEntity.isEnableFilterConsume()) {
+ enableFltCsmTopicSet.add(topicItem);
+ }
+ }
+ }
+ if (!disableCsmTopicSet.isEmpty()) {
+ result.setFailResult(TErrCodeConstants.CONSUME_GROUP_FORBIDDEN,
+ sBuffer.append("[unAuthorized Group] ").append(consumerId)
+ .append("'s consumerGroup not authorized by administrator, unAuthorizedTopics : ")
+ .append(disableCsmTopicSet).toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
+ }
+ // check if group enable filter consume
+ return checkFilterRstrTopics(groupName, consumerId,
+ enableFltCsmTopicSet, reqTopicCondMap, sBuffer, result);
+ }
+
+ private boolean checkConsumeRstrTopics(final String groupName, final String consumerId,
+ Set<String> enableFltCsmTopicSet,
+ Map<String, TreeSet<String>> reqTopicCondMap,
+ StringBuilder sBuffer, ProcessResult result) {
+ if (enableFltCsmTopicSet == null && enableFltCsmTopicSet.isEmpty()) {
+ result.setSuccResult("Ok!");
+ return result.isSuccess();
+ }
+ GroupConsumeCtrlEntity ctrlEntity;
+ for (String topicName : enableFltCsmTopicSet) {
+ ctrlEntity =
+ metaStoreService.getConsumeCtrlByGroupAndTopic(groupName, topicName);
+ if (ctrlEntity == null || !ctrlEntity.isEnableFilterConsume()) {
+ continue;
+ }
+ String allowedCondStr = ctrlEntity.getFilterCondStr();
+ if (allowedCondStr.length() == 2
+ && allowedCondStr.equals(TServerConstants.BLANK_FILTER_ITEM_STR)) {
+ result.setFailResult(TErrCodeConstants.CONSUME_CONTENT_FORBIDDEN,
+ sBuffer.append("[Restricted Group] ").append(consumerId)
+ .append(" : ").append(groupName)
+ .append(" not allowed to consume any data of topic ")
+ .append(topicName).toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
+ }
+ TreeSet<String> condItemSet = reqTopicCondMap.get(topicName);
+ if (condItemSet == null || condItemSet.isEmpty()) {
+ result.setFailResult(TErrCodeConstants.CONSUME_CONTENT_FORBIDDEN,
+ sBuffer.append("[Restricted Group] ").append(consumerId)
+ .append(" : ").append(groupName)
+ .append(" must set the filter conditions of topic ")
+ .append(topicName).toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
+ }
+ Map<String, List<String>> unAuthorizedCondMap = new HashMap<>();
+ for (String item : condItemSet) {
+ if (!allowedCondStr.contains(sBuffer.append(TokenConstants.ARRAY_SEP)
+ .append(item).append(TokenConstants.ARRAY_SEP).toString())) {
+ List<String> unAuthConds = unAuthorizedCondMap.get(topicName);
+ if (unAuthConds == null) {
+ unAuthConds = new ArrayList<>();
+ unAuthorizedCondMap.put(topicName, unAuthConds);
+ }
+ unAuthConds.add(item);
+ }
+ sBuffer.delete(0, sBuffer.length());
+ }
+ if (!unAuthorizedCondMap.isEmpty()) {
+ result.setFailResult(TErrCodeConstants.CONSUME_CONTENT_FORBIDDEN,
+ sBuffer.append("[Restricted Group] ").append(consumerId)
+ .append(" : unAuthorized filter conditions ")
+ .append(unAuthorizedCondMap).toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
+ }
+ }
+ result.setSuccResult("Ok!");
+ return result.isSuccess();
+ }
+
+
+ private boolean checkFilterRstrTopics(final String groupName, final String consumerId,
+ Set<String> enableFltCsmTopicSet,
+ Map<String, TreeSet<String>> reqTopicCondMap,
+ StringBuilder sBuffer, ProcessResult result) {
+ if (enableFltCsmTopicSet == null && enableFltCsmTopicSet.isEmpty()) {
+ result.setSuccResult("Ok!");
+ return result.isSuccess();
+ }
+ GroupConsumeCtrlEntity ctrlEntity;
+ for (String topicName : enableFltCsmTopicSet) {
+ ctrlEntity =
+ metaStoreService.getConsumeCtrlByGroupAndTopic(groupName, topicName);
+ if (ctrlEntity == null || !ctrlEntity.isEnableFilterConsume()) {
+ continue;
+ }
+ String allowedCondStr = ctrlEntity.getFilterCondStr();
+ if (allowedCondStr.length() == 2
+ && allowedCondStr.equals(TServerConstants.BLANK_FILTER_ITEM_STR)) {
+ result.setFailResult(TErrCodeConstants.CONSUME_CONTENT_FORBIDDEN,
+ sBuffer.append("[Restricted Group] ").append(consumerId)
+ .append(" : ").append(groupName)
+ .append(" not allowed to consume any data of topic ")
+ .append(topicName).toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
+ }
+ TreeSet<String> condItemSet = reqTopicCondMap.get(topicName);
+ if (condItemSet == null || condItemSet.isEmpty()) {
+ result.setFailResult(TErrCodeConstants.CONSUME_CONTENT_FORBIDDEN,
+ sBuffer.append("[Restricted Group] ").append(consumerId)
+ .append(" : ").append(groupName)
+ .append(" must set the filter conditions of topic ")
+ .append(topicName).toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
+ }
+ Map<String, List<String>> unAuthorizedCondMap = new HashMap<>();
+ for (String item : condItemSet) {
+ if (!allowedCondStr.contains(sBuffer.append(TokenConstants.ARRAY_SEP)
+ .append(item).append(TokenConstants.ARRAY_SEP).toString())) {
+ List<String> unAuthConds = unAuthorizedCondMap.get(topicName);
+ if (unAuthConds == null) {
+ unAuthConds = new ArrayList<>();
+ unAuthorizedCondMap.put(topicName, unAuthConds);
+ }
+ unAuthConds.add(item);
+ }
+ sBuffer.delete(0, sBuffer.length());
+ }
+ if (!unAuthorizedCondMap.isEmpty()) {
+ result.setFailResult(TErrCodeConstants.CONSUME_CONTENT_FORBIDDEN,
+ sBuffer.append("[Restricted Group] ").append(consumerId)
+ .append(" : unAuthorized filter conditions ")
+ .append(unAuthorizedCondMap).toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
+ }
+ }
+ result.setSuccResult("Ok!");
+ return result.isSuccess();
+ }
+
+
+ // ///////////////////////////////////////////////////////////////////////////////
+
+ /**
+ * Add or update broker configure information
+ *
+ * @param isAddOp whether add operation
+ * @param opInfoEntity operator inforamtion
+ * @param brokerId broker id
+ * @param brokerIp broker ip
+ * @param brokerPort broker port
+ * @param brokerTlsPort broker tls port
+ * @param brokerWebPort broker web port
+ * @param regionId region id
+ * @param groupId group id
+ * @param mngStatus manage status
+ * @param topicProps default topic proterty inforamtion
+ * @param sBuffer the print information string buffer
+ * @param result the process result return
+ * @return true if success otherwise false
+ */
+ public BrokerProcessResult addOrUpdBrokerConfig(boolean isAddOp, BaseEntity opInfoEntity,
+ int brokerId, String brokerIp, int brokerPort,
+ int brokerTlsPort, int brokerWebPort,
+ int regionId, int groupId,
+ ManageStatus mngStatus,
+ TopicPropGroup topicProps,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ BrokerConfEntity entity =
+ new BrokerConfEntity(opInfoEntity, brokerId, brokerIp);
+ entity.updModifyInfo(opInfoEntity.getDataVerId(), brokerPort,
+ brokerTlsPort, brokerWebPort, regionId, groupId, mngStatus, topicProps);
+ return addOrUpdBrokerConfig(isAddOp, entity, sBuffer, result);
+ }
+
+ /**
+ * Add or update broker configure information
+ *
+ * @param isAddOp whether add operation
+ * @param entity need add or update configure information
+ * @param sBuffer the print information string buffer
+ * @param result the process result return
+ * @return true if success otherwise false
+ */
+ public BrokerProcessResult addOrUpdBrokerConfig(boolean isAddOp, BrokerConfEntity entity,
+ StringBuilder sBuffer, ProcessResult result) {
+ if (isAddOp) {
+ if (metaStoreService.getBrokerConfByBrokerId(entity.getBrokerId()) == null
+ && metaStoreService.getBrokerConfByBrokerIp(entity.getBrokerIp()) == null) {
+ if (metaStoreService.addBrokerConf(entity, sBuffer, result)) {
+ this.tMaster.getBrokerRunManager().updBrokerStaticInfo(entity);
+ }
+ } else {
+ result.setFailResult(DataOpErrCode.DERR_EXISTED.getCode(),
+ sBuffer.append("Duplicated broker configure record! query index is :")
+ .append("brokerId=").append(entity.getBrokerId())
+ .append(",brokerIp=").append(entity.getBrokerIp()).toString());
+ sBuffer.delete(0, sBuffer.length());
+ }
+ } else {
+ BrokerConfEntity curEntity =
+ metaStoreService.getBrokerConfByBrokerId(entity.getBrokerId());
+ if (curEntity == null) {
+ result.setFailResult(DataOpErrCode.DERR_NOT_EXIST.getCode(),
+ sBuffer.append("Not found broker configure by brokerId=")
+ .append(entity.getBrokerId()).toString());
+ sBuffer.delete(0, sBuffer.length());
+ } else {
+ BrokerConfEntity newEntity = curEntity.clone();
+ newEntity.updBaseModifyInfo(entity);
+ if (newEntity.updModifyInfo(entity.getDataVerId(), entity.getBrokerPort(),
+ entity.getBrokerTLSPort(), entity.getBrokerWebPort(),
+ entity.getRegionId(), entity.getGroupId(),
+ entity.getManageStatus(), entity.getTopicProps())) {
+ if (metaStoreService.updBrokerConf(newEntity, sBuffer, result)) {
+ triggerBrokerConfDataSync(entity.getBrokerId(), sBuffer, result);
+ }
+ } else {
+ result.setSuccResult(null);
+ }
+ }
+ }
+ return new BrokerProcessResult(entity.getBrokerId(), entity.getBrokerIp(), result);
+ }
+
+ /**
+ * Modify broker configure information
+ *
+ * @param entity the broker configure entity will be update
+ * @param strBuffer the print information string buffer
+ * @param result the process result return
+ * @return true if success otherwise false
+ */
+ public boolean modBrokerConfig(BrokerConfEntity entity,
+ StringBuilder strBuffer,
+ ProcessResult result) {
+ metaStoreService.updBrokerConf(entity, strBuffer, result);
+ return result.isSuccess();
+ }
+
+ /**
+ * Get broker configure information
+ *
+ * @param qryEntity
+ * @return broker configure information
+ */
+ public Map<Integer, BrokerConfEntity> confGetBrokerConfInfo(
+ BrokerConfEntity qryEntity) {
+ return metaStoreService.getBrokerConfInfo(qryEntity);
+ }
+
+ /**
+ * Get broker configure information
+ *
+ * @param qryEntity
+ * @return broker configure information
+ */
+ public Map<Integer, BrokerConfEntity> getBrokerConfInfo(Set<Integer> brokerIdSet,
+ Set<String> brokerIpSet,
+ BrokerConfEntity qryEntity) {
+ return metaStoreService.getBrokerConfInfo(brokerIdSet, brokerIpSet, qryEntity);
+ }
+
+ /**
+ * Change broker configure status
+ *
+ * @param opEntity operator
+ * @param brokerIdSet need deleted broker id set
+ * @param newMngStatus manage status
+ * @param sBuffer the print information string buffer
+ * @param result the process result return
+ * @return true if success otherwise false
+ */
+ public List<BrokerProcessResult> changeBrokerConfStatus(BaseEntity opEntity,
+ Set<Integer> brokerIdSet,
+ ManageStatus newMngStatus,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ BrokerConfEntity curEntry;
+ BrokerConfEntity newEntry;
+ List<BrokerProcessResult> retInfo = new ArrayList<>();
+ // check target broker configure's status
+ for (Integer brokerId : brokerIdSet) {
+ curEntry = metaStoreService.getBrokerConfByBrokerId(brokerId);
+ if (curEntry == null) {
+ result.setFailResult(DataOpErrCode.DERR_NOT_EXIST.getCode(),
+ "The broker configure not exist!");
+ retInfo.add(new BrokerProcessResult(brokerId, "", result));
+ continue;
+ }
+ if (curEntry.getManageStatus() == newMngStatus) {
+ result.setSuccResult(null);
+ retInfo.add(new BrokerProcessResult(brokerId, curEntry.getBrokerIp(), result));
+ continue;
+ }
+ if (newMngStatus == ManageStatus.STATUS_MANAGE_OFFLINE) {
+ if (curEntry.getManageStatus().getCode()
+ < ManageStatus.STATUS_MANAGE_ONLINE.getCode()) {
+ result.setFailResult(DataOpErrCode.DERR_ILLEGAL_STATUS.getCode(),
+ sBuffer.append("Broker by brokerId=").append(brokerId)
+ .append(" on draft status, not need offline operate!")
+ .toString());
+ sBuffer.delete(0, sBuffer.length());
+ retInfo.add(new BrokerProcessResult(brokerId, "", result));
+ continue;
+ }
+ }
+ newEntry = curEntry.clone();
+ newEntry.updBaseModifyInfo(opEntity);
+ if (newEntry.updModifyInfo(opEntity.getDataVerId(),
+ TBaseConstants.META_VALUE_UNDEFINED, TBaseConstants.META_VALUE_UNDEFINED,
+ TBaseConstants.META_VALUE_UNDEFINED, TBaseConstants.META_VALUE_UNDEFINED,
+ TBaseConstants.META_VALUE_UNDEFINED, newMngStatus, null)) {
+ if (metaStoreService.updBrokerConf(newEntry, sBuffer, result)) {
+ triggerBrokerConfDataSync(newEntry.getBrokerId(), sBuffer, result);
+ }
+ } else {
+ result.setSuccResult(null);
+ }
+ retInfo.add(new BrokerProcessResult(brokerId, curEntry.getBrokerIp(), result));
+ }
+ return retInfo;
+ }
+
+ /**
+ * Change broker read write status
+ *
+ * @param opEntity operator
+ * @param brokerIdSet need deleted broker id set
+ * @param rdWtTpl need changed read or write status
+ * @param sBuffer the print information string buffer
+ * @param result the process result return
+ * @return true if success otherwise false
+ */
+ public List<BrokerProcessResult> changeBrokerRWStatus(BaseEntity opEntity,
+ Set<Integer> brokerIdSet,
+ Tuple2<Boolean, Boolean> rdWtTpl,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ BrokerConfEntity curEntry;
+ BrokerConfEntity newEntry;
+ List<BrokerProcessResult> retInfo = new ArrayList<>();
+ // check target broker configure's status
+ for (Integer brokerId : brokerIdSet) {
+ curEntry = metaStoreService.getBrokerConfByBrokerId(brokerId);
+ if (curEntry == null) {
+ result.setFailResult(DataOpErrCode.DERR_NOT_EXIST.getCode(),
+ "The broker configure not exist!");
+ retInfo.add(new BrokerProcessResult(brokerId, "", result));
+ continue;
+ }
+ if (curEntry.getManageStatus().getCode()
+ < ManageStatus.STATUS_MANAGE_ONLINE.getCode()) {
+ result.setFailResult(DataOpErrCode.DERR_CONDITION_LACK.getCode(),
+ "The broker configure under draft status, please online first!");
+ retInfo.add(new BrokerProcessResult(brokerId, "", result));
+ continue;
+ }
+ ManageStatus newMngStatus = ManageStatus.getNewStatus(
+ curEntry.getManageStatus(), rdWtTpl.getF0(), rdWtTpl.getF1());
+ if (curEntry.getManageStatus() == newMngStatus) {
+ result.setSuccResult(null);
+ retInfo.add(new BrokerProcessResult(brokerId, curEntry.getBrokerIp(), result));
+ continue;
+ }
+ newEntry = curEntry.clone();
+ newEntry.updBaseModifyInfo(opEntity);
+ if (newEntry.updModifyInfo(opEntity.getDataVerId(),
+ TBaseConstants.META_VALUE_UNDEFINED, TBaseConstants.META_VALUE_UNDEFINED,
+ TBaseConstants.META_VALUE_UNDEFINED, TBaseConstants.META_VALUE_UNDEFINED,
+ TBaseConstants.META_VALUE_UNDEFINED, newMngStatus, null)) {
+ if (metaStoreService.updBrokerConf(newEntry, sBuffer, result)) {
+ triggerBrokerConfDataSync(newEntry.getBrokerId(), sBuffer, result);
+ }
+ } else {
+ result.setSuccResult(null);
+ }
+ retInfo.add(new BrokerProcessResult(brokerId, curEntry.getBrokerIp(), result));
+ }
+ return retInfo;
+ }
+
+ /**
+ * Change broker configure status
+ *
+ * @param opEntity operator
+ * @param brokerIdSet need deleted broker id set
+ * @param sBuffer the print information string buffer
+ * @param result the process result return
+ * @return true if success otherwise false
+ */
+ public List<BrokerProcessResult> reloadBrokerConfInfo(BaseEntity opEntity,
+ Set<Integer> brokerIdSet,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ BrokerConfEntity curEntry;
+ List<BrokerProcessResult> retInfo = new ArrayList<>();
+ // check target broker configure's status
+ for (Integer brokerId : brokerIdSet) {
+ curEntry = metaStoreService.getBrokerConfByBrokerId(brokerId);
+ if (curEntry == null) {
+ result.setFailResult(DataOpErrCode.DERR_NOT_EXIST.getCode(),
+ "The broker configure not exist!");
+ retInfo.add(new BrokerProcessResult(brokerId, "", result));
+ continue;
+ }
+ if (!curEntry.getManageStatus().isOnlineStatus()) {
+ result.setFailResult(DataOpErrCode.DERR_ILLEGAL_STATUS.getCode(),
+ sBuffer.append("The broker manage status by brokerId=").append(brokerId)
+ .append(" is not in online status, can't reload this configure! ")
+ .toString());
+ sBuffer.delete(0, sBuffer.length());
+ retInfo.add(new BrokerProcessResult(brokerId, "", result));
+ continue;
+ }
+ triggerBrokerConfDataSync(curEntry.getBrokerId(), sBuffer, result);
+ retInfo.add(new BrokerProcessResult(brokerId, curEntry.getBrokerIp(), result));
+ }
+ return retInfo;
+ }
+
+ /**
+ * Delete broker configure information
+ *
+ * @param operator operator
+ * @param rsvData if reserve topic's data info
+ * @param brokerIdSet need deleted broker id set
+ * @param sBuffer the print information string buffer
+ * @param result the process result return
+ * @return true if success otherwise false
+ */
+ public List<BrokerProcessResult> delBrokerConfInfo(String operator, boolean rsvData,
+ Set<Integer> brokerIdSet,
+ StringBuilder sBuffer,
+ ProcessResult result) {
+ List<BrokerProcessResult> retInfo = new ArrayList<>();
+ for (int brokerId : brokerIdSet) {
+ // get broker configure
+ BrokerConfEntity entity =
+ metaStoreService.getBrokerConfByBrokerId(brokerId);
+ if (entity == null) {
+ result.setSuccResult(null);
+ retInfo.add(new BrokerProcessResult(brokerId, "", result));
+ continue;
+ }
+ // check broker's manage status
+ if (entity.getManageStatus().isOnlineStatus()) {
+ result.setFailResult(DataOpErrCode.DERR_ILLEGAL_STATUS.getCode(),
+ "Broker manage status is online, please offline first!");
+ retInfo.add(new BrokerProcessResult(brokerId, entity.getBrokerIp(), result));
+ continue;
+ }
+ BrokerRunManager brokerRunManager = tMaster.getBrokerRunManager();
+ BrokerRunStatusInfo runStatusInfo =
+ brokerRunManager.getBrokerRunStatusInfo(brokerId);
+ if (runStatusInfo != null
+ && entity.getManageStatus() == ManageStatus.STATUS_MANAGE_OFFLINE
+ && runStatusInfo.inProcessingStatus()) {
+ result.setFailResult(DataOpErrCode.DERR_ILLEGAL_STATUS.getCode(),
+ sBuffer.append("Illegal value: the broker is processing offline event by brokerId=")
+ .append(brokerId).append(", please offline first and try later!").toString());
+ sBuffer.delete(0, sBuffer.length());
+ retInfo.add(new BrokerProcessResult(brokerId, entity.getBrokerIp(), result));
+ continue;
+ }
+ if (!chkBrokerTopicConfAllowed(brokerId, rsvData, sBuffer, result)) {
+ retInfo.add(new BrokerProcessResult(brokerId, entity.getBrokerIp(), result));
+ continue;
+ }
+ delBrokerConfig(operator, entity.getBrokerId(), rsvData, sBuffer, result);
+ retInfo.add(new BrokerProcessResult(entity.getBrokerId(),
+ entity.getBrokerIp(), result));
+ }
+ return retInfo;
+ }
+
+ private boolean chkBrokerTopicConfAllowed(int brokerId, boolean rsvData,
+ StringBuilder sBuffer, ProcessResult result) {
+ // check broker's topic configures
+ Map<String, TopicDeployEntity> topiConfMap =
+ metaStoreService.getConfiguredTopicInfo(brokerId);
+ if (topiConfMap == null || topiConfMap.isEmpty()) {
+ result.setSuccResult(null);
+ return result.isSuccess();
+ }
+ if (rsvData) {
+ for (Map.Entry<String, TopicDeployEntity> entry : topiConfMap.entrySet()) {
+ if (entry.getValue() == null) {
+ continue;
+ }
+ if (entry.getValue().isAcceptPublish()
+ || entry.getValue().isAcceptSubscribe()) {
+ result.setFailResult(DataOpErrCode.DERR_ILLEGAL_STATUS.getCode(),
+ sBuffer.append("The topic ").append(entry.getKey())
+ .append("'s acceptPublish and acceptSubscribe parameters")
+ .append(" must be false in broker=").append(brokerId)
+ .append(" before broker delete by reserve data method!").toString());
+ sBuffer.delete(0, sBuffer.length());
+ return result.isSuccess();
+ }
+ }
... 28217 lines suppressed ...