You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2022/04/20 14:47:39 UTC
[iotdb] branch master updated: [IOTDB-2942] [IOTDB-2944] Delete duplicate data structure in node-commons and extend TDataNodeLocation (#5592)
This is an automated email from the ASF dual-hosted git repository.
qiaojialin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iotdb.git
The following commit(s) were added to refs/heads/master by this push:
new 7ce6033609 [IOTDB-2942] [IOTDB-2944] Delete duplicate data structure in node-commons and extend TDataNodeLocation (#5592)
7ce6033609 is described below
commit 7ce6033609f0df3c7b809dfc048c1bfcb47aefb7
Author: YongzaoDan <33...@users.noreply.github.com>
AuthorDate: Wed Apr 20 22:47:33 2022 +0800
[IOTDB-2942] [IOTDB-2944] Delete duplicate data structure in node-commons and extend TDataNodeLocation (#5592)
---
client-cpp/pom.xml | 7 +-
client-py/pom.xml | 8 +
confignode/pom.xml | 10 -
.../resources/conf/iotdb-confignode.properties | 18 +-
.../iotdb/confignode/cli/TemporaryClient.java | 54 ++++--
.../iotdb/confignode/conf/ConfigNodeConf.java | 59 +++---
.../confignode/conf/ConfigNodeDescriptor.java | 22 ++-
.../response/DataNodeConfigurationDataSet.java | 6 +-
...oDataSet.java => DataNodeLocationsDataSet.java} | 34 +---
.../consensus/response/DataPartitionDataSet.java | 53 +-----
.../consensus/response/SchemaPartitionDataSet.java | 31 +--
.../response/StorageGroupSchemaDataSet.java | 11 +-
.../iotdb/confignode/manager/ConfigManager.java | 10 +-
.../iotdb/confignode/manager/ConsensusManager.java | 6 +-
.../iotdb/confignode/manager/DataNodeManager.java | 25 +--
.../iotdb/confignode/manager/PartitionManager.java | 48 ++---
.../iotdb/confignode/manager/RegionManager.java | 85 +++++----
.../confignode/partition/StorageGroupSchema.java | 121 ------------
.../persistence/DataNodeInfoPersistence.java | 39 ++--
.../persistence/PartitionInfoPersistence.java | 22 +--
.../persistence/RegionInfoPersistence.java | 54 +++---
.../physical/crud/CreateDataPartitionPlan.java | 44 ++---
.../physical/crud/CreateRegionsPlan.java | 21 ++-
.../physical/crud/CreateSchemaPartitionPlan.java | 27 +--
.../crud/GetOrCreateDataPartitionPlan.java | 40 ++--
.../crud/GetOrCreateSchemaPartitionPlan.java | 22 ++-
.../iotdb/confignode/physical/sys/AuthorPlan.java | 24 +--
.../physical/sys/RegisterDataNodePlan.java | 32 ++--
.../physical/sys/SetStorageGroupPlan.java | 17 +-
.../server/ConfigNodeRPCServerProcessor.java | 50 +++--
.../iotdb/confignode/cli/TemporaryClientDemo.java | 4 +-
.../confignode/consensus/RatisConsensusDemo.java | 151 ---------------
.../manager/ConfigManagerManualTest.java | 133 -------------
...serializeUT.java => PhysicalPlanSerDeTest.java} | 110 +++++++----
.../server/ConfigNodeRPCServerProcessorTest.java | 209 +++++++++------------
consensus/pom.xml | 5 -
.../apache/iotdb/consensus/ConsensusFactory.java | 7 +-
.../org/apache/iotdb/consensus/common/Peer.java | 8 +-
.../iotdb/consensus/ratis/RatisConsensus.java | 7 +-
.../iotdb/consensus/ratis/RequestMessage.java | 2 +-
.../org/apache/iotdb/consensus/ratis/Utils.java | 16 +-
.../consensus/standalone/StandAloneConsensus.java | 6 +-
.../iotdb/consensus/ratis/RatisConsensusTest.java | 8 +-
.../standalone/StandAloneConsensusTest.java | 31 +--
node-commons/pom.xml | 10 +
.../iotdb/commons/cluster/DataNodeLocation.java | 81 --------
.../org/apache/iotdb/commons/cluster/Endpoint.java | 92 ---------
.../iotdb/commons/consensus/ConsensusGroupId.java | 44 +++--
.../iotdb/commons/consensus/DataRegionId.java | 21 +--
.../apache/iotdb/commons/consensus/GroupType.java | 27 ---
.../iotdb/commons/consensus/PartitionRegionId.java | 21 +--
.../iotdb/commons/consensus/SchemaRegionId.java | 21 +--
.../iotdb/commons/partition/DataPartition.java | 69 +++----
.../commons/partition/DataPartitionQueryParam.java | 8 +-
.../apache/iotdb/commons/partition/Partition.java | 3 +-
.../iotdb/commons/partition/RegionReplicaSet.java | 141 --------------
.../iotdb/commons/partition/SchemaPartition.java | 31 +--
.../commons/partition/SeriesPartitionSlot.java | 59 ------
.../iotdb/commons/partition/TimePartitionSlot.java | 64 -------
.../executor/SeriesPartitionExecutor.java | 4 +-
.../partition/executor/hash/APHashExecutor.java | 6 +-
.../partition/executor/hash/BKDRHashExecutor.java | 6 +-
.../partition/executor/hash/JSHashExecutor.java | 6 +-
.../partition/executor/hash/SDBMHashExecutor.java | 6 +-
.../commons/utils/BasicStructureSerDeUtil.java | 6 +-
.../apache/iotdb/commons/utils/CommonUtils.java | 12 +-
.../commons/utils/ThriftCommonsSerDeUtils.java | 144 ++++++++++++++
.../commons/utils/ThriftConfigNodeSerDeUtils.java | 87 +++++++++
.../apache/iotdb/commons/ConsensusGroupIdTest.java | 23 +--
.../commons/utils/BasicStructureSerDeUtilTest.java | 18 +-
.../commons/utils/ThriftCommonsSerDeUtilsTest.java | 116 ++++++++++++
.../utils/ThriftConfigNodeSerDeUtilsTest.java | 67 +++++++
pom.xml | 2 +
server/pom.xml | 15 +-
.../apache/iotdb/db/client/ConfigNodeClient.java | 23 +--
.../apache/iotdb/db/consensus/ConsensusImpl.java | 4 +-
.../org/apache/iotdb/db/engine/StorageEngine.java | 6 +-
.../iotdb/db/mpp/buffer/DataBlockManager.java | 6 +-
.../mpp/buffer/DataBlockServiceClientFactory.java | 6 +-
.../iotdb/db/mpp/buffer/IDataBlockManager.java | 6 +-
.../iotdb/db/mpp/common/MPPQueryContext.java | 8 +-
.../iotdb/db/mpp/common/ResultNodeContext.java | 8 +-
.../apache/iotdb/db/mpp/execution/Coordinator.java | 8 +-
.../iotdb/db/mpp/execution/QueryExecution.java | 4 +-
.../mpp/execution/config/SetStorageGroupTask.java | 6 +-
.../scheduler/AbstractFragInsStateTracker.java | 4 +-
.../scheduler/InternalServiceClientFactory.java | 6 +-
.../scheduler/SimpleFragInstanceDispatcher.java | 12 +-
.../execution/scheduler/SimpleQueryTerminator.java | 10 +-
.../apache/iotdb/db/mpp/sql/analyze/Analysis.java | 4 +-
.../mpp/sql/analyze/ClusterPartitionFetcher.java | 26 ++-
.../db/mpp/sql/analyze/ClusterSchemaFetcher.java | 6 +-
.../mpp/sql/analyze/FakePartitionFetcherImpl.java | 114 ++++++-----
.../db/mpp/sql/planner/DistributionPlanner.java | 28 +--
.../db/mpp/sql/planner/LocalExecutionPlanner.java | 10 +-
.../db/mpp/sql/planner/plan/FragmentInstance.java | 33 ++--
.../db/mpp/sql/planner/plan/PlanFragment.java | 8 +-
.../plan/SimpleFragmentParallelPlanner.java | 4 +-
.../sql/planner/plan/node/PlanGraphPrinter.java | 2 +-
.../mpp/sql/planner/plan/node/WritePlanNode.java | 4 +-
.../plan/node/metedata/read/SchemaScanNode.java | 8 +-
.../node/metedata/write/CreateTimeSeriesNode.java | 10 +-
.../planner/plan/node/process/ExchangeNode.java | 13 +-
.../planner/plan/node/sink/FragmentSinkNode.java | 13 +-
.../plan/node/source/SeriesAggregateScanNode.java | 13 +-
.../planner/plan/node/source/SeriesScanNode.java | 16 +-
.../sql/planner/plan/node/source/SourceNode.java | 6 +-
.../plan/node/write/InsertMultiTabletsNode.java | 6 +-
.../sql/planner/plan/node/write/InsertNode.java | 10 +-
.../sql/planner/plan/node/write/InsertRowNode.java | 4 +-
.../planner/plan/node/write/InsertRowsNode.java | 6 +-
.../plan/node/write/InsertRowsOfOneDeviceNode.java | 6 +-
.../planner/plan/node/write/InsertTabletNode.java | 14 +-
.../mpp/sql/statement/crud/InsertRowStatement.java | 4 +-
.../crud/InsertRowsOfOneDeviceStatement.java | 6 +-
.../sql/statement/crud/InsertTabletStatement.java | 8 +-
.../java/org/apache/iotdb/db/service/DataNode.java | 23 ++-
.../service/thrift/impl/InternalServiceImpl.java | 35 ++--
.../db/mpp/sql/plan/DistributionPlannerTest.java | 155 ++++++++-------
.../db/mpp/sql/plan/FragmentInstanceSerdeTest.java | 48 +++--
.../iotdb/db/mpp/sql/plan/QueryPlannerTest.java | 5 +-
.../read/DeviceSchemaScanNodeSerdeTest.java | 6 +-
.../read/TimeSeriesSchemaScanNodeSerdeTest.java | 6 +-
.../plan/node/process/AggregateNodeSerdeTest.java | 8 +-
.../node/process/DeviceMergeNodeSerdeTest.java | 8 +-
.../plan/node/process/ExchangeNodeSerdeTest.java | 14 +-
.../sql/plan/node/process/FillNodeSerdeTest.java | 8 +-
.../sql/plan/node/process/FilterNodeSerdeTest.java | 8 +-
.../plan/node/process/FilterNullNodeSerdeTest.java | 8 +-
.../node/process/GroupByLevelNodeSerdeTest.java | 8 +-
.../sql/plan/node/process/LimitNodeSerdeTest.java | 8 +-
.../sql/plan/node/process/OffsetNodeSerdeTest.java | 17 +-
.../sql/plan/node/process/SortNodeSerdeTest.java | 8 +-
.../plan/node/process/TimeJoinNodeSerdeTest.java | 8 +-
.../plan/node/sink/FragmentSinkNodeSerdeTest.java | 4 +-
.../source/SeriesAggregateScanNodeSerdeTest.java | 8 +-
.../plan/node/source/SeriesScanNodeSerdeTest.java | 8 +-
.../iotdb/db/service/InternalServiceImplTest.java | 57 +++---
thrift-cluster/pom.xml | 5 +
{thrift-cluster => thrift-commons}/pom.xml | 11 +-
.../src/main/thrift/common.thrift | 29 ++-
thrift-confignode/pom.xml | 2 +-
.../src/main/thrift/confignode.thrift | 38 ++--
thrift/pom.xml | 5 +
thrift/src/main/thrift/mpp.thrift | 7 +-
145 files changed, 1790 insertions(+), 2171 deletions(-)
diff --git a/client-cpp/pom.xml b/client-cpp/pom.xml
index f5428aaf95..a6feb1d413 100644
--- a/client-cpp/pom.xml
+++ b/client-cpp/pom.xml
@@ -50,6 +50,11 @@
<artifactId>client-cpp-tools-thrift</artifactId>
<version>${project.version}</version>
</dependency>
+ <dependency>
+ <groupId>org.apache.iotdb</groupId>
+ <artifactId>iotdb-thrift-commons</artifactId>
+ <version>${project.version}</version>
+ </dependency>
</dependencies>
<profiles>
<profile>
@@ -164,7 +169,7 @@
<destinationFile>${project.build.directory}/thrift/rpc.thrift</destinationFile>
</fileSet>
<fileSet>
- <sourceFile>../thrift/src/main/thrift/common.thrift</sourceFile>
+ <sourceFile>../thrift-commons/src/main/thrift/common.thrift</sourceFile>
<destinationFile>${project.build.directory}/thrift/common.thrift</destinationFile>
</fileSet>
</fileSets>
diff --git a/client-py/pom.xml b/client-py/pom.xml
index fa7cce5e17..a5edfb6d7d 100644
--- a/client-py/pom.xml
+++ b/client-py/pom.xml
@@ -36,6 +36,11 @@
<artifactId>iotdb-thrift</artifactId>
<version>${project.version}</version>
</dependency>
+ <dependency>
+ <groupId>org.apache.iotdb</groupId>
+ <artifactId>iotdb-thrift-commons</artifactId>
+ <version>${project.version}</version>
+ </dependency>
</dependencies>
<build>
<plugins>
@@ -92,6 +97,9 @@
<resource>
<directory>${basedir}/../thrift/target/generated-sources-python/iotdb/thrift/</directory>
</resource>
+ <resource>
+ <directory>${basedir}/../thrift-commons/target/generated-sources-python/iotdb/thrift/</directory>
+ </resource>
</resources>
</configuration>
</execution>
diff --git a/confignode/pom.xml b/confignode/pom.xml
index 651e5aa2c0..7f1ab3ff96 100644
--- a/confignode/pom.xml
+++ b/confignode/pom.xml
@@ -40,16 +40,6 @@
<artifactId>service-rpc</artifactId>
<version>${project.version}</version>
</dependency>
- <dependency>
- <groupId>org.apache.iotdb</groupId>
- <artifactId>iotdb-thrift-confignode</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.iotdb</groupId>
- <artifactId>node-commons</artifactId>
- <version>${project.version}</version>
- </dependency>
<dependency>
<groupId>org.apache.iotdb</groupId>
<artifactId>iotdb-consensus</artifactId>
diff --git a/confignode/src/assembly/resources/conf/iotdb-confignode.properties b/confignode/src/assembly/resources/conf/iotdb-confignode.properties
index a48fbdbbc8..dccb57f8ba 100644
--- a/confignode/src/assembly/resources/conf/iotdb-confignode.properties
+++ b/confignode/src/assembly/resources/conf/iotdb-confignode.properties
@@ -152,18 +152,18 @@ data_node_consensus_protocol_class=org.apache.iotdb.consensus.standalone.StandAl
# Datatype: long
# default_ttl=36000000
-####################
-### Region Configuration
-####################
+# Default number of SchemaRegion replicas
+# Datatype: int
+# schema_replication_factor=3
-# The number of replicas of each region
+# Default number of DataRegion replicas
# Datatype: int
-region_replica_count=1
+# data_replication_factor=3
-# The number of SchemaRegions of each StorageGroup
+# The initial number of SchemaRegions of each StorageGroup
# Datatype: int
-# schema_region_count=1
+# initial_schema_region_count=1
-# The number of DataRegions of each StorageGroup
+# The initial number of DataRegions of each StorageGroup
# Datatype: int
-# data_region_count=1
\ No newline at end of file
+# initial_data_region_count=1
\ No newline at end of file
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/cli/TemporaryClient.java b/confignode/src/main/java/org/apache/iotdb/confignode/cli/TemporaryClient.java
index 437f75a536..de570eb36f 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/cli/TemporaryClient.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/cli/TemporaryClient.java
@@ -18,9 +18,9 @@
*/
package org.apache.iotdb.confignode.cli;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
import org.apache.iotdb.confignode.persistence.DataNodeInfoPersistence;
import org.apache.iotdb.mpp.rpc.thrift.InternalService;
import org.apache.iotdb.mpp.rpc.thrift.TCreateDataRegionReq;
@@ -59,7 +59,7 @@ public class TemporaryClient {
this.clients = new HashMap<>();
}
- public void buildClient(int dataNodeId, Endpoint endpoint) {
+ public void buildClient(int dataNodeId, TEndPoint endpoint) {
for (int i = 0; i < retryNum; i++) {
try {
TTransport transport =
@@ -83,20 +83,22 @@ public class TemporaryClient {
}
private TCreateSchemaRegionReq genCreateSchemaRegionReq(
- String storageGroup, RegionReplicaSet regionReplicaSet) {
+ String storageGroup, TRegionReplicaSet regionReplicaSet) {
TCreateSchemaRegionReq req = new TCreateSchemaRegionReq();
req.setStorageGroup(storageGroup);
- req.setRegionReplicaSet(regionReplicaSet.convertToRPCTRegionReplicaSet());
+ req.setRegionReplicaSet(regionReplicaSet);
return req;
}
public void createSchemaRegion(
- int dataNodeId, String storageGroup, RegionReplicaSet regionReplicaSet) {
+ int dataNodeId, String storageGroup, TRegionReplicaSet regionReplicaSet) {
if (clients.get(dataNodeId) == null) {
buildClient(
dataNodeId,
- DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint());
+ DataNodeInfoPersistence.getInstance()
+ .getOnlineDataNode(dataNodeId)
+ .getInternalEndPoint());
}
TCreateSchemaRegionReq req = genCreateSchemaRegionReq(storageGroup, regionReplicaSet);
@@ -107,19 +109,25 @@ public class TemporaryClient {
if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
LOGGER.info(
"Create SchemaRegion on DataNode: {} success",
- DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint());
+ DataNodeInfoPersistence.getInstance()
+ .getOnlineDataNode(dataNodeId)
+ .getInternalEndPoint());
return;
} else {
LOGGER.error(
"Create SchemaRegion on DataNode: {} failed, {}. Retrying...",
- DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint(),
+ DataNodeInfoPersistence.getInstance()
+ .getOnlineDataNode(dataNodeId)
+ .getInternalEndPoint(),
status);
}
} catch (TException e) {
// TODO: Handler SocketTimeOutException
LOGGER.error(
"Create SchemaRegion on DataNode: {} failed, {}. Retrying...",
- DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint(),
+ DataNodeInfoPersistence.getInstance()
+ .getOnlineDataNode(dataNodeId)
+ .getInternalEndPoint(),
e.toString());
try {
TimeUnit.MILLISECONDS.sleep(retryWait);
@@ -131,25 +139,27 @@ public class TemporaryClient {
}
LOGGER.error(
"Create SchemaRegion on DataNode: {} failed.",
- DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint());
+ DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getInternalEndPoint());
}
private TCreateDataRegionReq genCreateDataRegionReq(
- String storageGroup, RegionReplicaSet regionReplicaSet, long TTL) {
+ String storageGroup, TRegionReplicaSet regionReplicaSet, long TTL) {
TCreateDataRegionReq req = new TCreateDataRegionReq();
req.setStorageGroup(storageGroup);
- req.setRegionReplicaSet(regionReplicaSet.convertToRPCTRegionReplicaSet());
+ req.setRegionReplicaSet(regionReplicaSet);
req.setTtl(TTL);
return req;
}
public void createDataRegion(
- int dataNodeId, String storageGroup, RegionReplicaSet regionReplicaSet, long TTL) {
+ int dataNodeId, String storageGroup, TRegionReplicaSet regionReplicaSet, long TTL) {
if (clients.get(dataNodeId) == null) {
buildClient(
dataNodeId,
- DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint());
+ DataNodeInfoPersistence.getInstance()
+ .getOnlineDataNode(dataNodeId)
+ .getInternalEndPoint());
}
TCreateDataRegionReq req = genCreateDataRegionReq(storageGroup, regionReplicaSet, TTL);
@@ -160,19 +170,25 @@ public class TemporaryClient {
if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
LOGGER.info(
"Create DataRegion on DataNode: {} success",
- DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint());
+ DataNodeInfoPersistence.getInstance()
+ .getOnlineDataNode(dataNodeId)
+ .getInternalEndPoint());
return;
} else {
LOGGER.error(
"Create DataRegion on DataNode: {} failed, {}. Retrying...",
- DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint(),
+ DataNodeInfoPersistence.getInstance()
+ .getOnlineDataNode(dataNodeId)
+ .getInternalEndPoint(),
status);
}
} catch (TException e) {
// TODO: Handler SocketTimeOutException
LOGGER.error(
"Create DataRegion on DataNode: {} failed, {}. Retrying...",
- DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint(),
+ DataNodeInfoPersistence.getInstance()
+ .getOnlineDataNode(dataNodeId)
+ .getInternalEndPoint(),
e.toString());
try {
TimeUnit.MILLISECONDS.sleep(retryWait);
@@ -184,7 +200,7 @@ public class TemporaryClient {
}
LOGGER.error(
"Create DataRegion on DataNode: {} failed.",
- DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getEndPoint());
+ DataNodeInfoPersistence.getInstance().getOnlineDataNode(dataNodeId).getInternalEndPoint());
}
private static class TemporaryClientHolder {
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConf.java b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConf.java
index 27a60ea35b..80279295a5 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConf.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConf.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.confignode.conf;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.rpc.RpcUtils;
@@ -43,8 +43,8 @@ public class ConfigNodeConf {
private String dataNodeConsensusProtocolClass = "org.apache.iotdb.consensus.ratis.RatisConsensus";
/** Used for building the ConfigNode consensus group */
- private Endpoint[] configNodeGroupAddressList =
- Collections.singletonList(new Endpoint("0.0.0.0", 22278)).toArray(new Endpoint[0]);
+ private TEndPoint[] configNodeGroupAddressList =
+ Collections.singletonList(new TEndPoint("0.0.0.0", 22278)).toArray(new TEndPoint[0]);
/** Number of SeriesPartitionSlots per StorageGroup */
private int seriesPartitionSlotNum = 10000;
@@ -90,12 +90,17 @@ public class ConfigNodeConf {
/** Default TTL for storage groups that are not set TTL by statements, in ms. */
private long defaultTTL = Long.MAX_VALUE;
- /** The number of replicas of each region */
- private int regionReplicaCount = 3;
- /** The number of SchemaRegions of each StorageGroup */
- private int schemaRegionCount = 1;
- /** The number of DataRegions of each StorageGroup */
- private int dataRegionCount = 1;
+ /** Default number of SchemaRegion replicas */
+ private int schemaReplicationFactor = 3;
+
+ /** Default number of DataRegion replicas */
+ private int dataReplicationFactor = 3;
+
+ /** The initial number of SchemaRegions of each StorageGroup */
+ private int initialSchemaRegionCount = 1;
+
+ /** The initial number of DataRegions of each StorageGroup */
+ private int initialDataRegionCount = 1;
public ConfigNodeConf() {
// empty constructor
@@ -237,11 +242,11 @@ public class ConfigNodeConf {
this.dataNodeConsensusProtocolClass = dataNodeConsensusProtocolClass;
}
- public Endpoint[] getConfigNodeGroupAddressList() {
+ public TEndPoint[] getConfigNodeGroupAddressList() {
return configNodeGroupAddressList;
}
- public void setConfigNodeGroupAddressList(Endpoint[] configNodeGroupAddressList) {
+ public void setConfigNodeGroupAddressList(TEndPoint[] configNodeGroupAddressList) {
this.configNodeGroupAddressList = configNodeGroupAddressList;
}
@@ -277,27 +282,35 @@ public class ConfigNodeConf {
this.defaultTTL = defaultTTL;
}
- public int getRegionReplicaCount() {
- return regionReplicaCount;
+ public int getSchemaReplicationFactor() {
+ return schemaReplicationFactor;
+ }
+
+ public void setSchemaReplicationFactor(int schemaReplicationFactor) {
+ this.schemaReplicationFactor = schemaReplicationFactor;
+ }
+
+ public int getDataReplicationFactor() {
+ return dataReplicationFactor;
}
- public void setDataRegionCount(int dataRegionCount) {
- this.dataRegionCount = dataRegionCount;
+ public void setDataReplicationFactor(int dataReplicationFactor) {
+ this.dataReplicationFactor = dataReplicationFactor;
}
- public int getSchemaRegionCount() {
- return schemaRegionCount;
+ public int getInitialSchemaRegionCount() {
+ return initialSchemaRegionCount;
}
- public void setSchemaRegionCount(int schemaRegionCount) {
- this.schemaRegionCount = schemaRegionCount;
+ public void setInitialSchemaRegionCount(int initialSchemaRegionCount) {
+ this.initialSchemaRegionCount = initialSchemaRegionCount;
}
- public int getDataRegionCount() {
- return dataRegionCount;
+ public int getInitialDataRegionCount() {
+ return initialDataRegionCount;
}
- public void setRegionReplicaCount(int regionReplicaCount) {
- this.regionReplicaCount = regionReplicaCount;
+ public void setInitialDataRegionCount(int initialDataRegionCount) {
+ this.initialDataRegionCount = initialDataRegionCount;
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
index f04d9d5f44..c42e81a70b 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.confignode.conf;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.commons.exception.BadNodeUrlException;
import org.apache.iotdb.commons.utils.CommonUtils;
@@ -174,25 +174,31 @@ public class ConfigNodeDescriptor {
Long.parseLong(
properties.getProperty("default_ttl", String.valueOf(conf.getDefaultTTL()))));
- conf.setRegionReplicaCount(
+ conf.setSchemaReplicationFactor(
Integer.parseInt(
properties.getProperty(
- "region_replica_count", String.valueOf(conf.getRegionReplicaCount()))));
+ "schema_replication_factor", String.valueOf(conf.getSchemaReplicationFactor()))));
- conf.setSchemaRegionCount(
+ conf.setDataReplicationFactor(
Integer.parseInt(
properties.getProperty(
- "schema_region_count", String.valueOf(conf.getSchemaRegionCount()))));
+ "data_replication_factor", String.valueOf(conf.getDataReplicationFactor()))));
- conf.setDataRegionCount(
+ conf.setInitialSchemaRegionCount(
Integer.parseInt(
properties.getProperty(
- "data_region_count", String.valueOf(conf.getDataRegionCount()))));
+ "initial_schema_region_count",
+ String.valueOf(conf.getInitialSchemaRegionCount()))));
+
+ conf.setInitialDataRegionCount(
+ Integer.parseInt(
+ properties.getProperty(
+ "initial_data_region_count", String.valueOf(conf.getInitialDataRegionCount()))));
String addresses = properties.getProperty("config_node_group_address_list", "0.0.0.0:22278");
String[] addressList = addresses.split(",");
- Endpoint[] endpointList = new Endpoint[addressList.length];
+ TEndPoint[] endpointList = new TEndPoint[addressList.length];
for (int i = 0; i < addressList.length; i++) {
endpointList[i] = CommonUtils.parseNodeUrl(addressList[i]);
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeConfigurationDataSet.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeConfigurationDataSet.java
index deb1a07ead..178210be01 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeConfigurationDataSet.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeConfigurationDataSet.java
@@ -43,10 +43,6 @@ public class DataNodeConfigurationDataSet implements DataSet {
this.status = status;
}
- public Integer getDataNodeId() {
- return dataNodeId;
- }
-
public void setDataNodeId(int dataNodeId) {
this.dataNodeId = dataNodeId;
}
@@ -59,7 +55,7 @@ public class DataNodeConfigurationDataSet implements DataSet {
resp.setStatus(status);
if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()
|| status.getCode() == TSStatusCode.DATANODE_ALREADY_REGISTERED.getStatusCode()) {
- resp.setDataNodeID(dataNodeId);
+ resp.setDataNodeId(dataNodeId);
resp.setGlobalConfig(globalConfig);
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodesInfoDataSet.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeLocationsDataSet.java
similarity index 55%
rename from confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodesInfoDataSet.java
rename to confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeLocationsDataSet.java
index b4807cde92..db23bbc4b9 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodesInfoDataSet.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataNodeLocationsDataSet.java
@@ -18,24 +18,20 @@
*/
package org.apache.iotdb.confignode.consensus.response;
-import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessage;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessageResp;
+import org.apache.iotdb.confignode.rpc.thrift.TDataNodeLocationResp;
import org.apache.iotdb.consensus.common.DataSet;
import org.apache.iotdb.rpc.TSStatusCode;
-import java.util.HashMap;
-import java.util.List;
import java.util.Map;
-public class DataNodesInfoDataSet implements DataSet {
+public class DataNodeLocationsDataSet implements DataSet {
private TSStatus status;
- private List<DataNodeLocation> dataNodeList;
+ private Map<Integer, TDataNodeLocation> dataNodeLocationMap;
- public DataNodesInfoDataSet() {
+ public DataNodeLocationsDataSet() {
// empty constructor
}
@@ -47,26 +43,14 @@ public class DataNodesInfoDataSet implements DataSet {
return status;
}
- public void setDataNodeList(List<DataNodeLocation> dataNodeList) {
- this.dataNodeList = dataNodeList;
+ public void setDataNodeLocations(Map<Integer, TDataNodeLocation> dataNodeLocationMap) {
+ this.dataNodeLocationMap = dataNodeLocationMap;
}
- public List<DataNodeLocation> getDataNodeList() {
- return this.dataNodeList;
- }
-
- public void convertToRPCDataNodeMessageResp(TDataNodeMessageResp resp) {
+ public void convertToRpcDataNodeLocationResp(TDataNodeLocationResp resp) {
resp.setStatus(status);
if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
- Map<Integer, TDataNodeMessage> msgMap = new HashMap<>();
- for (DataNodeLocation info : dataNodeList) {
- msgMap.put(
- info.getDataNodeId(),
- new TDataNodeMessage(
- info.getDataNodeId(),
- new TEndPoint(info.getEndPoint().getIp(), info.getEndPoint().getPort())));
- resp.setDataNodeMessageMap(msgMap);
- }
+ resp.setDataNodeLocationMap(dataNodeLocationMap);
}
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataPartitionDataSet.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataPartitionDataSet.java
index 324badfe37..15da627333 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataPartitionDataSet.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/DataPartitionDataSet.java
@@ -19,20 +19,12 @@
package org.apache.iotdb.confignode.consensus.response;
-import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
-import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.commons.partition.DataPartition;
import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionResp;
import org.apache.iotdb.consensus.common.DataSet;
import org.apache.iotdb.rpc.TSStatusCode;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
public class DataPartitionDataSet implements DataSet {
private TSStatus status;
@@ -61,53 +53,10 @@ public class DataPartitionDataSet implements DataSet {
* @param resp TDataPartitionResp
*/
public void convertToRpcDataPartitionResp(TDataPartitionResp resp) {
- Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
- dataPartitionMap = new HashMap<>();
resp.setStatus(status);
if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
- dataPartition
- .getDataPartitionMap()
- .forEach(
- ((storageGroup, seriesPartitionSlotTimePartitionSlotRegionReplicaSetListMap) -> {
- // Extract StorageGroupName
- dataPartitionMap.putIfAbsent(storageGroup, new HashMap<>());
-
- seriesPartitionSlotTimePartitionSlotRegionReplicaSetListMap.forEach(
- ((seriesPartitionSlot, timePartitionSlotReplicaSetListMap) -> {
- // Extract TSeriesPartitionSlot
- TSeriesPartitionSlot tSeriesPartitionSlot =
- new TSeriesPartitionSlot(seriesPartitionSlot.getSlotId());
- dataPartitionMap
- .get(storageGroup)
- .putIfAbsent(tSeriesPartitionSlot, new HashMap<>());
-
- timePartitionSlotReplicaSetListMap.forEach(
- ((timePartitionSlot, regionReplicaSets) -> {
- // Extract TTimePartitionSlot
- TTimePartitionSlot tTimePartitionSlot =
- new TTimePartitionSlot(timePartitionSlot.getStartTime());
- dataPartitionMap
- .get(storageGroup)
- .get(tSeriesPartitionSlot)
- .putIfAbsent(tTimePartitionSlot, new ArrayList<>());
-
- // Extract TRegionReplicaSets
- regionReplicaSets.forEach(
- regionReplicaSet -> {
- TRegionReplicaSet tRegionReplicaSet =
- regionReplicaSet.convertToRPCTRegionReplicaSet();
- dataPartitionMap
- .get(storageGroup)
- .get(tSeriesPartitionSlot)
- .get(tTimePartitionSlot)
- .add(tRegionReplicaSet);
- });
- }));
- }));
- }));
-
- resp.setDataPartitionMap(dataPartitionMap);
+ resp.setDataPartitionMap(dataPartition.getDataPartitionMap());
}
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/SchemaPartitionDataSet.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/SchemaPartitionDataSet.java
index 4955eebbf5..5ed93b7c05 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/SchemaPartitionDataSet.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/SchemaPartitionDataSet.java
@@ -19,17 +19,12 @@
package org.apache.iotdb.confignode.consensus.response;
-import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
import org.apache.iotdb.commons.partition.SchemaPartition;
import org.apache.iotdb.confignode.rpc.thrift.TSchemaPartitionResp;
import org.apache.iotdb.consensus.common.DataSet;
import org.apache.iotdb.rpc.TSStatusCode;
-import java.util.HashMap;
-import java.util.Map;
-
public class SchemaPartitionDataSet implements DataSet {
private TSStatus status;
@@ -53,34 +48,10 @@ public class SchemaPartitionDataSet implements DataSet {
}
public void convertToRpcSchemaPartitionResp(TSchemaPartitionResp resp) {
- Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap = new HashMap<>();
resp.setStatus(status);
if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
- schemaPartition
- .getSchemaPartitionMap()
- .forEach(
- (storageGroup, seriesPartitionSlotRegionReplicaSetMap) -> {
- // Extract StorageGroupName
- schemaPartitionMap.putIfAbsent(storageGroup, new HashMap<>());
-
- // Extract Map<SeriesPartitionSlot, RegionReplicaSet>
- seriesPartitionSlotRegionReplicaSetMap.forEach(
- ((seriesPartitionSlot, regionReplicaSet) -> {
- // Extract TSeriesPartitionSlot
- TSeriesPartitionSlot tSeriesPartitionSlot =
- new TSeriesPartitionSlot(seriesPartitionSlot.getSlotId());
-
- // Extract TRegionReplicaSet
- TRegionReplicaSet tRegionReplicaSet =
- regionReplicaSet.convertToRPCTRegionReplicaSet();
- schemaPartitionMap
- .get(storageGroup)
- .put(tSeriesPartitionSlot, tRegionReplicaSet);
- }));
- });
+ resp.setSchemaRegionMap(schemaPartition.getSchemaPartitionMap());
}
-
- resp.setSchemaRegionMap(schemaPartitionMap);
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/StorageGroupSchemaDataSet.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/StorageGroupSchemaDataSet.java
index 48e0c181ce..0874de13e3 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/StorageGroupSchemaDataSet.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/StorageGroupSchemaDataSet.java
@@ -19,7 +19,6 @@
package org.apache.iotdb.confignode.consensus.response;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.confignode.partition.StorageGroupSchema;
import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchemaResp;
import org.apache.iotdb.consensus.common.DataSet;
@@ -33,7 +32,7 @@ public class StorageGroupSchemaDataSet implements DataSet {
private TSStatus status;
- private List<StorageGroupSchema> schemaList;
+ private List<TStorageGroupSchema> schemaList;
public StorageGroupSchemaDataSet() {}
@@ -45,11 +44,11 @@ public class StorageGroupSchemaDataSet implements DataSet {
this.status = status;
}
- public List<StorageGroupSchema> getSchemaList() {
+ public List<TStorageGroupSchema> getSchemaList() {
return schemaList;
}
- public void setSchemaList(List<StorageGroupSchema> schemaList) {
+ public void setSchemaList(List<TStorageGroupSchema> schemaList) {
this.schemaList = schemaList;
}
@@ -57,8 +56,8 @@ public class StorageGroupSchemaDataSet implements DataSet {
resp.setStatus(status);
if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
Map<String, TStorageGroupSchema> storageGroupMessageMap = new HashMap<>();
- for (StorageGroupSchema schema : schemaList) {
- storageGroupMessageMap.put(schema.getName(), new TStorageGroupSchema(schema.getName()));
+ for (TStorageGroupSchema schema : schemaList) {
+ storageGroupMessageMap.put(schema.getName(), schema);
}
resp.setStorageGroupSchemaMap(storageGroupMessageMap);
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
index 43db8e07eb..5ac486eee4 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
@@ -20,9 +20,9 @@
package org.apache.iotdb.confignode.manager;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
import org.apache.iotdb.confignode.consensus.response.DataNodeConfigurationDataSet;
-import org.apache.iotdb.confignode.consensus.response.DataNodesInfoDataSet;
+import org.apache.iotdb.confignode.consensus.response.DataNodeLocationsDataSet;
import org.apache.iotdb.confignode.consensus.response.DataPartitionDataSet;
import org.apache.iotdb.confignode.consensus.response.PermissionInfoDataSet;
import org.apache.iotdb.confignode.consensus.response.SchemaPartitionDataSet;
@@ -102,7 +102,7 @@ public class ConfigManager implements Manager {
if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
return dataNodeManager.getDataNodeInfo((QueryDataNodeInfoPlan) physicalPlan);
} else {
- DataNodesInfoDataSet dataSet = new DataNodesInfoDataSet();
+ DataNodeLocationsDataSet dataSet = new DataNodeLocationsDataSet();
dataSet.setStatus(status);
return dataSet;
}
@@ -139,7 +139,7 @@ public class ConfigManager implements Manager {
GetOrCreateSchemaPartitionPlan getSchemaPartitionPlan =
new GetOrCreateSchemaPartitionPlan(PhysicalPlanType.GetSchemaPartition);
- Map<String, List<SeriesPartitionSlot>> partitionSlotsMap = new HashMap<>();
+ Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap = new HashMap<>();
boolean getAll = false;
Set<String> getAllSet = new HashSet<>();
@@ -196,7 +196,7 @@ public class ConfigManager implements Manager {
GetOrCreateSchemaPartitionPlan getOrCreateSchemaPartitionPlan =
new GetOrCreateSchemaPartitionPlan(PhysicalPlanType.GetOrCreateSchemaPartition);
- Map<String, List<SeriesPartitionSlot>> partitionSlotsMap = new HashMap<>();
+ Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap = new HashMap<>();
for (String devicePath : devicePaths) {
if (!devicePath.contains("*")) {
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java
index 09d1223df7..0058416429 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.confignode.manager;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.commons.consensus.ConsensusGroupId;
import org.apache.iotdb.commons.consensus.PartitionRegionId;
import org.apache.iotdb.confignode.conf.ConfigNodeConf;
@@ -65,7 +65,7 @@ public class ConsensusManager {
consensusImpl =
ConsensusFactory.getConsensusImpl(
conf.getConfigNodeConsensusProtocolClass(),
- new Endpoint(conf.getRpcAddress(), conf.getInternalPort()),
+ new TEndPoint(conf.getRpcAddress(), conf.getInternalPort()),
new File(conf.getConsensusDir()),
gid -> new PartitionRegionStateMachine())
.orElseThrow(
@@ -81,7 +81,7 @@ public class ConsensusManager {
"Set ConfigNode consensus group {}...",
Arrays.toString(conf.getConfigNodeGroupAddressList()));
List<Peer> peerList = new ArrayList<>();
- for (Endpoint endpoint : conf.getConfigNodeGroupAddressList()) {
+ for (TEndPoint endpoint : conf.getConfigNodeGroupAddressList()) {
peerList.add(new Peer(consensusGroupId, endpoint));
}
consensusImpl.addConsensusGroup(consensusGroupId, peerList);
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/DataNodeManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/DataNodeManager.java
index 21c2734ee7..6f28782cd1 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/DataNodeManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/DataNodeManager.java
@@ -18,11 +18,11 @@
*/
package org.apache.iotdb.confignode.manager;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
import org.apache.iotdb.confignode.consensus.response.DataNodeConfigurationDataSet;
-import org.apache.iotdb.confignode.consensus.response.DataNodesInfoDataSet;
+import org.apache.iotdb.confignode.consensus.response.DataNodeLocationsDataSet;
import org.apache.iotdb.confignode.persistence.DataNodeInfoPersistence;
import org.apache.iotdb.confignode.physical.sys.QueryDataNodeInfoPlan;
import org.apache.iotdb.confignode.physical.sys.RegisterDataNodePlan;
@@ -78,18 +78,19 @@ public class DataNodeManager {
public DataSet registerDataNode(RegisterDataNodePlan plan) {
DataNodeConfigurationDataSet dataSet = new DataNodeConfigurationDataSet();
- if (DataNodeInfoPersistence.getInstance().containsValue(plan.getInfo())) {
+ if (DataNodeInfoPersistence.getInstance().containsValue(plan.getLocation())) {
TSStatus status = new TSStatus(TSStatusCode.DATANODE_ALREADY_REGISTERED.getStatusCode());
status.setMessage("DataNode already registered.");
dataSet.setStatus(status);
} else {
// Persist DataNodeInfo
- plan.getInfo().setDataNodeId(DataNodeInfoPersistence.getInstance().generateNextDataNodeId());
+ plan.getLocation()
+ .setDataNodeId(DataNodeInfoPersistence.getInstance().generateNextDataNodeId());
ConsensusWriteResponse resp = getConsensusManager().write(plan);
dataSet.setStatus(resp.getStatus());
}
- dataSet.setDataNodeId(plan.getInfo().getDataNodeId());
+ dataSet.setDataNodeId(plan.getLocation().getDataNodeId());
setGlobalConfig(dataSet);
return dataSet;
}
@@ -101,15 +102,15 @@ public class DataNodeManager {
* @return The specific DataNode's info or all DataNode info if dataNodeId in
* QueryDataNodeInfoPlan is -1
*/
- public DataNodesInfoDataSet getDataNodeInfo(QueryDataNodeInfoPlan plan) {
- return (DataNodesInfoDataSet) getConsensusManager().read(plan).getDataset();
+ public DataNodeLocationsDataSet getDataNodeInfo(QueryDataNodeInfoPlan plan) {
+ return (DataNodeLocationsDataSet) getConsensusManager().read(plan).getDataset();
}
public int getOnlineDataNodeCount() {
return dataNodeInfoPersistence.getOnlineDataNodeCount();
}
- public List<DataNodeLocation> getOnlineDataNodes() {
+ public List<TDataNodeLocation> getOnlineDataNodes() {
return dataNodeInfoPersistence.getOnlineDataNodes();
}
@@ -138,12 +139,12 @@ public class DataNodeManager {
}
@Override
- public void addDataNode(DataNodeLocation DataNodeInfo) {
+ public void addDataNode(TDataNodeLocation DataNodeInfo) {
serverChanged();
}
@Override
- public void removeDataNode(DataNodeLocation dataNodeInfo) {
+ public void removeDataNode(TDataNodeLocation dataNodeInfo) {
serverChanged();
}
@@ -169,13 +170,13 @@ public class DataNodeManager {
*
* @param dataNodeInfo datanode info
*/
- void addDataNode(final DataNodeLocation dataNodeInfo);
+ void addDataNode(final TDataNodeLocation dataNodeInfo);
/**
* remove data node
*
* @param dataNodeInfo data node info
*/
- void removeDataNode(final DataNodeLocation dataNodeInfo);
+ void removeDataNode(final TDataNodeLocation dataNodeInfo);
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/PartitionManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/PartitionManager.java
index 99f07bc31b..607e4bba0e 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/PartitionManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/PartitionManager.java
@@ -18,9 +18,9 @@
*/
package org.apache.iotdb.confignode.manager;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
import org.apache.iotdb.confignode.conf.ConfigNodeConf;
import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
@@ -85,13 +85,13 @@ public class PartitionManager {
* @return SchemaPartitionDataSet
*/
public DataSet getOrCreateSchemaPartition(GetOrCreateSchemaPartitionPlan physicalPlan) {
- Map<String, List<SeriesPartitionSlot>> noAssignedSchemaPartitionSlots =
+ Map<String, List<TSeriesPartitionSlot>> noAssignedSchemaPartitionSlots =
partitionInfoPersistence.filterNoAssignedSchemaPartitionSlots(
physicalPlan.getPartitionSlotsMap());
if (noAssignedSchemaPartitionSlots.size() > 0) {
// Allocate SchemaPartition
- Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> assignedSchemaPartition =
+ Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> assignedSchemaPartition =
allocateSchemaPartition(noAssignedSchemaPartitionSlots);
// Persist SchemaPartition
@@ -109,18 +109,18 @@ public class PartitionManager {
* @param noAssignedSchemaPartitionSlotsMap Map<StorageGroupName, List<SeriesPartitionSlot>>
* @return assign result, Map<StorageGroupName, Map<SeriesPartitionSlot, RegionReplicaSet>>
*/
- private Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> allocateSchemaPartition(
- Map<String, List<SeriesPartitionSlot>> noAssignedSchemaPartitionSlotsMap) {
- Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> result = new HashMap<>();
+ private Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> allocateSchemaPartition(
+ Map<String, List<TSeriesPartitionSlot>> noAssignedSchemaPartitionSlotsMap) {
+ Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> result = new HashMap<>();
for (String storageGroup : noAssignedSchemaPartitionSlotsMap.keySet()) {
- List<SeriesPartitionSlot> noAssignedPartitionSlots =
+ List<TSeriesPartitionSlot> noAssignedPartitionSlots =
noAssignedSchemaPartitionSlotsMap.get(storageGroup);
- List<RegionReplicaSet> schemaRegionEndPoints =
+ List<TRegionReplicaSet> schemaRegionEndPoints =
RegionInfoPersistence.getInstance().getSchemaRegionEndPoint(storageGroup);
Random random = new Random();
- Map<SeriesPartitionSlot, RegionReplicaSet> allocateResult = new HashMap<>();
+ Map<TSeriesPartitionSlot, TRegionReplicaSet> allocateResult = new HashMap<>();
noAssignedPartitionSlots.forEach(
seriesPartitionSlot ->
allocateResult.put(
@@ -155,13 +155,13 @@ public class PartitionManager {
* @return DataPartitionDataSet
*/
public DataSet getOrCreateDataPartition(GetOrCreateDataPartitionPlan physicalPlan) {
- Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> noAssignedDataPartitionSlots =
+ Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> noAssignedDataPartitionSlots =
partitionInfoPersistence.filterNoAssignedDataPartitionSlots(
physicalPlan.getPartitionSlotsMap());
if (noAssignedDataPartitionSlots.size() > 0) {
// Allocate DataPartition
- Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+ Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
assignedDataPartition = allocateDataPartition(noAssignedDataPartitionSlots);
// Persist DataPartition
@@ -181,27 +181,27 @@ public class PartitionManager {
* @return assign result, Map<StorageGroupName, Map<SeriesPartitionSlot, Map<TimePartitionSlot,
* List<RegionReplicaSet>>>>
*/
- private Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+ private Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
allocateDataPartition(
- Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>>
+ Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>>
noAssignedDataPartitionSlotsMap) {
- Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>> result =
- new HashMap<>();
+ Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
+ result = new HashMap<>();
for (String storageGroup : noAssignedDataPartitionSlotsMap.keySet()) {
- Map<SeriesPartitionSlot, List<TimePartitionSlot>> noAssignedPartitionSlotsMap =
+ Map<TSeriesPartitionSlot, List<TTimePartitionSlot>> noAssignedPartitionSlotsMap =
noAssignedDataPartitionSlotsMap.get(storageGroup);
- List<RegionReplicaSet> dataRegionEndPoints =
+ List<TRegionReplicaSet> dataRegionEndPoints =
RegionInfoPersistence.getInstance().getDataRegionEndPoint(storageGroup);
Random random = new Random();
- Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>> allocateResult =
+ Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>> allocateResult =
new HashMap<>();
- for (Map.Entry<SeriesPartitionSlot, List<TimePartitionSlot>> seriesPartitionEntry :
+ for (Map.Entry<TSeriesPartitionSlot, List<TTimePartitionSlot>> seriesPartitionEntry :
noAssignedPartitionSlotsMap.entrySet()) {
allocateResult.put(seriesPartitionEntry.getKey(), new HashMap<>());
- for (TimePartitionSlot timePartitionSlot : seriesPartitionEntry.getValue()) {
+ for (TTimePartitionSlot timePartitionSlot : seriesPartitionEntry.getValue()) {
allocateResult
.get(seriesPartitionEntry.getKey())
.computeIfAbsent(timePartitionSlot, key -> new ArrayList<>())
@@ -223,12 +223,12 @@ public class PartitionManager {
}
/**
- * Get SeriesPartitionSlot
+ * Get TSeriesPartitionSlot
*
* @param devicePath Full path ending with device name
* @return SeriesPartitionSlot
*/
- public SeriesPartitionSlot getSeriesPartitionSlot(String devicePath) {
+ public TSeriesPartitionSlot getSeriesPartitionSlot(String devicePath) {
return executor.getSeriesPartitionSlot(devicePath);
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/RegionManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/RegionManager.java
index fa98266fc0..dfbacf4b2e 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/RegionManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/RegionManager.java
@@ -19,13 +19,11 @@
package org.apache.iotdb.confignode.manager;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.consensus.ConsensusGroupId;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.consensus.GroupType;
-import org.apache.iotdb.commons.consensus.SchemaRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
import org.apache.iotdb.confignode.cli.TemporaryClient;
import org.apache.iotdb.confignode.conf.ConfigNodeConf;
import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
@@ -44,9 +42,10 @@ import java.util.List;
public class RegionManager {
private static final ConfigNodeConf conf = ConfigNodeDescriptor.getInstance().getConf();
- private static final int regionReplicaCount = conf.getRegionReplicaCount();
- private static final int schemaRegionCount = conf.getSchemaRegionCount();
- private static final int dataRegionCount = conf.getDataRegionCount();
+ private static final int schemaReplicationFactor = conf.getSchemaReplicationFactor();
+ private static final int dataReplicationFactor = conf.getDataReplicationFactor();
+ private static final int initialSchemaRegionCount = conf.getInitialSchemaRegionCount();
+ private static final int initialDataRegionCount = conf.getInitialDataRegionCount();
private static final RegionInfoPersistence regionInfoPersistence =
RegionInfoPersistence.getInstance();
@@ -71,7 +70,8 @@ public class RegionManager {
*/
public TSStatus setStorageGroup(SetStorageGroupPlan plan) {
TSStatus result;
- if (configNodeManager.getDataNodeManager().getOnlineDataNodeCount() < regionReplicaCount) {
+ if (configNodeManager.getDataNodeManager().getOnlineDataNodeCount()
+ < Math.max(initialSchemaRegionCount, initialDataRegionCount)) {
result = new TSStatus(TSStatusCode.NOT_ENOUGH_DATA_NODE.getStatusCode());
result.setMessage("DataNode is not enough, please register more.");
} else {
@@ -84,29 +84,31 @@ public class RegionManager {
createPlan.setStorageGroup(plan.getSchema().getName());
// Allocate default Regions
- allocateRegions(GroupType.SchemaRegion, createPlan);
- allocateRegions(GroupType.DataRegion, createPlan);
+ allocateRegions(TConsensusGroupType.SchemaRegion, createPlan);
+ allocateRegions(TConsensusGroupType.DataRegion, createPlan);
// Persist StorageGroup and Regions
getConsensusManager().write(plan);
result = getConsensusManager().write(createPlan).getStatus();
// Create Regions in DataNode
- for (RegionReplicaSet regionReplicaSet : createPlan.getRegionReplicaSets()) {
- for (DataNodeLocation dataNodeLocation : regionReplicaSet.getDataNodeList()) {
- if (regionReplicaSet.getConsensusGroupId() instanceof SchemaRegionId) {
- TemporaryClient.getInstance()
- .createSchemaRegion(
- dataNodeLocation.getDataNodeId(),
- createPlan.getStorageGroup(),
- regionReplicaSet);
- } else if (regionReplicaSet.getConsensusGroupId() instanceof DataRegionId) {
- TemporaryClient.getInstance()
- .createDataRegion(
- dataNodeLocation.getDataNodeId(),
- createPlan.getStorageGroup(),
- regionReplicaSet,
- plan.getSchema().getTTL());
+ for (TRegionReplicaSet regionReplicaSet : createPlan.getRegionReplicaSets()) {
+ for (TDataNodeLocation dataNodeLocation : regionReplicaSet.getDataNodeLocations()) {
+ switch (regionReplicaSet.getRegionId().getType()) {
+ case SchemaRegion:
+ TemporaryClient.getInstance()
+ .createSchemaRegion(
+ dataNodeLocation.getDataNodeId(),
+ createPlan.getStorageGroup(),
+ regionReplicaSet);
+ break;
+ case DataRegion:
+ TemporaryClient.getInstance()
+ .createDataRegion(
+ dataNodeLocation.getDataNodeId(),
+ createPlan.getStorageGroup(),
+ regionReplicaSet,
+ plan.getSchema().getTTL());
}
}
}
@@ -119,26 +121,27 @@ public class RegionManager {
return configNodeManager.getDataNodeManager();
}
- private void allocateRegions(GroupType type, CreateRegionsPlan plan) {
+ private void allocateRegions(TConsensusGroupType type, CreateRegionsPlan plan) {
// TODO: Use CopySet algorithm to optimize region allocation policy
- int regionCount = type.equals(GroupType.SchemaRegion) ? schemaRegionCount : dataRegionCount;
- List<DataNodeLocation> onlineDataNodes = getDataNodeInfoManager().getOnlineDataNodes();
+ int replicaCount =
+ type.equals(TConsensusGroupType.SchemaRegion)
+ ? schemaReplicationFactor
+ : dataReplicationFactor;
+ int regionCount =
+ type.equals(TConsensusGroupType.SchemaRegion)
+ ? initialSchemaRegionCount
+ : initialDataRegionCount;
+ List<TDataNodeLocation> onlineDataNodes = getDataNodeInfoManager().getOnlineDataNodes();
for (int i = 0; i < regionCount; i++) {
Collections.shuffle(onlineDataNodes);
- RegionReplicaSet regionReplicaSet = new RegionReplicaSet();
- ConsensusGroupId consensusGroupId = null;
- switch (type) {
- case SchemaRegion:
- consensusGroupId = new SchemaRegionId(regionInfoPersistence.generateNextRegionGroupId());
- break;
- case DataRegion:
- consensusGroupId = new DataRegionId(regionInfoPersistence.generateNextRegionGroupId());
- }
- regionReplicaSet.setConsensusGroupId(consensusGroupId);
- regionReplicaSet.setDataNodeList(onlineDataNodes.subList(0, regionReplicaCount));
+ TRegionReplicaSet regionReplicaSet = new TRegionReplicaSet();
+ TConsensusGroupId consensusGroupId =
+ new TConsensusGroupId(type, regionInfoPersistence.generateNextRegionGroupId());
+ regionReplicaSet.setRegionId(consensusGroupId);
+ regionReplicaSet.setDataNodeLocations(onlineDataNodes.subList(0, replicaCount));
plan.addRegion(regionReplicaSet);
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/partition/StorageGroupSchema.java b/confignode/src/main/java/org/apache/iotdb/confignode/partition/StorageGroupSchema.java
deleted file mode 100644
index e128a2092c..0000000000
--- a/confignode/src/main/java/org/apache/iotdb/confignode/partition/StorageGroupSchema.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.confignode.partition;
-
-import org.apache.iotdb.commons.consensus.ConsensusGroupId;
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-
-public class StorageGroupSchema {
-
- private String name;
- private long TTL;
-
- private final List<ConsensusGroupId> schemaRegionGroupIds;
- private final List<ConsensusGroupId> dataRegionGroupIds;
-
- public StorageGroupSchema() {
- schemaRegionGroupIds = new ArrayList<>();
- dataRegionGroupIds = new ArrayList<>();
- }
-
- public StorageGroupSchema(String name) {
- this();
- this.name = name;
- }
-
- public String getName() {
- return name;
- }
-
- public long getTTL() {
- return TTL;
- }
-
- public void setTTL(long TTL) {
- this.TTL = TTL;
- }
-
- public List<ConsensusGroupId> getSchemaRegionGroupIds() {
- return schemaRegionGroupIds;
- }
-
- public void addSchemaRegionGroup(ConsensusGroupId id) {
- schemaRegionGroupIds.add(id);
- }
-
- public List<ConsensusGroupId> getDataRegionGroupIds() {
- return dataRegionGroupIds;
- }
-
- public void addDataRegionGroup(ConsensusGroupId id) {
- dataRegionGroupIds.add(id);
- }
-
- public void serialize(ByteBuffer buffer) {
- buffer.putInt(name.length());
- buffer.put(name.getBytes());
-
- buffer.putInt(schemaRegionGroupIds.size());
- for (ConsensusGroupId schemaRegionGroupId : schemaRegionGroupIds) {
- schemaRegionGroupId.serializeImpl(buffer);
- }
-
- buffer.putInt(dataRegionGroupIds.size());
- for (ConsensusGroupId dataRegionGroupId : dataRegionGroupIds) {
- dataRegionGroupId.serializeImpl(buffer);
- }
- }
-
- public void deserialize(ByteBuffer buffer) throws IOException {
- name = SerializeDeserializeUtil.readString(buffer);
-
- int length = buffer.getInt();
- for (int i = 0; i < length; i++) {
- ConsensusGroupId schemaRegionId = ConsensusGroupId.Factory.create(buffer);
- schemaRegionGroupIds.add(schemaRegionId);
- }
-
- length = buffer.getInt();
- for (int i = 0; i < length; i++) {
- ConsensusGroupId dataRegionId = ConsensusGroupId.Factory.create(buffer);
- dataRegionGroupIds.add(dataRegionId);
- }
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- StorageGroupSchema that = (StorageGroupSchema) o;
- return name.equals(that.name)
- && schemaRegionGroupIds.equals(that.schemaRegionGroupIds)
- && dataRegionGroupIds.equals(that.dataRegionGroupIds);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(name, schemaRegionGroupIds, dataRegionGroupIds);
- }
-}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/DataNodeInfoPersistence.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/DataNodeInfoPersistence.java
index c1a64bb2f8..d09b3dc740 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/DataNodeInfoPersistence.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/DataNodeInfoPersistence.java
@@ -18,16 +18,17 @@
*/
package org.apache.iotdb.confignode.persistence;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
import org.apache.iotdb.commons.utils.TestOnly;
-import org.apache.iotdb.confignode.consensus.response.DataNodesInfoDataSet;
+import org.apache.iotdb.confignode.consensus.response.DataNodeLocationsDataSet;
import org.apache.iotdb.confignode.physical.sys.QueryDataNodeInfoPlan;
import org.apache.iotdb.confignode.physical.sys.RegisterDataNodePlan;
import org.apache.iotdb.rpc.TSStatusCode;
import java.util.ArrayList;
import java.util.Collections;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -45,25 +46,25 @@ public class DataNodeInfoPersistence {
/** online data nodes */
// TODO: serialize and deserialize
- private final ConcurrentNavigableMap<Integer, DataNodeLocation> onlineDataNodes =
+ private final ConcurrentNavigableMap<Integer, TDataNodeLocation> onlineDataNodes =
new ConcurrentSkipListMap();
/** For remove node or draining node */
- private final Set<DataNodeLocation> drainingDataNodes = new HashSet<>();
+ private final Set<TDataNodeLocation> drainingDataNodes = new HashSet<>();
private DataNodeInfoPersistence() {
this.dataNodeInfoReadWriteLock = new ReentrantReadWriteLock();
}
- public boolean containsValue(DataNodeLocation info) {
+ public boolean containsValue(TDataNodeLocation info) {
boolean result = false;
dataNodeInfoReadWriteLock.readLock().lock();
try {
- for (Map.Entry<Integer, DataNodeLocation> entry : onlineDataNodes.entrySet()) {
- if (entry.getValue().getEndPoint().equals(info.getEndPoint())) {
+ for (Map.Entry<Integer, TDataNodeLocation> entry : onlineDataNodes.entrySet()) {
+ info.setDataNodeId(entry.getKey());
+ if (entry.getValue().equals(info)) {
result = true;
- info.setDataNodeId(entry.getKey());
break;
}
}
@@ -74,7 +75,7 @@ public class DataNodeInfoPersistence {
return result;
}
- public void put(int dataNodeID, DataNodeLocation info) {
+ public void put(int dataNodeID, TDataNodeLocation info) {
onlineDataNodes.put(dataNodeID, info);
}
@@ -86,7 +87,7 @@ public class DataNodeInfoPersistence {
*/
public TSStatus registerDataNode(RegisterDataNodePlan plan) {
TSStatus result;
- DataNodeLocation info = plan.getInfo();
+ TDataNodeLocation info = plan.getLocation();
dataNodeInfoReadWriteLock.writeLock().lock();
try {
nextDataNodeId = Math.max(nextDataNodeId, info.getDataNodeId());
@@ -105,17 +106,19 @@ public class DataNodeInfoPersistence {
* @return The specific DataNode's info or all DataNode info if dataNodeId in
* QueryDataNodeInfoPlan is -1
*/
- public DataNodesInfoDataSet getDataNodeInfo(QueryDataNodeInfoPlan plan) {
- DataNodesInfoDataSet result = new DataNodesInfoDataSet();
+ public DataNodeLocationsDataSet getDataNodeInfo(QueryDataNodeInfoPlan plan) {
+ DataNodeLocationsDataSet result = new DataNodeLocationsDataSet();
result.setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()));
int dataNodeId = plan.getDataNodeID();
dataNodeInfoReadWriteLock.readLock().lock();
try {
if (dataNodeId == -1) {
- result.setDataNodeList(new ArrayList<>(onlineDataNodes.values()));
+ result.setDataNodeLocations(new HashMap<>(onlineDataNodes));
} else {
- result.setDataNodeList(Collections.singletonList(onlineDataNodes.get(dataNodeId)));
+
+ result.setDataNodeLocations(
+ Collections.singletonMap(dataNodeId, onlineDataNodes.get(dataNodeId)));
}
} finally {
dataNodeInfoReadWriteLock.readLock().unlock();
@@ -135,8 +138,8 @@ public class DataNodeInfoPersistence {
return result;
}
- public List<DataNodeLocation> getOnlineDataNodes() {
- List<DataNodeLocation> result;
+ public List<TDataNodeLocation> getOnlineDataNodes() {
+ List<TDataNodeLocation> result;
dataNodeInfoReadWriteLock.readLock().lock();
try {
result = new ArrayList<>(onlineDataNodes.values());
@@ -146,8 +149,8 @@ public class DataNodeInfoPersistence {
return result;
}
- public DataNodeLocation getOnlineDataNode(int dataNodeId) {
- DataNodeLocation result;
+ public TDataNodeLocation getOnlineDataNode(int dataNodeId) {
+ TDataNodeLocation result;
dataNodeInfoReadWriteLock.readLock().lock();
try {
result = onlineDataNodes.get(dataNodeId);
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/PartitionInfoPersistence.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/PartitionInfoPersistence.java
index 762cad1047..348c21efa8 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/PartitionInfoPersistence.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/PartitionInfoPersistence.java
@@ -19,12 +19,12 @@
package org.apache.iotdb.confignode.persistence;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.commons.partition.DataPartition;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
import org.apache.iotdb.commons.partition.SchemaPartition;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
import org.apache.iotdb.commons.utils.TestOnly;
import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
import org.apache.iotdb.confignode.consensus.response.DataPartitionDataSet;
@@ -103,7 +103,7 @@ public class PartitionInfoPersistence {
try {
// Allocate SchemaPartition by CreateSchemaPartitionPlan
- Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> assignedResult =
+ Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> assignedResult =
physicalPlan.getAssignedSchemaPartition();
assignedResult.forEach(
(storageGroup, partitionSlots) ->
@@ -118,9 +118,9 @@ public class PartitionInfoPersistence {
return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
}
- public Map<String, List<SeriesPartitionSlot>> filterNoAssignedSchemaPartitionSlots(
- Map<String, List<SeriesPartitionSlot>> partitionSlotsMap) {
- Map<String, List<SeriesPartitionSlot>> result;
+ public Map<String, List<TSeriesPartitionSlot>> filterNoAssignedSchemaPartitionSlots(
+ Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap) {
+ Map<String, List<TSeriesPartitionSlot>> result;
schemaPartitionReadWriteLock.readLock().lock();
try {
result = schemaPartition.filterNoAssignedSchemaPartitionSlot(partitionSlotsMap);
@@ -165,7 +165,7 @@ public class PartitionInfoPersistence {
try {
// Allocate DataPartition by CreateDataPartitionPlan
- Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+ Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
assignedResult = physicalPlan.getAssignedDataPartition();
assignedResult.forEach(
(storageGroup, seriesPartitionTimePartitionSlots) ->
@@ -187,10 +187,10 @@ public class PartitionInfoPersistence {
return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
}
- public Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>>
+ public Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>>
filterNoAssignedDataPartitionSlots(
- Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> partitionSlotsMap) {
- Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> result;
+ Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> partitionSlotsMap) {
+ Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> result;
dataPartitionReadWriteLock.readLock().lock();
try {
result = dataPartition.filterNoAssignedDataPartitionSlots(partitionSlotsMap);
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/RegionInfoPersistence.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/RegionInfoPersistence.java
index 97249e85d9..0e426d44a8 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/RegionInfoPersistence.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/RegionInfoPersistence.java
@@ -19,16 +19,14 @@
package org.apache.iotdb.confignode.persistence;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.consensus.ConsensusGroupId;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.consensus.SchemaRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
import org.apache.iotdb.commons.utils.TestOnly;
import org.apache.iotdb.confignode.consensus.response.StorageGroupSchemaDataSet;
-import org.apache.iotdb.confignode.partition.StorageGroupSchema;
import org.apache.iotdb.confignode.physical.crud.CreateRegionsPlan;
import org.apache.iotdb.confignode.physical.sys.SetStorageGroupPlan;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
import org.apache.iotdb.rpc.TSStatusCode;
import java.util.ArrayList;
@@ -42,7 +40,7 @@ public class RegionInfoPersistence {
// TODO: Serialize and Deserialize
// Map<StorageGroupName, StorageGroupSchema>
- private final Map<String, StorageGroupSchema> storageGroupsMap;
+ private final Map<String, TStorageGroupSchema> storageGroupsMap;
// Region allocate lock
private final ReentrantReadWriteLock regionAllocateLock;
@@ -51,8 +49,7 @@ public class RegionInfoPersistence {
// Region read write lock
private final ReentrantReadWriteLock regionReadWriteLock;
- // Map<ConsensusGroupId, RegionReplicaSet>
- private final Map<ConsensusGroupId, RegionReplicaSet> regionMap;
+ private final Map<TConsensusGroupId, TRegionReplicaSet> regionMap;
public RegionInfoPersistence() {
this.regionAllocateLock = new ReentrantReadWriteLock();
@@ -71,8 +68,8 @@ public class RegionInfoPersistence {
TSStatus result;
regionReadWriteLock.writeLock().lock();
try {
- StorageGroupSchema schema = plan.getSchema();
- storageGroupsMap.put(schema.getName(), schema);
+ TStorageGroupSchema storageGroupSchema = plan.getSchema();
+ storageGroupsMap.put(storageGroupSchema.getName(), storageGroupSchema);
result = new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
} finally {
regionReadWriteLock.writeLock().unlock();
@@ -103,16 +100,17 @@ public class RegionInfoPersistence {
regionReadWriteLock.writeLock().lock();
regionAllocateLock.writeLock().lock();
try {
- StorageGroupSchema schema = storageGroupsMap.get(plan.getStorageGroup());
-
- for (RegionReplicaSet regionReplicaSet : plan.getRegionReplicaSets()) {
- nextRegionGroupId =
- Math.max(nextRegionGroupId, regionReplicaSet.getConsensusGroupId().getId());
- regionMap.put(regionReplicaSet.getConsensusGroupId(), regionReplicaSet);
- if (regionReplicaSet.getConsensusGroupId() instanceof DataRegionId) {
- schema.addDataRegionGroup(regionReplicaSet.getConsensusGroupId());
- } else if (regionReplicaSet.getConsensusGroupId() instanceof SchemaRegionId) {
- schema.addSchemaRegionGroup(regionReplicaSet.getConsensusGroupId());
+ TStorageGroupSchema storageGroupSchema = storageGroupsMap.get(plan.getStorageGroup());
+
+ for (TRegionReplicaSet regionReplicaSet : plan.getRegionReplicaSets()) {
+ nextRegionGroupId = Math.max(nextRegionGroupId, regionReplicaSet.getRegionId().getId());
+ regionMap.put(regionReplicaSet.getRegionId(), regionReplicaSet);
+ switch (regionReplicaSet.getRegionId().getType()) {
+ case SchemaRegion:
+ storageGroupSchema.getSchemaRegionGroupIds().add(regionReplicaSet.getRegionId());
+ break;
+ case DataRegion:
+ storageGroupSchema.getDataRegionGroupIds().add(regionReplicaSet.getRegionId());
}
}
@@ -125,14 +123,14 @@ public class RegionInfoPersistence {
}
/** @return The SchemaRegion ReplicaSets in the specific StorageGroup */
- public List<RegionReplicaSet> getSchemaRegionEndPoint(String storageGroup) {
- List<RegionReplicaSet> schemaRegionEndPoints = new ArrayList<>();
+ public List<TRegionReplicaSet> getSchemaRegionEndPoint(String storageGroup) {
+ List<TRegionReplicaSet> schemaRegionEndPoints = new ArrayList<>();
regionReadWriteLock.readLock().lock();
try {
if (storageGroupsMap.containsKey(storageGroup)) {
- List<ConsensusGroupId> schemaRegionIds =
+ List<TConsensusGroupId> schemaRegionIds =
storageGroupsMap.get(storageGroup).getSchemaRegionGroupIds();
- for (ConsensusGroupId consensusGroupId : schemaRegionIds) {
+ for (TConsensusGroupId consensusGroupId : schemaRegionIds) {
schemaRegionEndPoints.add(regionMap.get(consensusGroupId));
}
}
@@ -144,14 +142,14 @@ public class RegionInfoPersistence {
}
/** @return The DataRegion ReplicaSets in the specific StorageGroup */
- public List<RegionReplicaSet> getDataRegionEndPoint(String storageGroup) {
- List<RegionReplicaSet> dataRegionEndPoints = new ArrayList<>();
+ public List<TRegionReplicaSet> getDataRegionEndPoint(String storageGroup) {
+ List<TRegionReplicaSet> dataRegionEndPoints = new ArrayList<>();
regionReadWriteLock.readLock().lock();
try {
if (storageGroupsMap.containsKey(storageGroup)) {
- List<ConsensusGroupId> dataRegionIds =
+ List<TConsensusGroupId> dataRegionIds =
storageGroupsMap.get(storageGroup).getDataRegionGroupIds();
- for (ConsensusGroupId consensusGroupId : dataRegionIds) {
+ for (TConsensusGroupId consensusGroupId : dataRegionIds) {
dataRegionEndPoints.add(regionMap.get(consensusGroupId));
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateDataPartitionPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateDataPartitionPlan.java
index 9daf5ef921..0b1bc95bba 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateDataPartitionPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateDataPartitionPlan.java
@@ -18,12 +18,13 @@
*/
package org.apache.iotdb.confignode.physical.crud;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
import org.apache.iotdb.confignode.physical.PhysicalPlan;
import org.apache.iotdb.confignode.physical.PhysicalPlanType;
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
import java.io.IOException;
import java.nio.ByteBuffer;
@@ -36,20 +37,20 @@ import java.util.Objects;
/** Create DataPartition by assignedDataPartition */
public class CreateDataPartitionPlan extends PhysicalPlan {
- private Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+ private Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
assignedDataPartition;
public CreateDataPartitionPlan() {
super(PhysicalPlanType.CreateDataPartition);
}
- public Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+ public Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
getAssignedDataPartition() {
return assignedDataPartition;
}
public void setAssignedDataPartition(
- Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+ Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
assignedDataPartition) {
this.assignedDataPartition = assignedDataPartition;
}
@@ -59,20 +60,21 @@ public class CreateDataPartitionPlan extends PhysicalPlan {
buffer.putInt(PhysicalPlanType.CreateDataPartition.ordinal());
buffer.putInt(assignedDataPartition.size());
- for (Map.Entry<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+ for (Map.Entry<
+ String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
seriesPartitionTimePartitionEntry : assignedDataPartition.entrySet()) {
- SerializeDeserializeUtil.write(seriesPartitionTimePartitionEntry.getKey(), buffer);
+ BasicStructureSerDeUtil.write(seriesPartitionTimePartitionEntry.getKey(), buffer);
buffer.putInt(seriesPartitionTimePartitionEntry.getValue().size());
- for (Map.Entry<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>
+ for (Map.Entry<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>
timePartitionEntry : seriesPartitionTimePartitionEntry.getValue().entrySet()) {
- timePartitionEntry.getKey().serializeImpl(buffer);
+ ThriftCommonsSerDeUtils.writeTSeriesPartitionSlot(timePartitionEntry.getKey(), buffer);
buffer.putInt(timePartitionEntry.getValue().size());
- for (Map.Entry<TimePartitionSlot, List<RegionReplicaSet>> regionReplicaSetEntry :
+ for (Map.Entry<TTimePartitionSlot, List<TRegionReplicaSet>> regionReplicaSetEntry :
timePartitionEntry.getValue().entrySet()) {
- regionReplicaSetEntry.getKey().serializeImpl(buffer);
+ ThriftCommonsSerDeUtils.writeTTimePartitionSlot(regionReplicaSetEntry.getKey(), buffer);
buffer.putInt(regionReplicaSetEntry.getValue().size());
- for (RegionReplicaSet regionReplicaSet : regionReplicaSetEntry.getValue()) {
- regionReplicaSet.serializeImpl(buffer);
+ for (TRegionReplicaSet regionReplicaSet : regionReplicaSetEntry.getValue()) {
+ ThriftCommonsSerDeUtils.writeTRegionReplicaSet(regionReplicaSet, buffer);
}
}
}
@@ -84,17 +86,17 @@ public class CreateDataPartitionPlan extends PhysicalPlan {
assignedDataPartition = new HashMap<>();
int storageGroupNum = buffer.getInt();
for (int i = 0; i < storageGroupNum; i++) {
- String storageGroupName = SerializeDeserializeUtil.readString(buffer);
+ String storageGroupName = BasicStructureSerDeUtil.readString(buffer);
assignedDataPartition.put(storageGroupName, new HashMap<>());
int seriesPartitionSlotNum = buffer.getInt();
for (int j = 0; j < seriesPartitionSlotNum; j++) {
- SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot();
- seriesPartitionSlot.deserializeImpl(buffer);
+ TSeriesPartitionSlot seriesPartitionSlot =
+ ThriftCommonsSerDeUtils.readTSeriesPartitionSlot(buffer);
assignedDataPartition.get(storageGroupName).put(seriesPartitionSlot, new HashMap<>());
int timePartitionSlotNum = buffer.getInt();
for (int k = 0; k < timePartitionSlotNum; k++) {
- TimePartitionSlot timePartitionSlot = new TimePartitionSlot();
- timePartitionSlot.deserializeImpl(buffer);
+ TTimePartitionSlot timePartitionSlot =
+ ThriftCommonsSerDeUtils.readTTimePartitionSlot(buffer);
assignedDataPartition
.get(storageGroupName)
.get(seriesPartitionSlot)
@@ -105,7 +107,7 @@ public class CreateDataPartitionPlan extends PhysicalPlan {
.get(storageGroupName)
.get(seriesPartitionSlot)
.get(timePartitionSlot)
- .add(RegionReplicaSet.deserializeImpl(buffer));
+ .add(ThriftCommonsSerDeUtils.readTRegionReplicaSet(buffer));
}
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateRegionsPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateRegionsPlan.java
index 56176a70dc..e5ceca415b 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateRegionsPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateRegionsPlan.java
@@ -18,10 +18,11 @@
*/
package org.apache.iotdb.confignode.physical.crud;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
import org.apache.iotdb.confignode.physical.PhysicalPlan;
import org.apache.iotdb.confignode.physical.PhysicalPlanType;
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
import java.io.IOException;
import java.nio.ByteBuffer;
@@ -34,7 +35,7 @@ public class CreateRegionsPlan extends PhysicalPlan {
private String storageGroup;
- private final List<RegionReplicaSet> regionReplicaSets;
+ private final List<TRegionReplicaSet> regionReplicaSets;
public CreateRegionsPlan() {
super(PhysicalPlanType.CreateRegions);
@@ -49,11 +50,11 @@ public class CreateRegionsPlan extends PhysicalPlan {
this.storageGroup = storageGroup;
}
- public void addRegion(RegionReplicaSet regionReplicaSet) {
+ public void addRegion(TRegionReplicaSet regionReplicaSet) {
this.regionReplicaSets.add(regionReplicaSet);
}
- public List<RegionReplicaSet> getRegionReplicaSets() {
+ public List<TRegionReplicaSet> getRegionReplicaSets() {
return regionReplicaSets;
}
@@ -61,21 +62,21 @@ public class CreateRegionsPlan extends PhysicalPlan {
protected void serializeImpl(ByteBuffer buffer) {
buffer.putInt(PhysicalPlanType.CreateRegions.ordinal());
- SerializeDeserializeUtil.write(storageGroup, buffer);
+ BasicStructureSerDeUtil.write(storageGroup, buffer);
buffer.putInt(regionReplicaSets.size());
- for (RegionReplicaSet regionReplicaSet : regionReplicaSets) {
- regionReplicaSet.serializeImpl(buffer);
+ for (TRegionReplicaSet regionReplicaSet : regionReplicaSets) {
+ ThriftCommonsSerDeUtils.writeTRegionReplicaSet(regionReplicaSet, buffer);
}
}
@Override
protected void deserializeImpl(ByteBuffer buffer) throws IOException {
- storageGroup = SerializeDeserializeUtil.readString(buffer);
+ storageGroup = BasicStructureSerDeUtil.readString(buffer);
int length = buffer.getInt();
for (int i = 0; i < length; i++) {
- regionReplicaSets.add(RegionReplicaSet.deserializeImpl(buffer));
+ regionReplicaSets.add(ThriftCommonsSerDeUtils.readTRegionReplicaSet(buffer));
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateSchemaPartitionPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateSchemaPartitionPlan.java
index 6a93dfdb6b..417227b920 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateSchemaPartitionPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/CreateSchemaPartitionPlan.java
@@ -18,11 +18,12 @@
*/
package org.apache.iotdb.confignode.physical.crud;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
import org.apache.iotdb.confignode.physical.PhysicalPlan;
import org.apache.iotdb.confignode.physical.PhysicalPlanType;
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
import java.io.IOException;
import java.nio.ByteBuffer;
@@ -33,18 +34,18 @@ import java.util.Objects;
/** Create SchemaPartition by assignedSchemaPartition */
public class CreateSchemaPartitionPlan extends PhysicalPlan {
- private Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> assignedSchemaPartition;
+ private Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> assignedSchemaPartition;
public CreateSchemaPartitionPlan() {
super(PhysicalPlanType.CreateSchemaPartition);
}
- public Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> getAssignedSchemaPartition() {
+ public Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> getAssignedSchemaPartition() {
return assignedSchemaPartition;
}
public void setAssignedSchemaPartition(
- Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> assignedSchemaPartition) {
+ Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> assignedSchemaPartition) {
this.assignedSchemaPartition = assignedSchemaPartition;
}
@@ -55,12 +56,12 @@ public class CreateSchemaPartitionPlan extends PhysicalPlan {
buffer.putInt(assignedSchemaPartition.size());
assignedSchemaPartition.forEach(
(storageGroup, partitionSlots) -> {
- SerializeDeserializeUtil.write(storageGroup, buffer);
+ BasicStructureSerDeUtil.write(storageGroup, buffer);
buffer.putInt(partitionSlots.size());
partitionSlots.forEach(
(seriesPartitionSlot, regionReplicaSet) -> {
- seriesPartitionSlot.serializeImpl(buffer);
- regionReplicaSet.serializeImpl(buffer);
+ ThriftCommonsSerDeUtils.writeTSeriesPartitionSlot(seriesPartitionSlot, buffer);
+ ThriftCommonsSerDeUtils.writeTRegionReplicaSet(regionReplicaSet, buffer);
});
});
}
@@ -71,15 +72,15 @@ public class CreateSchemaPartitionPlan extends PhysicalPlan {
int storageGroupNum = buffer.getInt();
for (int i = 0; i < storageGroupNum; i++) {
- String storageGroup = SerializeDeserializeUtil.readString(buffer);
+ String storageGroup = BasicStructureSerDeUtil.readString(buffer);
assignedSchemaPartition.put(storageGroup, new HashMap<>());
int seriesPartitionSlotNum = buffer.getInt();
for (int j = 0; j < seriesPartitionSlotNum; j++) {
- SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot();
- seriesPartitionSlot.deserializeImpl(buffer);
+ TSeriesPartitionSlot seriesPartitionSlot =
+ ThriftCommonsSerDeUtils.readTSeriesPartitionSlot(buffer);
assignedSchemaPartition
.get(storageGroup)
- .put(seriesPartitionSlot, RegionReplicaSet.deserializeImpl(buffer));
+ .put(seriesPartitionSlot, ThriftCommonsSerDeUtils.readTRegionReplicaSet(buffer));
}
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateDataPartitionPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateDataPartitionPlan.java
index 7715e7e6da..8547b7118f 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateDataPartitionPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateDataPartitionPlan.java
@@ -18,13 +18,14 @@
*/
package org.apache.iotdb.confignode.physical.crud;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
import org.apache.iotdb.confignode.physical.PhysicalPlan;
import org.apache.iotdb.confignode.physical.PhysicalPlanType;
import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq;
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
import java.nio.ByteBuffer;
import java.util.ArrayList;
@@ -36,24 +37,24 @@ import java.util.Objects;
/** Get or create DataPartition by the specific partitionSlotsMap. */
public class GetOrCreateDataPartitionPlan extends PhysicalPlan {
- private Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> partitionSlotsMap;
+ private Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> partitionSlotsMap;
public GetOrCreateDataPartitionPlan(PhysicalPlanType physicalPlanType) {
super(physicalPlanType);
}
- public Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> getPartitionSlotsMap() {
+ public Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> getPartitionSlotsMap() {
return partitionSlotsMap;
}
@TestOnly
public void setPartitionSlotsMap(
- Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> partitionSlotsMap) {
+ Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> partitionSlotsMap) {
this.partitionSlotsMap = partitionSlotsMap;
}
/**
- * Convert TDataPartitionReq to DataPartitionPlan
+ * Convert TDataPartitionReq to GetOrCreateDataPartitionPlan
*
* @param req TDataPartitionReq
*/
@@ -68,19 +69,17 @@ public class GetOrCreateDataPartitionPlan extends PhysicalPlan {
tSeriesPartitionTimePartitionSlots.forEach(
((tSeriesPartitionSlot, tTimePartitionSlots) -> {
// Extract SeriesPartitionSlot
- SeriesPartitionSlot seriesPartitionSlot =
- new SeriesPartitionSlot(tSeriesPartitionSlot.getSlotId());
partitionSlotsMap
.get(storageGroup)
- .putIfAbsent(seriesPartitionSlot, new ArrayList<>());
+ .putIfAbsent(tSeriesPartitionSlot, new ArrayList<>());
// Extract TimePartitionSlots
tTimePartitionSlots.forEach(
tTimePartitionSlot ->
partitionSlotsMap
.get(storageGroup)
- .get(seriesPartitionSlot)
- .add(new TimePartitionSlot(tTimePartitionSlot.getStartTime())));
+ .get(tSeriesPartitionSlot)
+ .add(tTimePartitionSlot));
}));
}));
}
@@ -92,14 +91,15 @@ public class GetOrCreateDataPartitionPlan extends PhysicalPlan {
buffer.putInt(partitionSlotsMap.size());
partitionSlotsMap.forEach(
((storageGroup, seriesPartitionTimePartitionSlots) -> {
- SerializeDeserializeUtil.write(storageGroup, buffer);
+ BasicStructureSerDeUtil.write(storageGroup, buffer);
buffer.putInt(seriesPartitionTimePartitionSlots.size());
seriesPartitionTimePartitionSlots.forEach(
((seriesPartitionSlot, timePartitionSlots) -> {
- seriesPartitionSlot.serializeImpl(buffer);
+ ThriftCommonsSerDeUtils.writeTSeriesPartitionSlot(seriesPartitionSlot, buffer);
buffer.putInt(timePartitionSlots.size());
timePartitionSlots.forEach(
- timePartitionSlot -> timePartitionSlot.serializeImpl(buffer));
+ timePartitionSlot ->
+ ThriftCommonsSerDeUtils.writeTTimePartitionSlot(timePartitionSlot, buffer));
}));
}));
}
@@ -109,17 +109,17 @@ public class GetOrCreateDataPartitionPlan extends PhysicalPlan {
partitionSlotsMap = new HashMap<>();
int storageGroupNum = buffer.getInt();
for (int i = 0; i < storageGroupNum; i++) {
- String storageGroup = SerializeDeserializeUtil.readString(buffer);
+ String storageGroup = BasicStructureSerDeUtil.readString(buffer);
partitionSlotsMap.put(storageGroup, new HashMap<>());
int seriesPartitionSlotNum = buffer.getInt();
for (int j = 0; j < seriesPartitionSlotNum; j++) {
- SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot();
- seriesPartitionSlot.deserializeImpl(buffer);
+ TSeriesPartitionSlot seriesPartitionSlot =
+ ThriftCommonsSerDeUtils.readTSeriesPartitionSlot(buffer);
partitionSlotsMap.get(storageGroup).put(seriesPartitionSlot, new ArrayList<>());
int timePartitionSlotNum = buffer.getInt();
for (int k = 0; k < timePartitionSlotNum; k++) {
- TimePartitionSlot timePartitionSlot = new TimePartitionSlot();
- timePartitionSlot.deserializeImpl(buffer);
+ TTimePartitionSlot timePartitionSlot =
+ ThriftCommonsSerDeUtils.readTTimePartitionSlot(buffer);
partitionSlotsMap.get(storageGroup).get(seriesPartitionSlot).add(timePartitionSlot);
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateSchemaPartitionPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateSchemaPartitionPlan.java
index 5b6e3d5149..2f3df69725 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateSchemaPartitionPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/crud/GetOrCreateSchemaPartitionPlan.java
@@ -18,10 +18,11 @@
*/
package org.apache.iotdb.confignode.physical.crud;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
import org.apache.iotdb.confignode.physical.PhysicalPlan;
import org.apache.iotdb.confignode.physical.PhysicalPlanType;
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
import java.io.IOException;
import java.nio.ByteBuffer;
@@ -37,17 +38,17 @@ public class GetOrCreateSchemaPartitionPlan extends PhysicalPlan {
// Map<StorageGroup, List<SeriesPartitionSlot>>
// Get all SchemaPartitions when the partitionSlotsMap is empty
// Get all exists SchemaPartitions in one StorageGroup when the SeriesPartitionSlot is empty
- private Map<String, List<SeriesPartitionSlot>> partitionSlotsMap;
+ private Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap;
public GetOrCreateSchemaPartitionPlan(PhysicalPlanType physicalPlanType) {
super(physicalPlanType);
}
- public void setPartitionSlotsMap(Map<String, List<SeriesPartitionSlot>> partitionSlotsMap) {
+ public void setPartitionSlotsMap(Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap) {
this.partitionSlotsMap = partitionSlotsMap;
}
- public Map<String, List<SeriesPartitionSlot>> getPartitionSlotsMap() {
+ public Map<String, List<TSeriesPartitionSlot>> getPartitionSlotsMap() {
return partitionSlotsMap;
}
@@ -58,10 +59,11 @@ public class GetOrCreateSchemaPartitionPlan extends PhysicalPlan {
buffer.putInt(partitionSlotsMap.size());
partitionSlotsMap.forEach(
(storageGroup, seriesPartitionSlots) -> {
- SerializeDeserializeUtil.write(storageGroup, buffer);
+ BasicStructureSerDeUtil.write(storageGroup, buffer);
buffer.putInt(seriesPartitionSlots.size());
seriesPartitionSlots.forEach(
- seriesPartitionSlot -> seriesPartitionSlot.serializeImpl(buffer));
+ seriesPartitionSlot ->
+ ThriftCommonsSerDeUtils.writeTSeriesPartitionSlot(seriesPartitionSlot, buffer));
});
}
@@ -70,12 +72,12 @@ public class GetOrCreateSchemaPartitionPlan extends PhysicalPlan {
partitionSlotsMap = new HashMap<>();
int storageGroupNum = buffer.getInt();
for (int i = 0; i < storageGroupNum; i++) {
- String storageGroup = SerializeDeserializeUtil.readString(buffer);
+ String storageGroup = BasicStructureSerDeUtil.readString(buffer);
partitionSlotsMap.put(storageGroup, new ArrayList<>());
int seriesPartitionSlotNum = buffer.getInt();
for (int j = 0; j < seriesPartitionSlotNum; j++) {
- SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot();
- seriesPartitionSlot.deserializeImpl(buffer);
+ TSeriesPartitionSlot seriesPartitionSlot =
+ ThriftCommonsSerDeUtils.readTSeriesPartitionSlot(buffer);
partitionSlotsMap.get(storageGroup).add(seriesPartitionSlot);
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/AuthorPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/AuthorPlan.java
index 32f6649736..bc90b6edfb 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/AuthorPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/AuthorPlan.java
@@ -18,9 +18,9 @@
*/
package org.apache.iotdb.confignode.physical.sys;
+import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
import org.apache.iotdb.confignode.physical.PhysicalPlan;
import org.apache.iotdb.confignode.physical.PhysicalPlanType;
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
import org.apache.iotdb.db.auth.AuthException;
import java.nio.ByteBuffer;
@@ -132,11 +132,11 @@ public class AuthorPlan extends PhysicalPlan {
@Override
protected void serializeImpl(ByteBuffer buffer) {
- SerializeDeserializeUtil.write(getPlanTypeOrdinal(authorType), buffer);
- SerializeDeserializeUtil.write(userName, buffer);
- SerializeDeserializeUtil.write(roleName, buffer);
- SerializeDeserializeUtil.write(password, buffer);
- SerializeDeserializeUtil.write(newPassword, buffer);
+ BasicStructureSerDeUtil.write(getPlanTypeOrdinal(authorType), buffer);
+ BasicStructureSerDeUtil.write(userName, buffer);
+ BasicStructureSerDeUtil.write(roleName, buffer);
+ BasicStructureSerDeUtil.write(password, buffer);
+ BasicStructureSerDeUtil.write(newPassword, buffer);
if (permissions == null) {
buffer.put((byte) 0);
} else {
@@ -146,15 +146,15 @@ public class AuthorPlan extends PhysicalPlan {
buffer.putInt(permission);
}
}
- SerializeDeserializeUtil.write(nodeName, buffer);
+ BasicStructureSerDeUtil.write(nodeName, buffer);
}
@Override
protected void deserializeImpl(ByteBuffer buffer) {
- userName = SerializeDeserializeUtil.readString(buffer);
- roleName = SerializeDeserializeUtil.readString(buffer);
- password = SerializeDeserializeUtil.readString(buffer);
- newPassword = SerializeDeserializeUtil.readString(buffer);
+ userName = BasicStructureSerDeUtil.readString(buffer);
+ roleName = BasicStructureSerDeUtil.readString(buffer);
+ password = BasicStructureSerDeUtil.readString(buffer);
+ newPassword = BasicStructureSerDeUtil.readString(buffer);
byte hasPermissions = buffer.get();
if (hasPermissions == (byte) 0) {
this.permissions = null;
@@ -165,7 +165,7 @@ public class AuthorPlan extends PhysicalPlan {
permissions.add(buffer.getInt());
}
}
- nodeName = SerializeDeserializeUtil.readString(buffer);
+ nodeName = BasicStructureSerDeUtil.readString(buffer);
}
private int getPlanTypeOrdinal(PhysicalPlanType physicalPlanType) {
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/RegisterDataNodePlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/RegisterDataNodePlan.java
index c4c3fd5d52..01a8199ee6 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/RegisterDataNodePlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/RegisterDataNodePlan.java
@@ -18,8 +18,8 @@
*/
package org.apache.iotdb.confignode.physical.sys;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
import org.apache.iotdb.confignode.physical.PhysicalPlan;
import org.apache.iotdb.confignode.physical.PhysicalPlanType;
@@ -28,40 +28,30 @@ import java.util.Objects;
public class RegisterDataNodePlan extends PhysicalPlan {
- private DataNodeLocation info;
+ private TDataNodeLocation location;
public RegisterDataNodePlan() {
super(PhysicalPlanType.RegisterDataNode);
}
- public RegisterDataNodePlan(DataNodeLocation info) {
+ public RegisterDataNodePlan(TDataNodeLocation location) {
this();
- this.info = info;
+ this.location = location;
}
- public DataNodeLocation getInfo() {
- return info;
+ public TDataNodeLocation getLocation() {
+ return location;
}
@Override
protected void serializeImpl(ByteBuffer buffer) {
buffer.putInt(PhysicalPlanType.RegisterDataNode.ordinal());
- buffer.putInt(info.getDataNodeId());
- buffer.putInt(info.getEndPoint().getIp().length());
- buffer.put(info.getEndPoint().getIp().getBytes());
- buffer.putInt(info.getEndPoint().getPort());
+ ThriftCommonsSerDeUtils.writeTDataNodeLocation(location, buffer);
}
@Override
protected void deserializeImpl(ByteBuffer buffer) {
- int dataNodeID = buffer.getInt();
- int ipLength = buffer.getInt();
- byte[] byteIp = new byte[ipLength];
- buffer.get(byteIp, 0, ipLength);
- String ip = new String(byteIp, 0, ipLength);
- int port = buffer.getInt();
-
- this.info = new DataNodeLocation(dataNodeID, new Endpoint(ip, port));
+ location = ThriftCommonsSerDeUtils.readTDataNodeLocation(buffer);
}
@Override
@@ -69,11 +59,11 @@ public class RegisterDataNodePlan extends PhysicalPlan {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
RegisterDataNodePlan plan = (RegisterDataNodePlan) o;
- return info.equals(plan.info);
+ return location.equals(plan.location);
}
@Override
public int hashCode() {
- return Objects.hash(info);
+ return Objects.hash(location);
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/SetStorageGroupPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/SetStorageGroupPlan.java
index 865829592d..80078672e2 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/SetStorageGroupPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/physical/sys/SetStorageGroupPlan.java
@@ -18,9 +18,10 @@
*/
package org.apache.iotdb.confignode.physical.sys;
-import org.apache.iotdb.confignode.partition.StorageGroupSchema;
+import org.apache.iotdb.commons.utils.ThriftConfigNodeSerDeUtils;
import org.apache.iotdb.confignode.physical.PhysicalPlan;
import org.apache.iotdb.confignode.physical.PhysicalPlanType;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
import java.io.IOException;
import java.nio.ByteBuffer;
@@ -28,35 +29,35 @@ import java.util.Objects;
public class SetStorageGroupPlan extends PhysicalPlan {
- private StorageGroupSchema schema;
+ private TStorageGroupSchema schema;
public SetStorageGroupPlan() {
super(PhysicalPlanType.SetStorageGroup);
- this.schema = new StorageGroupSchema();
+ this.schema = new TStorageGroupSchema();
}
- public SetStorageGroupPlan(StorageGroupSchema schema) {
+ public SetStorageGroupPlan(TStorageGroupSchema schema) {
this();
this.schema = schema;
}
- public StorageGroupSchema getSchema() {
+ public TStorageGroupSchema getSchema() {
return schema;
}
- public void setSchema(StorageGroupSchema schema) {
+ public void setSchema(TStorageGroupSchema schema) {
this.schema = schema;
}
@Override
protected void serializeImpl(ByteBuffer buffer) {
buffer.putInt(PhysicalPlanType.SetStorageGroup.ordinal());
- schema.serialize(buffer);
+ ThriftConfigNodeSerDeUtils.writeTStorageGroupSchema(schema, buffer);
}
@Override
protected void deserializeImpl(ByteBuffer buffer) throws IOException {
- schema.deserialize(buffer);
+ schema = ThriftConfigNodeSerDeUtils.readTStorageGroupSchema(buffer);
}
@Override
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessor.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessor.java
index af6647c3b7..99c1de0334 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessor.java
@@ -19,17 +19,14 @@
package org.apache.iotdb.confignode.service.thrift.server;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
import org.apache.iotdb.confignode.consensus.response.DataNodeConfigurationDataSet;
-import org.apache.iotdb.confignode.consensus.response.DataNodesInfoDataSet;
+import org.apache.iotdb.confignode.consensus.response.DataNodeLocationsDataSet;
import org.apache.iotdb.confignode.consensus.response.DataPartitionDataSet;
import org.apache.iotdb.confignode.consensus.response.PermissionInfoDataSet;
import org.apache.iotdb.confignode.consensus.response.SchemaPartitionDataSet;
import org.apache.iotdb.confignode.consensus.response.StorageGroupSchemaDataSet;
import org.apache.iotdb.confignode.manager.ConfigManager;
-import org.apache.iotdb.confignode.partition.StorageGroupSchema;
import org.apache.iotdb.confignode.physical.PhysicalPlanType;
import org.apache.iotdb.confignode.physical.crud.GetOrCreateDataPartitionPlan;
import org.apache.iotdb.confignode.physical.sys.AuthorPlan;
@@ -39,7 +36,7 @@ import org.apache.iotdb.confignode.physical.sys.SetStorageGroupPlan;
import org.apache.iotdb.confignode.rpc.thrift.ConfigIService;
import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerReq;
import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerResp;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessageResp;
+import org.apache.iotdb.confignode.rpc.thrift.TDataNodeLocationResp;
import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq;
import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterResp;
import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq;
@@ -49,6 +46,8 @@ import org.apache.iotdb.confignode.rpc.thrift.TSchemaPartitionReq;
import org.apache.iotdb.confignode.rpc.thrift.TSchemaPartitionResp;
import org.apache.iotdb.confignode.rpc.thrift.TSetStorageGroupReq;
import org.apache.iotdb.confignode.rpc.thrift.TSetTTLReq;
+import org.apache.iotdb.confignode.rpc.thrift.TSetTimePartitionIntervalReq;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchemaResp;
import org.apache.iotdb.db.auth.AuthException;
import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
@@ -60,6 +59,7 @@ import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.ByteBuffer;
+import java.util.ArrayList;
/** ConfigNodeRPCServer exposes the interface that interacts with the DataNode */
public class ConfigNodeRPCServerProcessor implements ConfigIService.Iface {
@@ -78,10 +78,7 @@ public class ConfigNodeRPCServerProcessor implements ConfigIService.Iface {
@Override
public TDataNodeRegisterResp registerDataNode(TDataNodeRegisterReq req) throws TException {
- RegisterDataNodePlan plan =
- new RegisterDataNodePlan(
- new DataNodeLocation(
- -1, new Endpoint(req.getEndPoint().getIp(), req.getEndPoint().getPort())));
+ RegisterDataNodePlan plan = new RegisterDataNodePlan(req.getDataNodeLocation());
DataNodeConfigurationDataSet dataSet =
(DataNodeConfigurationDataSet) configManager.registerDataNode(plan);
@@ -92,23 +89,32 @@ public class ConfigNodeRPCServerProcessor implements ConfigIService.Iface {
}
@Override
- public TDataNodeMessageResp getDataNodesMessage(int dataNodeID) throws TException {
+ public TDataNodeLocationResp getDataNodeLocations(int dataNodeID) throws TException {
QueryDataNodeInfoPlan plan = new QueryDataNodeInfoPlan(dataNodeID);
- DataNodesInfoDataSet dataSet = (DataNodesInfoDataSet) configManager.getDataNodeInfo(plan);
+ DataNodeLocationsDataSet dataSet =
+ (DataNodeLocationsDataSet) configManager.getDataNodeInfo(plan);
- TDataNodeMessageResp resp = new TDataNodeMessageResp();
- dataSet.convertToRPCDataNodeMessageResp(resp);
+ TDataNodeLocationResp resp = new TDataNodeLocationResp();
+ dataSet.convertToRpcDataNodeLocationResp(resp);
return resp;
}
@Override
public TSStatus setStorageGroup(TSetStorageGroupReq req) throws TException {
- SetStorageGroupPlan plan =
- new SetStorageGroupPlan(new StorageGroupSchema(req.getStorageGroup()));
-
- // TODO: Set TTL by optional field TSetStorageGroupReq.TTL
- plan.getSchema().setTTL(ConfigNodeDescriptor.getInstance().getConf().getDefaultTTL());
-
+ TStorageGroupSchema storageGroupSchema = req.getStorageGroup();
+ // TODO: Set this filed by optional fields in TSetStorageGroupReq
+ storageGroupSchema.setTTL(ConfigNodeDescriptor.getInstance().getConf().getDefaultTTL());
+ storageGroupSchema.setSchemaReplicationFactor(
+ ConfigNodeDescriptor.getInstance().getConf().getSchemaReplicationFactor());
+ storageGroupSchema.setDataReplicationFactor(
+ ConfigNodeDescriptor.getInstance().getConf().getDataReplicationFactor());
+ storageGroupSchema.setTimePartitionInterval(
+ ConfigNodeDescriptor.getInstance().getConf().getTimePartitionInterval());
+
+ storageGroupSchema.setSchemaRegionGroupIds(new ArrayList<>());
+ storageGroupSchema.setDataRegionGroupIds(new ArrayList<>());
+
+ SetStorageGroupPlan plan = new SetStorageGroupPlan(storageGroupSchema);
return configManager.setStorageGroup(plan);
}
@@ -124,6 +130,12 @@ public class ConfigNodeRPCServerProcessor implements ConfigIService.Iface {
return null;
}
+ @Override
+ public TSStatus setTimePartitionInterval(TSetTimePartitionIntervalReq req) throws TException {
+ // TODO: Set TimePartitionInterval
+ return null;
+ }
+
@Override
public TStorageGroupSchemaResp getStorageGroupsSchema() throws TException {
StorageGroupSchemaDataSet dataSet =
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/cli/TemporaryClientDemo.java b/confignode/src/test/java/org/apache/iotdb/confignode/cli/TemporaryClientDemo.java
index 245c0b0acc..2071b75277 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/cli/TemporaryClientDemo.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/cli/TemporaryClientDemo.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.confignode.cli;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
import org.apache.iotdb.confignode.rpc.thrift.ConfigIService;
import org.apache.iotdb.confignode.rpc.thrift.TSetStorageGroupReq;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
import org.apache.iotdb.rpc.RpcTransportFactory;
import org.apache.iotdb.rpc.TSStatusCode;
@@ -46,7 +47,8 @@ public class TemporaryClientDemo {
defaultClient = clients.get(22277);
for (int i = 0; i < 5; i++) {
- TSetStorageGroupReq setReq = new TSetStorageGroupReq("root.sg" + i);
+ TSetStorageGroupReq setReq =
+ new TSetStorageGroupReq(new TStorageGroupSchema().setName("root.sg" + i));
while (true) {
TSStatus status = defaultClient.setStorageGroup(setReq);
System.out.println(status.toString());
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/consensus/RatisConsensusDemo.java b/confignode/src/test/java/org/apache/iotdb/confignode/consensus/RatisConsensusDemo.java
deleted file mode 100644
index 6956d486ed..0000000000
--- a/confignode/src/test/java/org/apache/iotdb/confignode/consensus/RatisConsensusDemo.java
+++ /dev/null
@@ -1,151 +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.iotdb.confignode.consensus;
-
-import org.apache.iotdb.common.rpc.thrift.TEndPoint;
-import org.apache.iotdb.confignode.rpc.thrift.ConfigIService;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessageResp;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterResp;
-import org.apache.iotdb.confignode.rpc.thrift.TSetStorageGroupReq;
-import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
-import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchemaResp;
-import org.apache.iotdb.rpc.RpcTransportFactory;
-import org.apache.iotdb.rpc.TSStatusCode;
-
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.junit.Assert;
-
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-/** Demo for ConfigNode's integration with the ratis-consensus protocol. */
-public class RatisConsensusDemo {
-
- private static final String localhost = "0.0.0.0";
- private static final int timeOutInMS = 2000;
-
- private ConfigIService.Client[] clients;
-
- /**
- * To run this code, follow these steps: 1. Compile IoTDB 2. Copy at least three
- * iotdb-confignode-0.14.0-SNAPSHOT 3. Make sure these parameters: config_node_rpc_address(all
- * 0.0.0.0), config_node_rpc_port(22277, 22279, 22281), config_node_internal_port(22278, 22280,
- * 22282), consensus_type(all ratis) and config_node_group_address_list(all 0.0.0.0:22278,
- * 0.0.0.0:22280, 0.0.0.0:22282) in each iotdb-confignode.properties file are set 4. Start these
- * ConfigNode by yourself 5. Add @Test 6. run ratisConsensusRegisterDemo 7. run
- * ratisConsensusQueryDemo
- */
- public void ratisConsensusSetStorageGroupsDemo() throws TException, InterruptedException {
- createClients();
- setStorageGroups();
- }
-
- public void ratisConsensusQueryDataNodesDemo() throws TException, InterruptedException {
- createClients();
- queryDataNodes();
- }
-
- public void ratisConsensusQueryStorageGroupsDemo() throws TException, InterruptedException {
- createClients();
- queryStorageGroups();
- }
-
- public void ratisConsensusLeaderRedirectDemo() throws TException {
- createClients();
- registerDataNodeOnLeader();
- }
-
- private void createClients() throws TTransportException {
- // Create clients for these three ConfigNodes
- // to simulate DataNodes to send RPC requests
- clients = new ConfigIService.Client[3];
- for (int i = 0; i < 3; i++) {
- TTransport transport =
- RpcTransportFactory.INSTANCE.getTransport(localhost, 22277 + i * 2, timeOutInMS);
- transport.open();
- clients[i] = new ConfigIService.Client(new TBinaryProtocol(transport));
- }
- }
-
- private void registerDataNodes() throws TException, InterruptedException {
- // DataNodes can connect to any ConfigNode and send write requests
- for (int i = 0; i < 10; i++) {
- TEndPoint endPoint = new TEndPoint("0.0.0.0", 6667 + i);
- TDataNodeRegisterReq req = new TDataNodeRegisterReq(endPoint);
- TDataNodeRegisterResp resp = clients[0].registerDataNode(req);
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
- Assert.assertEquals(i, resp.getDataNodeID());
- System.out.printf(
- "\nRegister DataNode successful. DataNodeID: %d, %s\n", resp.getDataNodeID(), endPoint);
-
- TimeUnit.SECONDS.sleep(1);
- }
- }
-
- private void queryDataNodes() throws InterruptedException, TException {
- // sleep 1s to make sure all ConfigNode in ConfigNodeGroup hold the same PartitionTable
- TimeUnit.SECONDS.sleep(1);
-
- // DataNodes can connect to any ConfigNode and send read requests
- for (int i = 0; i < 3; i++) {
- TDataNodeMessageResp msgMap = clients[i].getDataNodesMessage(-1);
- System.out.printf(
- "\nQuery DataNode message from ConfigNode 0.0.0.0:%d. Result: %s\n",
- 22277 + i * 2, msgMap);
- }
- }
-
- private void setStorageGroups() throws TException, InterruptedException {
- for (int i = 0; i < 10; i++) {
- TSetStorageGroupReq req = new TSetStorageGroupReq("root.sg" + i);
- clients[0].setStorageGroup(req);
- System.out.printf("\nSet StorageGroup successful. StorageGroup: %s\n", "root.sg" + i);
- TimeUnit.SECONDS.sleep(1);
- }
- }
-
- private void queryStorageGroups() throws InterruptedException, TException {
- // sleep 1s to make sure all ConfigNode in ConfigNodeGroup hold the same PartitionTable
- TimeUnit.SECONDS.sleep(1);
-
- for (int i = 0; i < 3; i++) {
- TStorageGroupSchemaResp msgMap = clients[i].getStorageGroupsSchema();
- System.out.printf(
- "\nQuery StorageGroup message from ConfigNode 0.0.0.0:%d. Result: {\n", 22277 + i * 2);
- for (Map.Entry<String, TStorageGroupSchema> entry :
- msgMap.getStorageGroupSchemaMap().entrySet()) {
- System.out.printf(" Key(%s)=%s\n", entry.getKey(), entry.getValue().toString());
- }
- System.out.println("}");
- }
- }
-
- private void registerDataNodeOnLeader() throws TException {
- for (int i = 0; i < 3; i++) {
- TEndPoint endPoint = new TEndPoint("0.0.0.0", 6667);
- TDataNodeRegisterReq req = new TDataNodeRegisterReq(endPoint);
- TDataNodeRegisterResp resp = clients[i].registerDataNode(req);
- System.out.println(resp);
- }
- }
-}
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/manager/ConfigManagerManualTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/manager/ConfigManagerManualTest.java
deleted file mode 100644
index d7d54b355f..0000000000
--- a/confignode/src/test/java/org/apache/iotdb/confignode/manager/ConfigManagerManualTest.java
+++ /dev/null
@@ -1,133 +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.iotdb.confignode.manager;
-
-import org.apache.iotdb.common.rpc.thrift.TEndPoint;
-import org.apache.iotdb.confignode.rpc.thrift.ConfigIService;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessage;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterResp;
-import org.apache.iotdb.rpc.RpcTransportFactory;
-import org.apache.iotdb.rpc.TSStatusCode;
-
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.junit.Assert;
-
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-public class ConfigManagerManualTest {
-
- // TODO: Optimize this manual test to automatic test after the test environment is set up.
- // @YongzaoDan
-
- private static final String localhost = "0.0.0.0";
- private static final int timeOutInMS = 2000;
-
- private ConfigIService.Client[] clients;
-
- /**
- * This is a temporary test of ConfigNode's integration with the ratis-consensus protocol. To run
- * this code, follow these steps: 1. Compile IoTDB 2. Copy at least three
- * iotdb-confignode-0.14.0-SNAPSHOT 3. Make sure these parameters: config_node_rpc_address(all
- * 0.0.0.0), config_node_rpc_port(22277, 22279, 22281), config_node_internal_port(22278, 22280,
- * 22282), consensus_type(all ratis) and config_node_group_address_list(all 0.0.0.0:22278,
- * 0.0.0.0:22280, 0.0.0.0:22282) in each iotdb-confignode.properties file are set 4. Start these
- * ConfigNode by yourself 5. Add @Test and run
- */
- public void ratisConsensusTest() throws TException, InterruptedException {
- createClients();
-
- registerDataNodes();
-
- queryDataNodes();
- }
-
- private void createClients() throws TTransportException {
- clients = new ConfigIService.Client[3];
- for (int i = 0; i < 3; i++) {
- TTransport transport =
- RpcTransportFactory.INSTANCE.getTransport(localhost, 22277 + i * 2, timeOutInMS);
- transport.open();
- clients[i] = new ConfigIService.Client(new TBinaryProtocol(transport));
- }
- }
-
- private void registerDataNodes() throws TException {
- for (int i = 0; i < 3; i++) {
- TDataNodeRegisterReq req = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6667 + i));
- TDataNodeRegisterResp resp = clients[0].registerDataNode(req);
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
- Assert.assertEquals(i, resp.getDataNodeID());
- }
- }
-
- private void queryDataNodes() throws InterruptedException, TException {
- // sleep 1s to make sure all ConfigNode in ConfigNodeGroup hold the same PartitionTable
- TimeUnit.SECONDS.sleep(1);
-
- for (int i = 0; i < 3; i++) {
- Map<Integer, TDataNodeMessage> msgMap =
- clients[i].getDataNodesMessage(-1).getDataNodeMessageMap();
- Assert.assertEquals(3, msgMap.size());
- for (int j = 0; j < 3; j++) {
- Assert.assertNotNull(msgMap.get(j));
- Assert.assertEquals(j, msgMap.get(j).getDataNodeId());
- Assert.assertEquals(localhost, msgMap.get(j).getEndPoint().getIp());
- Assert.assertEquals(6667 + j, msgMap.get(j).getEndPoint().getPort());
- }
- }
- }
-
- /**
- * This is a temporary test of ConfigNode's integration with the ratis-consensus protocol. This
- * code tests the high availability of the ratis-consensus protocol. Make sure that you have run
- * according to the comments of ratisConsensusTest before executing this code. Next, close
- * ConfigNode that occupies ports 22281 and 22282 on the local machine. Finally, run this test.
- */
- public void killTest() throws TException {
- clients = new ConfigIService.Client[2];
- for (int i = 0; i < 2; i++) {
- TTransport transport =
- RpcTransportFactory.INSTANCE.getTransport(localhost, 22277 + i * 2, timeOutInMS);
- transport.open();
- clients[i] = new ConfigIService.Client(new TBinaryProtocol(transport));
- }
-
- TDataNodeRegisterResp resp =
- clients[1].registerDataNode(new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6670)));
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
- Assert.assertEquals(3, resp.getDataNodeID());
-
- for (int i = 0; i < 2; i++) {
- Map<Integer, TDataNodeMessage> msgMap =
- clients[i].getDataNodesMessage(-1).getDataNodeMessageMap();
- Assert.assertEquals(4, msgMap.size());
- for (int j = 0; j < 4; j++) {
- Assert.assertNotNull(msgMap.get(j));
- Assert.assertEquals(j, msgMap.get(j).getDataNodeId());
- Assert.assertEquals(localhost, msgMap.get(j).getEndPoint().getIp());
- Assert.assertEquals(6667 + j, msgMap.get(j).getEndPoint().getPort());
- }
- }
- }
-}
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/physical/SerializeDeserializeUT.java b/confignode/src/test/java/org/apache/iotdb/confignode/physical/PhysicalPlanSerDeTest.java
similarity index 73%
rename from confignode/src/test/java/org/apache/iotdb/confignode/physical/SerializeDeserializeUT.java
rename to confignode/src/test/java/org/apache/iotdb/confignode/physical/PhysicalPlanSerDeTest.java
index 3dc71321d7..78386bee94 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/physical/SerializeDeserializeUT.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/physical/PhysicalPlanSerDeTest.java
@@ -18,14 +18,13 @@
*/
package org.apache.iotdb.confignode.physical;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.consensus.SchemaRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
-import org.apache.iotdb.confignode.partition.StorageGroupSchema;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.confignode.physical.crud.CreateDataPartitionPlan;
import org.apache.iotdb.confignode.physical.crud.CreateRegionsPlan;
import org.apache.iotdb.confignode.physical.crud.CreateSchemaPartitionPlan;
@@ -35,6 +34,7 @@ import org.apache.iotdb.confignode.physical.sys.AuthorPlan;
import org.apache.iotdb.confignode.physical.sys.QueryDataNodeInfoPlan;
import org.apache.iotdb.confignode.physical.sys.RegisterDataNodePlan;
import org.apache.iotdb.confignode.physical.sys.SetStorageGroupPlan;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
import org.apache.iotdb.db.auth.AuthException;
import org.apache.iotdb.db.auth.entity.PrivilegeType;
@@ -52,7 +52,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
-public class SerializeDeserializeUT {
+public class PhysicalPlanSerDeTest {
private final ByteBuffer buffer = ByteBuffer.allocate(10240);
@@ -63,8 +63,13 @@ public class SerializeDeserializeUT {
@Test
public void RegisterDataNodePlanTest() throws IOException {
- RegisterDataNodePlan plan0 =
- new RegisterDataNodePlan(new DataNodeLocation(1, new Endpoint("0.0.0.0", 6667)));
+ TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+ dataNodeLocation.setDataNodeId(1);
+ dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667));
+ dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+ dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777));
+ dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 7777));
+ RegisterDataNodePlan plan0 = new RegisterDataNodePlan(dataNodeLocation);
plan0.serialize(buffer);
buffer.flip();
RegisterDataNodePlan plan1 = (RegisterDataNodePlan) PhysicalPlan.Factory.create(buffer);
@@ -82,7 +87,16 @@ public class SerializeDeserializeUT {
@Test
public void SetStorageGroupPlanTest() throws IOException {
- SetStorageGroupPlan plan0 = new SetStorageGroupPlan(new StorageGroupSchema("sg"));
+ SetStorageGroupPlan plan0 =
+ new SetStorageGroupPlan(
+ new TStorageGroupSchema()
+ .setName("sg")
+ .setTTL(Long.MAX_VALUE)
+ .setSchemaReplicationFactor(3)
+ .setDataReplicationFactor(3)
+ .setTimePartitionInterval(604800)
+ .setSchemaRegionGroupIds(new ArrayList<>())
+ .setDataRegionGroupIds(new ArrayList<>()));
plan0.serialize(buffer);
buffer.flip();
SetStorageGroupPlan plan1 = (SetStorageGroupPlan) PhysicalPlan.Factory.create(buffer);
@@ -96,17 +110,23 @@ public class SerializeDeserializeUT {
@Test
public void CreateRegionsPlanTest() throws IOException {
+ TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+ dataNodeLocation.setDataNodeId(0);
+ dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667));
+ dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+ dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777));
+ dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010));
+
CreateRegionsPlan plan0 = new CreateRegionsPlan();
plan0.setStorageGroup("sg");
- RegionReplicaSet dataRegionSet = new RegionReplicaSet();
- dataRegionSet.setConsensusGroupId(new DataRegionId(0));
- dataRegionSet.setDataNodeList(
- Collections.singletonList(new DataNodeLocation(0, new Endpoint("0.0.0.0", 6667))));
+ TRegionReplicaSet dataRegionSet = new TRegionReplicaSet();
+ dataRegionSet.setRegionId(new TConsensusGroupId(TConsensusGroupType.DataRegion, 0));
+ dataRegionSet.setDataNodeLocations(Collections.singletonList(dataNodeLocation));
plan0.addRegion(dataRegionSet);
- RegionReplicaSet schemaRegionSet = new RegionReplicaSet();
- schemaRegionSet.setConsensusGroupId(new SchemaRegionId(1));
- schemaRegionSet.setDataNodeList(
- Collections.singletonList(new DataNodeLocation(0, new Endpoint("0.0.0.0", 6667))));
+
+ TRegionReplicaSet schemaRegionSet = new TRegionReplicaSet();
+ schemaRegionSet.setRegionId(new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 1));
+ schemaRegionSet.setDataNodeLocations(Collections.singletonList(dataNodeLocation));
plan0.addRegion(schemaRegionSet);
plan0.serialize(buffer);
@@ -117,14 +137,20 @@ public class SerializeDeserializeUT {
@Test
public void CreateSchemaPartitionPlanTest() throws IOException {
+ TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+ dataNodeLocation.setDataNodeId(0);
+ dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667));
+ dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+ dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777));
+ dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010));
+
String storageGroup = "root.sg0";
- SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot(10);
- RegionReplicaSet regionReplicaSet = new RegionReplicaSet();
- regionReplicaSet.setConsensusGroupId(new SchemaRegionId(0));
- regionReplicaSet.setDataNodeList(
- Collections.singletonList(new DataNodeLocation(0, new Endpoint("0.0.0.0", 6667))));
+ TSeriesPartitionSlot seriesPartitionSlot = new TSeriesPartitionSlot(10);
+ TRegionReplicaSet regionReplicaSet = new TRegionReplicaSet();
+ regionReplicaSet.setRegionId(new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 0));
+ regionReplicaSet.setDataNodeLocations(Collections.singletonList(dataNodeLocation));
- Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> assignedSchemaPartition =
+ Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> assignedSchemaPartition =
new HashMap<>();
assignedSchemaPartition.put(storageGroup, new HashMap<>());
assignedSchemaPartition.get(storageGroup).put(seriesPartitionSlot, regionReplicaSet);
@@ -141,9 +167,9 @@ public class SerializeDeserializeUT {
@Test
public void GetOrCreateSchemaPartitionPlanTest() throws IOException {
String storageGroup = "root.sg0";
- SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot(10);
+ TSeriesPartitionSlot seriesPartitionSlot = new TSeriesPartitionSlot(10);
- Map<String, List<SeriesPartitionSlot>> partitionSlotsMap = new HashMap<>();
+ Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap = new HashMap<>();
partitionSlotsMap.put(storageGroup, Collections.singletonList(seriesPartitionSlot));
GetOrCreateSchemaPartitionPlan plan0 =
@@ -158,15 +184,21 @@ public class SerializeDeserializeUT {
@Test
public void CreateDataPartitionPlanTest() throws IOException {
+ TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+ dataNodeLocation.setDataNodeId(0);
+ dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667));
+ dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+ dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777));
+ dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010));
+
String storageGroup = "root.sg0";
- SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot(10);
- TimePartitionSlot timePartitionSlot = new TimePartitionSlot(100);
- RegionReplicaSet regionReplicaSet = new RegionReplicaSet();
- regionReplicaSet.setConsensusGroupId(new DataRegionId(0));
- regionReplicaSet.setDataNodeList(
- Collections.singletonList(new DataNodeLocation(0, new Endpoint("0.0.0.0", 6667))));
-
- Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+ TSeriesPartitionSlot seriesPartitionSlot = new TSeriesPartitionSlot(10);
+ TTimePartitionSlot timePartitionSlot = new TTimePartitionSlot(100);
+ TRegionReplicaSet regionReplicaSet = new TRegionReplicaSet();
+ regionReplicaSet.setRegionId(new TConsensusGroupId(TConsensusGroupType.DataRegion, 0));
+ regionReplicaSet.setDataNodeLocations(Collections.singletonList(dataNodeLocation));
+
+ Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
assignedDataPartition = new HashMap<>();
assignedDataPartition.put(storageGroup, new HashMap<>());
assignedDataPartition.get(storageGroup).put(seriesPartitionSlot, new HashMap<>());
@@ -191,10 +223,10 @@ public class SerializeDeserializeUT {
@Test
public void GetOrCreateDataPartitionPlanTest() throws IOException {
String storageGroup = "root.sg0";
- SeriesPartitionSlot seriesPartitionSlot = new SeriesPartitionSlot(10);
- TimePartitionSlot timePartitionSlot = new TimePartitionSlot(100);
+ TSeriesPartitionSlot seriesPartitionSlot = new TSeriesPartitionSlot(10);
+ TTimePartitionSlot timePartitionSlot = new TTimePartitionSlot(100);
- Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> partitionSlotsMap =
+ Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> partitionSlotsMap =
new HashMap<>();
partitionSlotsMap.put(storageGroup, new HashMap<>());
partitionSlotsMap.get(storageGroup).put(seriesPartitionSlot, new ArrayList<>());
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessorTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessorTest.java
index 4cfd90e65f..e40f601a01 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessorTest.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/service/thrift/server/ConfigNodeRPCServerProcessorTest.java
@@ -18,23 +18,21 @@
*/
package org.apache.iotdb.confignode.service.thrift.server;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.commons.conf.IoTDBConstant;
-import org.apache.iotdb.commons.consensus.ConsensusGroupId;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.consensus.SchemaRegionId;
import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
import org.apache.iotdb.confignode.persistence.DataNodeInfoPersistence;
import org.apache.iotdb.confignode.persistence.PartitionInfoPersistence;
import org.apache.iotdb.confignode.persistence.RegionInfoPersistence;
import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerReq;
import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerResp;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessage;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessageResp;
+import org.apache.iotdb.confignode.rpc.thrift.TDataNodeLocationResp;
import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq;
import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterResp;
import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq;
@@ -104,56 +102,71 @@ public class ConfigNodeRPCServerProcessorTest {
globalConfig.getSeriesPartitionExecutorClass());
}
+ private void registerDataNodes() throws TException {
+ for (int i = 0; i < 3; i++) {
+ TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+ dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667 + i));
+ dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003 + i));
+ dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777 + i));
+ dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010 + i));
+
+ TDataNodeRegisterReq req = new TDataNodeRegisterReq(dataNodeLocation);
+ TDataNodeRegisterResp resp = processor.registerDataNode(req);
+
+ Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
+ Assert.assertEquals(i, resp.getDataNodeId());
+ checkGlobalConfig(resp.getGlobalConfig());
+ }
+ }
+
@Test
public void registerAndQueryDataNodeTest() throws TException {
- TDataNodeRegisterResp resp;
- TDataNodeRegisterReq registerReq0 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6667));
- TDataNodeRegisterReq registerReq1 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6668));
- TDataNodeRegisterReq registerReq2 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6669));
-
- // test success register
- resp = processor.registerDataNode(registerReq0);
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
- Assert.assertEquals(0, resp.getDataNodeID());
- checkGlobalConfig(resp.getGlobalConfig());
-
- resp = processor.registerDataNode(registerReq1);
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
- Assert.assertEquals(1, resp.getDataNodeID());
- checkGlobalConfig(resp.getGlobalConfig());
-
- resp = processor.registerDataNode(registerReq2);
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
- Assert.assertEquals(2, resp.getDataNodeID());
- checkGlobalConfig(resp.getGlobalConfig());
+ registerDataNodes();
// test success re-register
- resp = processor.registerDataNode(registerReq1);
+ TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+ dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6668));
+ dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9004));
+ dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8778));
+ dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40011));
+
+ TDataNodeRegisterReq req = new TDataNodeRegisterReq(dataNodeLocation);
+ TDataNodeRegisterResp resp = processor.registerDataNode(req);
Assert.assertEquals(
TSStatusCode.DATANODE_ALREADY_REGISTERED.getStatusCode(), resp.getStatus().getCode());
- Assert.assertEquals(1, resp.getDataNodeID());
+ Assert.assertEquals(1, resp.getDataNodeId());
checkGlobalConfig(resp.getGlobalConfig());
// test query DataNodeInfo
- TDataNodeMessageResp msgResp = processor.getDataNodesMessage(-1);
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), msgResp.getStatus().getCode());
- Map<Integer, TDataNodeMessage> msgMap = msgResp.getDataNodeMessageMap();
- Assert.assertEquals(3, msgMap.size());
- List<Map.Entry<Integer, TDataNodeMessage>> messageList = new ArrayList<>(msgMap.entrySet());
- messageList.sort(Comparator.comparingInt(Map.Entry::getKey));
+ TDataNodeLocationResp locationResp = processor.getDataNodeLocations(-1);
+ Assert.assertEquals(
+ TSStatusCode.SUCCESS_STATUS.getStatusCode(), locationResp.getStatus().getCode());
+ Map<Integer, TDataNodeLocation> locationMap = locationResp.getDataNodeLocationMap();
+ Assert.assertEquals(3, locationMap.size());
+ List<Map.Entry<Integer, TDataNodeLocation>> locationList =
+ new ArrayList<>(locationMap.entrySet());
+ locationList.sort(Comparator.comparingInt(Map.Entry::getKey));
for (int i = 0; i < 3; i++) {
- Assert.assertEquals(i, messageList.get(i).getValue().getDataNodeId());
- Assert.assertEquals("0.0.0.0", messageList.get(i).getValue().getEndPoint().getIp());
- Assert.assertEquals(6667 + i, messageList.get(i).getValue().getEndPoint().getPort());
+ dataNodeLocation.setDataNodeId(i);
+ dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667 + i));
+ dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003 + i));
+ dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777 + i));
+ dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010 + i));
+ Assert.assertEquals(dataNodeLocation, locationList.get(i).getValue());
}
- msgResp = processor.getDataNodesMessage(1);
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), msgResp.getStatus().getCode());
- msgMap = msgResp.getDataNodeMessageMap();
- Assert.assertEquals(1, msgMap.size());
- Assert.assertNotNull(msgMap.get(1));
- Assert.assertEquals("0.0.0.0", msgMap.get(1).getEndPoint().getIp());
- Assert.assertEquals(6668, msgMap.get(1).getEndPoint().getPort());
+ locationResp = processor.getDataNodeLocations(1);
+ Assert.assertEquals(
+ TSStatusCode.SUCCESS_STATUS.getStatusCode(), locationResp.getStatus().getCode());
+ locationMap = locationResp.getDataNodeLocationMap();
+ Assert.assertEquals(1, locationMap.size());
+ Assert.assertNotNull(locationMap.get(1));
+ dataNodeLocation.setDataNodeId(1);
+ dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6668));
+ dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9004));
+ dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8778));
+ dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40011));
+ Assert.assertEquals(dataNodeLocation, locationMap.get(1));
}
@Test
@@ -162,21 +175,13 @@ public class ConfigNodeRPCServerProcessorTest {
final String sg = "root.sg0";
// failed because there are not enough DataNodes
- TSetStorageGroupReq setReq = new TSetStorageGroupReq(sg);
+ TSetStorageGroupReq setReq = new TSetStorageGroupReq(new TStorageGroupSchema(sg));
status = processor.setStorageGroup(setReq);
Assert.assertEquals(TSStatusCode.NOT_ENOUGH_DATA_NODE.getStatusCode(), status.getCode());
Assert.assertEquals("DataNode is not enough, please register more.", status.getMessage());
// register DataNodes
- TDataNodeRegisterReq registerReq0 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6667));
- TDataNodeRegisterReq registerReq1 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6668));
- TDataNodeRegisterReq registerReq2 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6669));
- status = processor.registerDataNode(registerReq0).getStatus();
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
- status = processor.registerDataNode(registerReq1).getStatus();
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
- status = processor.registerDataNode(registerReq2).getStatus();
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
+ registerDataNodes();
// set StorageGroup
status = processor.setStorageGroup(setReq);
@@ -185,10 +190,15 @@ public class ConfigNodeRPCServerProcessorTest {
// query StorageGroupSchema
TStorageGroupSchemaResp resp = processor.getStorageGroupsSchema();
Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), resp.getStatus().getCode());
- Map<String, TStorageGroupSchema> msgMap = resp.getStorageGroupSchemaMap();
- Assert.assertEquals(1, msgMap.size());
- Assert.assertNotNull(msgMap.get(sg));
- Assert.assertEquals(sg, msgMap.get(sg).getStorageGroup());
+ Map<String, TStorageGroupSchema> schemaMap = resp.getStorageGroupSchemaMap();
+ Assert.assertEquals(1, schemaMap.size());
+ TStorageGroupSchema storageGroupSchema = schemaMap.get(sg);
+ Assert.assertNotNull(storageGroupSchema);
+ Assert.assertEquals(sg, storageGroupSchema.getName());
+ Assert.assertEquals(Long.MAX_VALUE, storageGroupSchema.getTTL());
+ Assert.assertEquals(3, storageGroupSchema.getSchemaReplicationFactor());
+ Assert.assertEquals(3, storageGroupSchema.getDataReplicationFactor());
+ Assert.assertEquals(604800, storageGroupSchema.getTimePartitionInterval());
// test fail by re-register
status = processor.setStorageGroup(setReq);
@@ -234,20 +244,12 @@ public class ConfigNodeRPCServerProcessorTest {
Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap;
// register DataNodes
- TDataNodeRegisterReq registerReq0 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6667));
- TDataNodeRegisterReq registerReq1 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6668));
- TDataNodeRegisterReq registerReq2 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6669));
- status = processor.registerDataNode(registerReq0).getStatus();
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
- status = processor.registerDataNode(registerReq1).getStatus();
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
- status = processor.registerDataNode(registerReq2).getStatus();
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
+ registerDataNodes();
// Set StorageGroups
- status = processor.setStorageGroup(new TSetStorageGroupReq(sg0));
+ status = processor.setStorageGroup(new TSetStorageGroupReq(new TStorageGroupSchema(sg0)));
Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
- status = processor.setStorageGroup(new TSetStorageGroupReq(sg1));
+ status = processor.setStorageGroup(new TSetStorageGroupReq(new TStorageGroupSchema(sg1)));
Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
// Test getSchemaPartition, the result should be empty
@@ -273,13 +275,9 @@ public class ConfigNodeRPCServerProcessorTest {
.get(sg + i)
.forEach(
(tSeriesPartitionSlot, tRegionReplicaSet) -> {
- Assert.assertEquals(3, tRegionReplicaSet.getEndpointSize());
- ConsensusGroupId regionId = null;
- regionId =
- ConsensusGroupId.Factory.create(
- ByteBuffer.wrap(tRegionReplicaSet.getRegionId()));
-
- Assert.assertTrue(regionId instanceof SchemaRegionId);
+ Assert.assertEquals(3, tRegionReplicaSet.getDataNodeLocationsSize());
+ Assert.assertEquals(
+ TConsensusGroupType.SchemaRegion, tRegionReplicaSet.getRegionId().getType());
});
}
@@ -300,14 +298,9 @@ public class ConfigNodeRPCServerProcessorTest {
.get(sg + i)
.forEach(
(tSeriesPartitionSlot, tRegionReplicaSet) -> {
- Assert.assertEquals(3, tRegionReplicaSet.getEndpointSize());
- ConsensusGroupId regionId = null;
-
- regionId =
- ConsensusGroupId.Factory.create(
- ByteBuffer.wrap(tRegionReplicaSet.getRegionId()));
-
- Assert.assertTrue(regionId instanceof SchemaRegionId);
+ Assert.assertEquals(3, tRegionReplicaSet.getDataNodeLocationsSize());
+ Assert.assertEquals(
+ TConsensusGroupType.SchemaRegion, tRegionReplicaSet.getRegionId().getType());
});
}
@@ -327,11 +320,9 @@ public class ConfigNodeRPCServerProcessorTest {
.get(sg0)
.forEach(
(tSeriesPartitionSlot, tRegionReplicaSet) -> {
- Assert.assertEquals(3, tRegionReplicaSet.getEndpointSize());
- ConsensusGroupId regionId = null;
- regionId =
- ConsensusGroupId.Factory.create(ByteBuffer.wrap(tRegionReplicaSet.getRegionId()));
- Assert.assertTrue(regionId instanceof SchemaRegionId);
+ Assert.assertEquals(3, tRegionReplicaSet.getDataNodeLocationsSize());
+ Assert.assertEquals(
+ TConsensusGroupType.SchemaRegion, tRegionReplicaSet.getRegionId().getType());
});
// Check "root.sg1"
Assert.assertTrue(schemaPartitionMap.containsKey(sg1));
@@ -340,11 +331,9 @@ public class ConfigNodeRPCServerProcessorTest {
.get(sg1)
.forEach(
(tSeriesPartitionSlot, tRegionReplicaSet) -> {
- Assert.assertEquals(3, tRegionReplicaSet.getEndpointSize());
- ConsensusGroupId regionId = null;
- regionId =
- ConsensusGroupId.Factory.create(ByteBuffer.wrap(tRegionReplicaSet.getRegionId()));
- Assert.assertTrue(regionId instanceof SchemaRegionId);
+ Assert.assertEquals(3, tRegionReplicaSet.getDataNodeLocationsSize());
+ Assert.assertEquals(
+ TConsensusGroupType.SchemaRegion, tRegionReplicaSet.getRegionId().getType());
});
}
@@ -404,17 +393,15 @@ public class ConfigNodeRPCServerProcessorTest {
.get(timePartitionSlot)
.size());
// Is DataRegion
- ConsensusGroupId regionId = null;
- regionId =
- ConsensusGroupId.Factory.create(
- ByteBuffer.wrap(
- dataPartitionMap
- .get(storageGroup)
- .get(seriesPartitionSlot)
- .get(timePartitionSlot)
- .get(0)
- .getRegionId()));
- Assert.assertTrue(regionId instanceof DataRegionId);
+ Assert.assertEquals(
+ TConsensusGroupType.DataRegion,
+ dataPartitionMap
+ .get(storageGroup)
+ .get(seriesPartitionSlot)
+ .get(timePartitionSlot)
+ .get(0)
+ .getRegionId()
+ .getType());
// Including three RegionReplica
Assert.assertEquals(
3,
@@ -423,7 +410,7 @@ public class ConfigNodeRPCServerProcessorTest {
.get(seriesPartitionSlot)
.get(timePartitionSlot)
.get(0)
- .getEndpointSize());
+ .getDataNodeLocationsSize());
}
}
}
@@ -441,15 +428,7 @@ public class ConfigNodeRPCServerProcessorTest {
TDataPartitionResp dataPartitionResp;
// register DataNodes
- TDataNodeRegisterReq registerReq0 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6667));
- TDataNodeRegisterReq registerReq1 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6668));
- TDataNodeRegisterReq registerReq2 = new TDataNodeRegisterReq(new TEndPoint("0.0.0.0", 6669));
- status = processor.registerDataNode(registerReq0).getStatus();
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
- status = processor.registerDataNode(registerReq1).getStatus();
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
- status = processor.registerDataNode(registerReq2).getStatus();
- Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
+ registerDataNodes();
// Prepare partitionSlotsMap
Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> partitionSlotsMap0 =
@@ -460,7 +439,7 @@ public class ConfigNodeRPCServerProcessorTest {
// set StorageGroups
for (int i = 0; i < storageGroupNum; i++) {
- TSetStorageGroupReq setReq = new TSetStorageGroupReq(sg + i);
+ TSetStorageGroupReq setReq = new TSetStorageGroupReq(new TStorageGroupSchema(sg + i));
status = processor.setStorageGroup(setReq);
Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
}
diff --git a/consensus/pom.xml b/consensus/pom.xml
index 857276360e..f773dd846a 100644
--- a/consensus/pom.xml
+++ b/consensus/pom.xml
@@ -46,11 +46,6 @@
<artifactId>ratis-grpc</artifactId>
<version>2.2.0</version>
</dependency>
- <dependency>
- <groupId>org.apache.iotdb</groupId>
- <artifactId>iotdb-thrift</artifactId>
- <version>${project.version}</version>
- </dependency>
<dependency>
<groupId>org.apache.iotdb</groupId>
<artifactId>node-commons</artifactId>
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java b/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java
index 8d218ac9f3..d9364ad13e 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ConsensusFactory.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.consensus;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.consensus.statemachine.IStateMachine;
import org.slf4j.Logger;
@@ -37,11 +37,12 @@ public class ConsensusFactory {
private static final Logger logger = LoggerFactory.getLogger(ConsensusFactory.class);
public static Optional<IConsensus> getConsensusImpl(
- String className, Endpoint endpoint, File storageDir, IStateMachine.Registry registry) {
+ String className, TEndPoint endpoint, File storageDir, IStateMachine.Registry registry) {
try {
Class<?> executor = Class.forName(className);
Constructor<?> executorConstructor =
- executor.getDeclaredConstructor(Endpoint.class, File.class, IStateMachine.Registry.class);
+ executor.getDeclaredConstructor(
+ TEndPoint.class, File.class, IStateMachine.Registry.class);
executorConstructor.setAccessible(true);
return Optional.of(
(IConsensus) executorConstructor.newInstance(endpoint, storageDir, registry));
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/common/Peer.java b/consensus/src/main/java/org/apache/iotdb/consensus/common/Peer.java
index 4e7b8f1743..db537e9022 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/common/Peer.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/common/Peer.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.consensus.common;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.commons.consensus.ConsensusGroupId;
import java.util.Objects;
@@ -28,9 +28,9 @@ import java.util.Objects;
public class Peer {
private final ConsensusGroupId groupId;
- private final Endpoint endpoint;
+ private final TEndPoint endpoint;
- public Peer(ConsensusGroupId groupId, Endpoint endpoint) {
+ public Peer(ConsensusGroupId groupId, TEndPoint endpoint) {
this.groupId = groupId;
this.endpoint = endpoint;
}
@@ -39,7 +39,7 @@ public class Peer {
return groupId;
}
- public Endpoint getEndpoint() {
+ public TEndPoint getEndpoint() {
return endpoint;
}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java
index 327ce09639..29b822da8c 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java
@@ -21,7 +21,6 @@ package org.apache.iotdb.consensus.ratis;
import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
import org.apache.iotdb.commons.consensus.ConsensusGroupId;
import org.apache.iotdb.consensus.IConsensus;
import org.apache.iotdb.consensus.common.DataSet;
@@ -90,7 +89,7 @@ class RatisConsensus implements IConsensus {
private Logger logger = LoggerFactory.getLogger(RatisConsensus.class);
- public RatisConsensus(Endpoint endpoint, File ratisStorageDir, IStateMachine.Registry registry)
+ public RatisConsensus(TEndPoint endpoint, File ratisStorageDir, IStateMachine.Registry registry)
throws IOException {
this.clientMap = new ConcurrentHashMap<>();
@@ -180,7 +179,7 @@ class RatisConsensus implements IConsensus {
}
if (suggestedLeader != null) {
- Endpoint leaderEndPoint = Utils.getEndpoint(suggestedLeader);
+ TEndPoint leaderEndPoint = Utils.getEndpoint(suggestedLeader);
writeResult.setRedirectNode(new TEndPoint(leaderEndPoint.getIp(), leaderEndPoint.getPort()));
}
@@ -460,7 +459,7 @@ class RatisConsensus implements IConsensus {
if (client == null) {
return null;
}
- Endpoint leaderEndpoint = Utils.parseFromRatisId(client.getLeaderId().toString());
+ TEndPoint leaderEndpoint = Utils.parseFromRatisId(client.getLeaderId().toString());
return new Peer(groupId, leaderEndpoint);
}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RequestMessage.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RequestMessage.java
index 5607c805ac..9dc73d815c 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RequestMessage.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RequestMessage.java
@@ -34,7 +34,7 @@ public class RequestMessage implements Message {
private final IConsensusRequest actualRequest;
private volatile ByteString serializedContent;
- private final int DEFAULT_BUFFER_SIZE = 1024 * 10;
+ private static final int DEFAULT_BUFFER_SIZE = 2048 * 10;
public RequestMessage(IConsensusRequest request) {
this.actualRequest = request;
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java
index d85efae7e2..ff3210717b 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java
@@ -18,8 +18,8 @@
*/
package org.apache.iotdb.consensus.ratis;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
import org.apache.iotdb.commons.consensus.ConsensusGroupId;
import org.apache.iotdb.commons.consensus.DataRegionId;
import org.apache.iotdb.commons.consensus.PartitionRegionId;
@@ -44,7 +44,7 @@ public class Utils {
private static final String SchemaRegionAbbr = "SR";
private static final String PartitionRegionAbbr = "PR";
- public static String IPAddress(Endpoint endpoint) {
+ public static String IPAddress(TEndPoint endpoint) {
return String.format("%s:%d", endpoint.getIp(), endpoint.getPort());
}
@@ -71,17 +71,17 @@ public class Utils {
return String.format("%s-%d", groupTypeAbbr, consensusGroupId.getId());
}
- public static String RatisPeerId(Endpoint endpoint) {
+ public static String RatisPeerId(TEndPoint endpoint) {
return String.format("%s-%d", endpoint.getIp(), endpoint.getPort());
}
- public static Endpoint parseFromRatisId(String ratisId) {
+ public static TEndPoint parseFromRatisId(String ratisId) {
String[] items = ratisId.split("-");
- return new Endpoint(items[0], Integer.parseInt(items[1]));
+ return new TEndPoint(items[0], Integer.parseInt(items[1]));
}
// priority is used as ordinal of leader election
- public static RaftPeer toRaftPeer(Endpoint endpoint, int priority) {
+ public static RaftPeer toRaftPeer(TEndPoint endpoint, int priority) {
return RaftPeer.newBuilder()
.setId(RatisPeerId(endpoint))
.setAddress(IPAddress(endpoint))
@@ -93,10 +93,10 @@ public class Utils {
return toRaftPeer(peer.getEndpoint(), priority);
}
- public static Endpoint getEndpoint(RaftPeer raftPeer) {
+ public static TEndPoint getEndpoint(RaftPeer raftPeer) {
String address = raftPeer.getAddress(); // ip:port
String[] split = address.split(":");
- return new Endpoint(split[0], Integer.parseInt(split[1]));
+ return new TEndPoint(split[0], Integer.parseInt(split[1]));
}
/** Given ConsensusGroupId, generate a deterministic RaftGroupId current scheme: */
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/standalone/StandAloneConsensus.java b/consensus/src/main/java/org/apache/iotdb/consensus/standalone/StandAloneConsensus.java
index 7365bbe855..4650db4c16 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/standalone/StandAloneConsensus.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/standalone/StandAloneConsensus.java
@@ -19,8 +19,8 @@
package org.apache.iotdb.consensus.standalone;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
import org.apache.iotdb.commons.consensus.ConsensusGroupId;
import org.apache.iotdb.consensus.IConsensus;
import org.apache.iotdb.consensus.common.DataSet;
@@ -53,13 +53,13 @@ import java.util.concurrent.atomic.AtomicReference;
*/
class StandAloneConsensus implements IConsensus {
- private final Endpoint thisNode;
+ private final TEndPoint thisNode;
private final File storageDir;
private final IStateMachine.Registry registry;
private final Map<ConsensusGroupId, StandAloneServerImpl> stateMachineMap =
new ConcurrentHashMap<>();
- public StandAloneConsensus(Endpoint thisNode, File storageDir, Registry registry) {
+ public StandAloneConsensus(TEndPoint thisNode, File storageDir, Registry registry) {
this.thisNode = thisNode;
this.storageDir = storageDir;
this.registry = registry;
diff --git a/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RatisConsensusTest.java b/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RatisConsensusTest.java
index f0e5093f0a..f132dbbf01 100644
--- a/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RatisConsensusTest.java
+++ b/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RatisConsensusTest.java
@@ -18,8 +18,8 @@
*/
package org.apache.iotdb.consensus.ratis;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
import org.apache.iotdb.commons.consensus.ConsensusGroupId;
import org.apache.iotdb.commons.consensus.DataRegionId;
import org.apache.iotdb.consensus.ConsensusFactory;
@@ -137,9 +137,9 @@ public class RatisConsensusTest {
public void setUp() throws IOException {
gid = new DataRegionId(1);
peers = new ArrayList<>();
- peer0 = new Peer(gid, new Endpoint("127.0.0.1", 6000));
- peer1 = new Peer(gid, new Endpoint("127.0.0.1", 6001));
- peer2 = new Peer(gid, new Endpoint("127.0.0.1", 6002));
+ peer0 = new Peer(gid, new TEndPoint("127.0.0.1", 6000));
+ peer1 = new Peer(gid, new TEndPoint("127.0.0.1", 6001));
+ peer2 = new Peer(gid, new TEndPoint("127.0.0.1", 6002));
peers.add(peer0);
peers.add(peer1);
peers.add(peer2);
diff --git a/consensus/src/test/java/org/apache/iotdb/consensus/standalone/StandAloneConsensusTest.java b/consensus/src/test/java/org/apache/iotdb/consensus/standalone/StandAloneConsensusTest.java
index b7eb47d975..1482f21c04 100644
--- a/consensus/src/test/java/org/apache/iotdb/consensus/standalone/StandAloneConsensusTest.java
+++ b/consensus/src/test/java/org/apache/iotdb/consensus/standalone/StandAloneConsensusTest.java
@@ -19,8 +19,8 @@
package org.apache.iotdb.consensus.standalone;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
import org.apache.iotdb.commons.consensus.ConsensusGroupId;
import org.apache.iotdb.commons.consensus.DataRegionId;
import org.apache.iotdb.commons.consensus.PartitionRegionId;
@@ -131,7 +131,7 @@ public class StandAloneConsensusTest {
consensusImpl =
ConsensusFactory.getConsensusImpl(
STANDALONE_CONSENSUS_CLASS_NAME,
- new Endpoint("localhost", 6667),
+ new TEndPoint("localhost", 6667),
new File("./"),
gid -> {
switch (gid.getType()) {
@@ -161,14 +161,14 @@ public class StandAloneConsensusTest {
ConsensusGenericResponse response1 =
consensusImpl.addConsensusGroup(
dataRegionId,
- Collections.singletonList(new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667))));
+ Collections.singletonList(new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667))));
assertTrue(response1.isSuccess());
assertNull(response1.getException());
ConsensusGenericResponse response2 =
consensusImpl.addConsensusGroup(
dataRegionId,
- Collections.singletonList(new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667))));
+ Collections.singletonList(new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667))));
assertFalse(response2.isSuccess());
assertTrue(response2.getException() instanceof ConsensusGroupAlreadyExistException);
@@ -176,15 +176,15 @@ public class StandAloneConsensusTest {
consensusImpl.addConsensusGroup(
dataRegionId,
Arrays.asList(
- new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667)),
- new Peer(dataRegionId, new Endpoint("0.0.0.1", 6667))));
+ new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667)),
+ new Peer(dataRegionId, new TEndPoint("0.0.0.1", 6667))));
assertFalse(response3.isSuccess());
assertTrue(response3.getException() instanceof IllegalPeerNumException);
ConsensusGenericResponse response4 =
consensusImpl.addConsensusGroup(
schemaRegionId,
- Collections.singletonList(new Peer(schemaRegionId, new Endpoint("0.0.0.0", 6667))));
+ Collections.singletonList(new Peer(schemaRegionId, new TEndPoint("0.0.0.0", 6667))));
assertTrue(response4.isSuccess());
assertNull(response4.getException());
}
@@ -198,7 +198,7 @@ public class StandAloneConsensusTest {
ConsensusGenericResponse response2 =
consensusImpl.addConsensusGroup(
dataRegionId,
- Collections.singletonList(new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667))));
+ Collections.singletonList(new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667))));
assertTrue(response2.isSuccess());
assertNull(response2.getException());
@@ -210,7 +210,7 @@ public class StandAloneConsensusTest {
@Test
public void addPeer() {
ConsensusGenericResponse response =
- consensusImpl.addPeer(dataRegionId, new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667)));
+ consensusImpl.addPeer(dataRegionId, new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667)));
assertFalse(response.isSuccess());
}
@@ -218,7 +218,7 @@ public class StandAloneConsensusTest {
public void removePeer() {
ConsensusGenericResponse response =
consensusImpl.removePeer(
- dataRegionId, new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667)));
+ dataRegionId, new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667)));
assertFalse(response.isSuccess());
}
@@ -227,7 +227,7 @@ public class StandAloneConsensusTest {
ConsensusGenericResponse response =
consensusImpl.changePeer(
dataRegionId,
- Collections.singletonList(new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667))));
+ Collections.singletonList(new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667))));
assertFalse(response.isSuccess());
}
@@ -235,7 +235,7 @@ public class StandAloneConsensusTest {
public void transferLeader() {
ConsensusGenericResponse response =
consensusImpl.transferLeader(
- dataRegionId, new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667)));
+ dataRegionId, new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667)));
assertFalse(response.isSuccess());
}
@@ -250,20 +250,21 @@ public class StandAloneConsensusTest {
ConsensusGenericResponse response1 =
consensusImpl.addConsensusGroup(
dataRegionId,
- Collections.singletonList(new Peer(dataRegionId, new Endpoint("0.0.0.0", 6667))));
+ Collections.singletonList(new Peer(dataRegionId, new TEndPoint("0.0.0.0", 6667))));
assertTrue(response1.isSuccess());
assertNull(response1.getException());
ConsensusGenericResponse response2 =
consensusImpl.addConsensusGroup(
schemaRegionId,
- Collections.singletonList(new Peer(schemaRegionId, new Endpoint("0.0.0.0", 6667))));
+ Collections.singletonList(new Peer(schemaRegionId, new TEndPoint("0.0.0.0", 6667))));
assertTrue(response2.isSuccess());
assertNull(response2.getException());
ConsensusGenericResponse response3 =
consensusImpl.addConsensusGroup(
- configId, Collections.singletonList(new Peer(configId, new Endpoint("0.0.0.0", 6667))));
+ configId,
+ Collections.singletonList(new Peer(configId, new TEndPoint("0.0.0.0", 6667))));
assertTrue(response3.isSuccess());
assertNull(response3.getException());
diff --git a/node-commons/pom.xml b/node-commons/pom.xml
index 3997cc6c00..999bbfd373 100644
--- a/node-commons/pom.xml
+++ b/node-commons/pom.xml
@@ -51,6 +51,16 @@
<artifactId>guava</artifactId>
<version>[${guava.version},)</version>
</dependency>
+ <dependency>
+ <groupId>org.apache.iotdb</groupId>
+ <artifactId>iotdb-thrift</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.iotdb</groupId>
+ <artifactId>iotdb-thrift-confignode</artifactId>
+ <version>${project.version}</version>
+ </dependency>
</dependencies>
<build>
<plugins>
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/cluster/DataNodeLocation.java b/node-commons/src/main/java/org/apache/iotdb/commons/cluster/DataNodeLocation.java
deleted file mode 100644
index 6223ac43b2..0000000000
--- a/node-commons/src/main/java/org/apache/iotdb/commons/cluster/DataNodeLocation.java
+++ /dev/null
@@ -1,81 +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.iotdb.commons.cluster;
-
-import java.nio.ByteBuffer;
-import java.util.Objects;
-
-public class DataNodeLocation {
-
- private int dataNodeId;
- private Endpoint endPoint;
-
- public DataNodeLocation() {}
-
- public DataNodeLocation(int dataNodeId, Endpoint endPoint) {
- this.dataNodeId = dataNodeId;
- this.endPoint = endPoint;
- }
-
- public DataNodeLocation(Endpoint endPoint) {
- this.endPoint = endPoint;
- }
-
- public int getDataNodeId() {
- return dataNodeId;
- }
-
- public void setDataNodeId(int dataNodeId) {
- this.dataNodeId = dataNodeId;
- }
-
- public Endpoint getEndPoint() {
- return endPoint;
- }
-
- public void serializeImpl(ByteBuffer buffer) {
- buffer.putInt(dataNodeId);
- endPoint.serializeImpl(buffer);
- }
-
- public static DataNodeLocation deserializeImpl(ByteBuffer buffer) {
- return new DataNodeLocation(buffer.getInt(), Endpoint.deserializeImpl(buffer));
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- DataNodeLocation that = (DataNodeLocation) o;
- return dataNodeId == that.dataNodeId && Objects.equals(endPoint, that.endPoint);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(dataNodeId, endPoint);
- }
-
- public String toString() {
- return String.format("DataNode[%d, %s]", dataNodeId, endPoint);
- }
-}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/cluster/Endpoint.java b/node-commons/src/main/java/org/apache/iotdb/commons/cluster/Endpoint.java
deleted file mode 100644
index b5c15cc85f..0000000000
--- a/node-commons/src/main/java/org/apache/iotdb/commons/cluster/Endpoint.java
+++ /dev/null
@@ -1,92 +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.iotdb.commons.cluster;
-
-import java.nio.ByteBuffer;
-import java.util.Objects;
-
-// TODO Use a mature IDL framework such as Protobuf to manage this structure
-public class Endpoint {
-
- private String ip;
- private int port;
-
- public Endpoint() {}
-
- public Endpoint(String ip, int port) {
- this.ip = ip;
- this.port = port;
- }
-
- public String getIp() {
- return ip;
- }
-
- public Endpoint setIp(String ip) {
- this.ip = ip;
- return this;
- }
-
- public int getPort() {
- return port;
- }
-
- public Endpoint setPort(int port) {
- this.port = port;
- return this;
- }
-
- public void serializeImpl(ByteBuffer buffer) {
- byte[] bytes = ip.getBytes();
- buffer.putInt(bytes.length);
- buffer.put(bytes);
-
- buffer.putInt(port);
- }
-
- public static Endpoint deserializeImpl(ByteBuffer buffer) {
- int length = buffer.getInt();
- byte[] bytes = new byte[length];
- buffer.get(bytes, 0, length);
- return new Endpoint(new String(bytes, 0, length), buffer.getInt());
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- Endpoint endpoint = (Endpoint) o;
- return port == endpoint.port && Objects.equals(ip, endpoint.ip);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(ip, port);
- }
-
- @Override
- public String toString() {
- return String.format("%s:%d", ip, port);
- }
-}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConsensusGroupId.java b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConsensusGroupId.java
index ba38960173..aee4bcee55 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConsensusGroupId.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/ConsensusGroupId.java
@@ -19,37 +19,21 @@
package org.apache.iotdb.commons.consensus;
-import java.nio.ByteBuffer;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
public interface ConsensusGroupId {
- // contains specific id and type
- void serializeImpl(ByteBuffer buffer);
-
- // only deserialize specific id
- void deserializeImpl(ByteBuffer buffer);
-
// return specific id
int getId();
void setId(int id);
// return specific type
- GroupType getType();
+ TConsensusGroupType getType();
class Factory {
- public static ConsensusGroupId create(ByteBuffer buffer) {
- int index = buffer.get();
- if (index >= GroupType.values().length) {
- throw new IllegalArgumentException("invalid ConsensusGroup type. Ordinal is: " + index);
- }
- GroupType type = GroupType.values()[index];
- ConsensusGroupId groupId = createEmpty(type);
- groupId.deserializeImpl(buffer);
- return groupId;
- }
-
- public static ConsensusGroupId createEmpty(GroupType type) {
+ public static ConsensusGroupId createEmpty(TConsensusGroupType type) {
ConsensusGroupId groupId;
switch (type) {
case DataRegion:
@@ -67,10 +51,24 @@ public interface ConsensusGroupId {
return groupId;
}
- public static ConsensusGroupId create(int id, GroupType type) {
- ConsensusGroupId groupId = createEmpty(type);
- groupId.setId(id);
+ public static ConsensusGroupId convertFromTConsensusGroupId(
+ TConsensusGroupId tConsensusGroupId) {
+ ConsensusGroupId groupId = createEmpty(tConsensusGroupId.getType());
+ groupId.setId(tConsensusGroupId.getId());
return groupId;
}
+
+ public static TConsensusGroupId convertToTConsensusGroupId(ConsensusGroupId consensusGroupId) {
+ TConsensusGroupId result = new TConsensusGroupId();
+ if (consensusGroupId instanceof SchemaRegionId) {
+ result.setType(TConsensusGroupType.SchemaRegion);
+ } else if (consensusGroupId instanceof DataRegionId) {
+ result.setType(TConsensusGroupType.DataRegion);
+ } else if (consensusGroupId instanceof PartitionRegionId) {
+ result.setType(TConsensusGroupType.PartitionRegion);
+ }
+ result.setId(consensusGroupId.getId());
+ return result;
+ }
}
}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/DataRegionId.java b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/DataRegionId.java
index b0c5d1ec46..81fed3fe62 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/DataRegionId.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/DataRegionId.java
@@ -19,7 +19,8 @@
package org.apache.iotdb.commons.consensus;
-import java.nio.ByteBuffer;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+
import java.util.Objects;
public class DataRegionId implements ConsensusGroupId {
@@ -32,18 +33,6 @@ public class DataRegionId implements ConsensusGroupId {
this.id = id;
}
- @Override
- public void serializeImpl(ByteBuffer buffer) {
- buffer.put((byte) GroupType.DataRegion.ordinal());
- buffer.putInt(id);
- }
-
- @Override
- public void deserializeImpl(ByteBuffer buffer) {
- // TODO: (xingtanzjr) should we add validation for the ordinal ?
- id = buffer.getInt();
- }
-
@Override
public int getId() {
return id;
@@ -55,8 +44,8 @@ public class DataRegionId implements ConsensusGroupId {
}
@Override
- public GroupType getType() {
- return GroupType.DataRegion;
+ public TConsensusGroupType getType() {
+ return TConsensusGroupType.DataRegion;
}
@Override
@@ -73,7 +62,7 @@ public class DataRegionId implements ConsensusGroupId {
@Override
public int hashCode() {
- return Objects.hash(id, GroupType.DataRegion);
+ return Objects.hash(id, TConsensusGroupType.DataRegion);
}
public String toString() {
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/GroupType.java b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/GroupType.java
deleted file mode 100644
index 65b5163a0d..0000000000
--- a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/GroupType.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.commons.consensus;
-
-// TODO Use a mature IDL framework such as Protobuf to manage this structure
-public enum GroupType {
- PartitionRegion,
- DataRegion,
- SchemaRegion
-}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/PartitionRegionId.java b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/PartitionRegionId.java
index b9f64786c5..6f317547ed 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/PartitionRegionId.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/PartitionRegionId.java
@@ -19,7 +19,8 @@
package org.apache.iotdb.commons.consensus;
-import java.nio.ByteBuffer;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+
import java.util.Objects;
public class PartitionRegionId implements ConsensusGroupId {
@@ -32,18 +33,6 @@ public class PartitionRegionId implements ConsensusGroupId {
this.id = id;
}
- @Override
- public void serializeImpl(ByteBuffer buffer) {
- buffer.put((byte) GroupType.PartitionRegion.ordinal());
- buffer.putInt(id);
- }
-
- @Override
- public void deserializeImpl(ByteBuffer buffer) {
- // TODO: (xingtanzjr) should we add validation for the ordinal ?
- id = buffer.getInt();
- }
-
@Override
public int getId() {
return id;
@@ -55,8 +44,8 @@ public class PartitionRegionId implements ConsensusGroupId {
}
@Override
- public GroupType getType() {
- return GroupType.PartitionRegion;
+ public TConsensusGroupType getType() {
+ return TConsensusGroupType.PartitionRegion;
}
@Override
@@ -73,7 +62,7 @@ public class PartitionRegionId implements ConsensusGroupId {
@Override
public int hashCode() {
- return Objects.hash(id, GroupType.PartitionRegion);
+ return Objects.hash(id, TConsensusGroupType.PartitionRegion);
}
public String toString() {
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/SchemaRegionId.java b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/SchemaRegionId.java
index 9649653c7f..61c4ca80c5 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/consensus/SchemaRegionId.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/consensus/SchemaRegionId.java
@@ -19,7 +19,8 @@
package org.apache.iotdb.commons.consensus;
-import java.nio.ByteBuffer;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+
import java.util.Objects;
public class SchemaRegionId implements ConsensusGroupId {
@@ -32,18 +33,6 @@ public class SchemaRegionId implements ConsensusGroupId {
this.id = id;
}
- @Override
- public void serializeImpl(ByteBuffer buffer) {
- buffer.put((byte) GroupType.SchemaRegion.ordinal());
- buffer.putInt(id);
- }
-
- @Override
- public void deserializeImpl(ByteBuffer buffer) {
- // TODO: (xingtanzjr) should we add validation for the ordinal ?
- id = buffer.getInt();
- }
-
@Override
public int getId() {
return id;
@@ -55,8 +44,8 @@ public class SchemaRegionId implements ConsensusGroupId {
}
@Override
- public GroupType getType() {
- return GroupType.SchemaRegion;
+ public TConsensusGroupType getType() {
+ return TConsensusGroupType.SchemaRegion;
}
@Override
@@ -73,7 +62,7 @@ public class SchemaRegionId implements ConsensusGroupId {
@Override
public int hashCode() {
- return Objects.hash(id, GroupType.SchemaRegion);
+ return Objects.hash(id, TConsensusGroupType.SchemaRegion);
}
public String toString() {
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java
index e1d24db22d..5156ba47af 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java
@@ -18,6 +18,10 @@
*/
package org.apache.iotdb.commons.partition;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@@ -28,8 +32,8 @@ import java.util.stream.Collectors;
public class DataPartition extends Partition {
- // Map<StorageGroup, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionMessage>>>>
- private Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+ // Map<StorageGroup, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionMessage>>>>
+ private Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
dataPartitionMap;
public DataPartition(String seriesSlotExecutorName, int seriesPartitionSlotNum) {
@@ -37,7 +41,7 @@ public class DataPartition extends Partition {
}
public DataPartition(
- Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+ Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
dataPartitionMap,
String seriesSlotExecutorName,
int seriesPartitionSlotNum) {
@@ -45,34 +49,34 @@ public class DataPartition extends Partition {
this.dataPartitionMap = dataPartitionMap;
}
- public Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+ public Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
getDataPartitionMap() {
return dataPartitionMap;
}
public void setDataPartitionMap(
- Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+ Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
dataPartitionMap) {
this.dataPartitionMap = dataPartitionMap;
}
- public List<RegionReplicaSet> getDataRegionReplicaSet(
- String deviceName, List<TimePartitionSlot> timePartitionSlotList) {
+ public List<TRegionReplicaSet> getDataRegionReplicaSet(
+ String deviceName, List<TTimePartitionSlot> timePartitionSlotList) {
String storageGroup = getStorageGroupByDevice(deviceName);
- SeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
+ TSeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
// TODO: (xingtanzjr) the timePartitionIdList is ignored
return dataPartitionMap.get(storageGroup).get(seriesPartitionSlot).values().stream()
.flatMap(Collection::stream)
.collect(Collectors.toList());
}
- public List<RegionReplicaSet> getDataRegionReplicaSetForWriting(
- String deviceName, List<TimePartitionSlot> timePartitionSlotList) {
+ public List<TRegionReplicaSet> getDataRegionReplicaSetForWriting(
+ String deviceName, List<TTimePartitionSlot> timePartitionSlotList) {
// A list of data region replica sets will store data in a same time partition.
// We will insert data to the last set in the list.
// TODO return the latest dataRegionReplicaSet for each time partition
String storageGroup = getStorageGroupByDevice(deviceName);
- SeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
+ TSeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
// IMPORTANT TODO: (xingtanzjr) need to handle the situation for write operation that there are
// more than 1 Regions for one timeSlot
return dataPartitionMap.get(storageGroup).get(seriesPartitionSlot).entrySet().stream()
@@ -81,14 +85,14 @@ public class DataPartition extends Partition {
.collect(Collectors.toList());
}
- public RegionReplicaSet getDataRegionReplicaSetForWriting(
- String deviceName, TimePartitionSlot timePartitionSlot) {
+ public TRegionReplicaSet getDataRegionReplicaSetForWriting(
+ String deviceName, TTimePartitionSlot timePartitionSlot) {
// A list of data region replica sets will store data in a same time partition.
// We will insert data to the last set in the list.
// TODO return the latest dataRegionReplicaSet for each time partition
String storageGroup = getStorageGroupByDevice(deviceName);
- SeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
- List<RegionReplicaSet> regions =
+ TSeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
+ List<TRegionReplicaSet> regions =
dataPartitionMap.get(storageGroup).get(seriesPartitionSlot).entrySet().stream()
.filter(entry -> entry.getKey().equals(timePartitionSlot))
.flatMap(entry -> entry.getValue().stream())
@@ -119,31 +123,30 @@ public class DataPartition extends Partition {
* Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
*/
public DataPartition getDataPartition(
- Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> partitionSlotsMap,
+ Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> partitionSlotsMap,
String seriesSlotExecutorName,
int seriesPartitionSlotNum) {
- Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>> result =
- new HashMap<>();
+ Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
+ result = new HashMap<>();
for (String storageGroupName : partitionSlotsMap.keySet()) {
// Compare StorageGroupName
if (dataPartitionMap.containsKey(storageGroupName)) {
- Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>
+ Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>
seriesTimePartitionSlotMap = dataPartitionMap.get(storageGroupName);
- for (SeriesPartitionSlot seriesPartitionSlot :
+ for (TSeriesPartitionSlot seriesPartitionSlot :
partitionSlotsMap.get(storageGroupName).keySet()) {
// Compare SeriesPartitionSlot
if (seriesTimePartitionSlotMap.containsKey(seriesPartitionSlot)) {
- Map<TimePartitionSlot, List<RegionReplicaSet>> timePartitionSlotMap =
+ Map<TTimePartitionSlot, List<TRegionReplicaSet>> timePartitionSlotMap =
seriesTimePartitionSlotMap.get(seriesPartitionSlot);
- // TODO: (xingtanzjr) optimize if timeSlotPartition is empty
if (partitionSlotsMap.get(storageGroupName).get(seriesPartitionSlot).size() == 0) {
result
.computeIfAbsent(storageGroupName, key -> new HashMap<>())
.computeIfAbsent(seriesPartitionSlot, key -> new HashMap<>())
.putAll(new HashMap<>(timePartitionSlotMap));
} else {
- for (TimePartitionSlot timePartitionSlot :
+ for (TTimePartitionSlot timePartitionSlot :
partitionSlotsMap.get(storageGroupName).get(seriesPartitionSlot)) {
// Compare TimePartitionSlot
if (timePartitionSlotMap.containsKey(timePartitionSlot)) {
@@ -172,23 +175,23 @@ public class DataPartition extends Partition {
* @return Map<StorageGroupName, Map < SeriesPartitionSlot, List < TimePartitionSlot>>>,
* unassigned PartitionSlots
*/
- public Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>>
+ public Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>>
filterNoAssignedDataPartitionSlots(
- Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> partitionSlotsMap) {
- Map<String, Map<SeriesPartitionSlot, List<TimePartitionSlot>>> result = new HashMap<>();
+ Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> partitionSlotsMap) {
+ Map<String, Map<TSeriesPartitionSlot, List<TTimePartitionSlot>>> result = new HashMap<>();
for (String storageGroupName : partitionSlotsMap.keySet()) {
// Compare StorageGroupName
if (dataPartitionMap.containsKey(storageGroupName)) {
- Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>
+ Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>
seriesTimePartitionSlotMap = dataPartitionMap.get(storageGroupName);
- for (SeriesPartitionSlot seriesPartitionSlot :
+ for (TSeriesPartitionSlot seriesPartitionSlot :
partitionSlotsMap.get(storageGroupName).keySet()) {
// Compare SeriesPartitionSlot
if (seriesTimePartitionSlotMap.containsKey(seriesPartitionSlot)) {
- Map<TimePartitionSlot, List<RegionReplicaSet>> timePartitionSlotMap =
+ Map<TTimePartitionSlot, List<TRegionReplicaSet>> timePartitionSlotMap =
seriesTimePartitionSlotMap.get(seriesPartitionSlot);
- for (TimePartitionSlot timePartitionSlot :
+ for (TTimePartitionSlot timePartitionSlot :
partitionSlotsMap.get(storageGroupName).get(seriesPartitionSlot)) {
// Compare TimePartitionSlot
if (!timePartitionSlotMap.containsKey(timePartitionSlot)) {
@@ -220,9 +223,9 @@ public class DataPartition extends Partition {
/** Create a DataPartition by ConfigNode */
public void createDataPartition(
String storageGroup,
- SeriesPartitionSlot seriesPartitionSlot,
- TimePartitionSlot timePartitionSlot,
- RegionReplicaSet regionReplicaSet) {
+ TSeriesPartitionSlot seriesPartitionSlot,
+ TTimePartitionSlot timePartitionSlot,
+ TRegionReplicaSet regionReplicaSet) {
dataPartitionMap
.computeIfAbsent(storageGroup, key -> new HashMap<>())
.computeIfAbsent(seriesPartitionSlot, key -> new HashMap<>())
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionQueryParam.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionQueryParam.java
index 23b61902a9..25ddb433e5 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionQueryParam.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionQueryParam.java
@@ -18,13 +18,15 @@
*/
package org.apache.iotdb.commons.partition;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+
import java.util.ArrayList;
import java.util.List;
public class DataPartitionQueryParam {
private String devicePath;
- private List<TimePartitionSlot> timePartitionSlotList = new ArrayList<>();
+ private List<TTimePartitionSlot> timePartitionSlotList = new ArrayList<>();
public String getDevicePath() {
return devicePath;
@@ -34,11 +36,11 @@ public class DataPartitionQueryParam {
this.devicePath = devicePath;
}
- public List<TimePartitionSlot> getTimePartitionSlotList() {
+ public List<TTimePartitionSlot> getTimePartitionSlotList() {
return timePartitionSlotList;
}
- public void setTimePartitionSlotList(List<TimePartitionSlot> timePartitionSlotList) {
+ public void setTimePartitionSlotList(List<TTimePartitionSlot> timePartitionSlotList) {
this.timePartitionSlotList = timePartitionSlotList;
}
}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/Partition.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/Partition.java
index e0c6e53bd1..839c330f90 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/Partition.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/Partition.java
@@ -18,6 +18,7 @@
*/
package org.apache.iotdb.commons.partition;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
public abstract class Partition {
@@ -34,7 +35,7 @@ public abstract class Partition {
seriesSlotExecutorName, seriesPartitionSlotNum);
}
- protected SeriesPartitionSlot calculateDeviceGroupId(String deviceName) {
+ protected TSeriesPartitionSlot calculateDeviceGroupId(String deviceName) {
return executor.getSeriesPartitionSlot(deviceName);
}
}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/RegionReplicaSet.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/RegionReplicaSet.java
deleted file mode 100644
index 27cf396fce..0000000000
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/RegionReplicaSet.java
+++ /dev/null
@@ -1,141 +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.iotdb.commons.partition;
-
-import org.apache.iotdb.common.rpc.thrift.TEndPoint;
-import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.consensus.ConsensusGroupId;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-import java.util.stream.Collectors;
-
-public class RegionReplicaSet {
- private ConsensusGroupId consensusGroupId;
- private List<DataNodeLocation> dataNodeList;
-
- public RegionReplicaSet() {}
-
- public RegionReplicaSet(ConsensusGroupId consensusGroupId, List<DataNodeLocation> dataNodeList) {
- this.consensusGroupId = consensusGroupId;
- this.dataNodeList = dataNodeList;
- }
-
- public RegionReplicaSet(TRegionReplicaSet respRegionReplicaSet) {
- this.consensusGroupId = ConsensusGroupId.Factory.create(respRegionReplicaSet.regionId);
- this.dataNodeList =
- respRegionReplicaSet.getEndpoint().stream()
- .map(
- respEndpoint ->
- new DataNodeLocation(
- new Endpoint(respEndpoint.getIp(), respEndpoint.getPort())))
- .collect(Collectors.toList());
- }
-
- public List<DataNodeLocation> getDataNodeList() {
- return dataNodeList;
- }
-
- public void setDataNodeList(List<DataNodeLocation> dataNodeList) {
- this.dataNodeList = dataNodeList;
- }
-
- public ConsensusGroupId getConsensusGroupId() {
- return consensusGroupId;
- }
-
- public void setConsensusGroupId(ConsensusGroupId consensusGroupId) {
- this.consensusGroupId = consensusGroupId;
- }
-
- /** Convert RegionReplicaSet to TRegionReplicaSet */
- public TRegionReplicaSet convertToRPCTRegionReplicaSet() {
- TRegionReplicaSet tRegionReplicaSet = new TRegionReplicaSet();
-
- // Convert ConsensusGroupId
- ByteBuffer buffer = ByteBuffer.allocate(Byte.BYTES + Integer.BYTES);
- consensusGroupId.serializeImpl(buffer);
- buffer.flip();
- tRegionReplicaSet.setRegionId(buffer);
-
- // Convert EndPoints
- List<TEndPoint> endPointList = new ArrayList<>();
- dataNodeList.forEach(
- dataNodeLocation ->
- endPointList.add(
- new TEndPoint(
- dataNodeLocation.getEndPoint().getIp(),
- dataNodeLocation.getEndPoint().getPort())));
- tRegionReplicaSet.setEndpoint(endPointList);
-
- return tRegionReplicaSet;
- }
-
- @Override
- public String toString() {
- return String.format("RegionReplicaSet[%s]: %s", consensusGroupId, dataNodeList);
- }
-
- public void serializeImpl(ByteBuffer buffer) {
- consensusGroupId.serializeImpl(buffer);
- buffer.putInt(dataNodeList.size());
- dataNodeList.forEach(
- dataNode -> {
- dataNode.serializeImpl(buffer);
- });
- }
-
- public static RegionReplicaSet deserializeImpl(ByteBuffer buffer) {
- ConsensusGroupId consensusGroupId = ConsensusGroupId.Factory.create(buffer);
-
- int size = buffer.getInt();
- // We should always make dataNodeList as a new Object when deserialization
- List<DataNodeLocation> dataNodeList = new ArrayList<>();
- for (int i = 0; i < size; i++) {
- dataNodeList.add(DataNodeLocation.deserializeImpl(buffer));
- }
- return new RegionReplicaSet(consensusGroupId, dataNodeList);
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- RegionReplicaSet that = (RegionReplicaSet) o;
- return Objects.equals(consensusGroupId, that.consensusGroupId)
- && Objects.equals(dataNodeList, that.dataNodeList);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(consensusGroupId, dataNodeList);
- }
-
- public boolean isEmpty() {
- return dataNodeList == null || dataNodeList.isEmpty();
- }
-}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/SchemaPartition.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/SchemaPartition.java
index b3f8c61986..a93048ff83 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/SchemaPartition.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/SchemaPartition.java
@@ -18,6 +18,9 @@
*/
package org.apache.iotdb.commons.partition;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
@@ -25,36 +28,36 @@ import java.util.Map;
public class SchemaPartition extends Partition {
- // Map<StorageGroup, Map<SeriesPartitionSlot, SchemaRegionPlaceInfo>>
- private Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> schemaPartitionMap;
+ // Map<StorageGroup, Map<TSeriesPartitionSlot, TSchemaRegionPlaceInfo>>
+ private Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap;
public SchemaPartition(String seriesSlotExecutorName, int seriesPartitionSlotNum) {
super(seriesSlotExecutorName, seriesPartitionSlotNum);
}
public SchemaPartition(
- Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> schemaPartitionMap,
+ Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap,
String seriesSlotExecutorName,
int seriesPartitionSlotNum) {
this(seriesSlotExecutorName, seriesPartitionSlotNum);
this.schemaPartitionMap = schemaPartitionMap;
}
- public Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> getSchemaPartitionMap() {
+ public Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> getSchemaPartitionMap() {
return schemaPartitionMap;
}
public void setSchemaPartitionMap(
- Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> schemaPartitionMap) {
+ Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap) {
this.schemaPartitionMap = schemaPartitionMap;
}
- public RegionReplicaSet getSchemaRegionReplicaSet(String deviceName) {
+ public TRegionReplicaSet getSchemaRegionReplicaSet(String deviceName) {
// A list of data region replica sets will store data in a same time partition.
// We will insert data to the last set in the list.
// TODO return the latest dataRegionReplicaSet for each time partition
String storageGroup = getStorageGroupByDevice(deviceName);
- SeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
+ TSeriesPartitionSlot seriesPartitionSlot = calculateDeviceGroupId(deviceName);
return schemaPartitionMap.get(storageGroup).get(seriesPartitionSlot);
}
@@ -78,13 +81,13 @@ public class SchemaPartition extends Partition {
* RegionReplicaSet>>
*/
public SchemaPartition getSchemaPartition(
- Map<String, List<SeriesPartitionSlot>> partitionSlotsMap) {
+ Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap) {
if (partitionSlotsMap.isEmpty()) {
// Return all SchemaPartitions when the partitionSlotsMap is empty
return new SchemaPartition(
new HashMap<>(schemaPartitionMap), seriesSlotExecutorName, seriesPartitionSlotNum);
} else {
- Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> result = new HashMap<>();
+ Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> result = new HashMap<>();
partitionSlotsMap.forEach(
(storageGroup, seriesPartitionSlots) -> {
@@ -119,9 +122,9 @@ public class SchemaPartition extends Partition {
* @param partitionSlotsMap Map<StorageGroupName, List<SeriesPartitionSlot>>
* @return Map<String, List<SeriesPartitionSlot>>, unassigned PartitionSlots
*/
- public Map<String, List<SeriesPartitionSlot>> filterNoAssignedSchemaPartitionSlot(
- Map<String, List<SeriesPartitionSlot>> partitionSlotsMap) {
- Map<String, List<SeriesPartitionSlot>> result = new HashMap<>();
+ public Map<String, List<TSeriesPartitionSlot>> filterNoAssignedSchemaPartitionSlot(
+ Map<String, List<TSeriesPartitionSlot>> partitionSlotsMap) {
+ Map<String, List<TSeriesPartitionSlot>> result = new HashMap<>();
partitionSlotsMap.forEach(
(storageGroup, seriesPartitionSlots) -> {
@@ -147,8 +150,8 @@ public class SchemaPartition extends Partition {
/** Create a SchemaPartition by ConfigNode */
public void createSchemaPartition(
String storageGroup,
- SeriesPartitionSlot seriesPartitionSlot,
- RegionReplicaSet regionReplicaSet) {
+ TSeriesPartitionSlot seriesPartitionSlot,
+ TRegionReplicaSet regionReplicaSet) {
schemaPartitionMap
.computeIfAbsent(storageGroup, key -> new HashMap<>())
.put(seriesPartitionSlot, regionReplicaSet);
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/SeriesPartitionSlot.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/SeriesPartitionSlot.java
deleted file mode 100644
index 553aeff163..0000000000
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/SeriesPartitionSlot.java
+++ /dev/null
@@ -1,59 +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.iotdb.commons.partition;
-
-import java.nio.ByteBuffer;
-
-public class SeriesPartitionSlot {
- private int slotId;
-
- public SeriesPartitionSlot() {
- // Empty constructor
- }
-
- public SeriesPartitionSlot(int slotId) {
- this.slotId = slotId;
- }
-
- public int getSlotId() {
- return slotId;
- }
-
- public void setSlotId(int slotId) {
- this.slotId = slotId;
- }
-
- @Override
- public int hashCode() {
- return new Integer(slotId).hashCode();
- }
-
- @Override
- public boolean equals(Object obj) {
- return obj instanceof SeriesPartitionSlot && this.slotId == ((SeriesPartitionSlot) obj).slotId;
- }
-
- public void serializeImpl(ByteBuffer buffer) {
- buffer.putInt(slotId);
- }
-
- public void deserializeImpl(ByteBuffer buffer) {
- slotId = buffer.getInt();
- }
-}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/TimePartitionSlot.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/TimePartitionSlot.java
deleted file mode 100644
index 69a3416fcf..0000000000
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/TimePartitionSlot.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.commons.partition;
-
-import java.nio.ByteBuffer;
-import java.util.Objects;
-
-public class TimePartitionSlot {
-
- private long startTime;
-
- public TimePartitionSlot() {
- // empty constructor
- }
-
- public TimePartitionSlot(long startTime) {
- this.startTime = startTime;
- }
-
- public long getStartTime() {
- return startTime;
- }
-
- public void setStartTime(long startTime) {
- this.startTime = startTime;
- }
-
- public void serializeImpl(ByteBuffer buffer) {
- buffer.putLong(startTime);
- }
-
- public void deserializeImpl(ByteBuffer buffer) {
- startTime = buffer.getLong();
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- TimePartitionSlot that = (TimePartitionSlot) o;
- return startTime == that.startTime;
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(startTime);
- }
-}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/SeriesPartitionExecutor.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/SeriesPartitionExecutor.java
index 8631951342..7a9de34825 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/SeriesPartitionExecutor.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/SeriesPartitionExecutor.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.commons.partition.executor;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
@@ -32,7 +32,7 @@ public abstract class SeriesPartitionExecutor {
this.seriesPartitionSlotNum = seriesPartitionSlotNum;
}
- public abstract SeriesPartitionSlot getSeriesPartitionSlot(String device);
+ public abstract TSeriesPartitionSlot getSeriesPartitionSlot(String device);
public static SeriesPartitionExecutor getSeriesPartitionExecutor(
String executorName, int seriesPartitionSlotNum) {
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/APHashExecutor.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/APHashExecutor.java
index 8da263e4ae..cad082bfe2 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/APHashExecutor.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/APHashExecutor.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.commons.partition.executor.hash;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
public class APHashExecutor extends SeriesPartitionExecutor {
@@ -28,7 +28,7 @@ public class APHashExecutor extends SeriesPartitionExecutor {
}
@Override
- public SeriesPartitionSlot getSeriesPartitionSlot(String device) {
+ public TSeriesPartitionSlot getSeriesPartitionSlot(String device) {
int hash = 0;
for (int i = 0; i < device.length(); i++) {
@@ -40,6 +40,6 @@ public class APHashExecutor extends SeriesPartitionExecutor {
}
hash &= Integer.MAX_VALUE;
- return new SeriesPartitionSlot(hash % seriesPartitionSlotNum);
+ return new TSeriesPartitionSlot(hash % seriesPartitionSlotNum);
}
}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/BKDRHashExecutor.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/BKDRHashExecutor.java
index a238afb416..a1f1e05503 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/BKDRHashExecutor.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/BKDRHashExecutor.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.commons.partition.executor.hash;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
public class BKDRHashExecutor extends SeriesPartitionExecutor {
@@ -30,7 +30,7 @@ public class BKDRHashExecutor extends SeriesPartitionExecutor {
}
@Override
- public SeriesPartitionSlot getSeriesPartitionSlot(String device) {
+ public TSeriesPartitionSlot getSeriesPartitionSlot(String device) {
int hash = 0;
for (int i = 0; i < device.length(); i++) {
@@ -38,6 +38,6 @@ public class BKDRHashExecutor extends SeriesPartitionExecutor {
}
hash &= Integer.MAX_VALUE;
- return new SeriesPartitionSlot(hash % seriesPartitionSlotNum);
+ return new TSeriesPartitionSlot(hash % seriesPartitionSlotNum);
}
}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/JSHashExecutor.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/JSHashExecutor.java
index 13f9468154..19914eed2d 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/JSHashExecutor.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/JSHashExecutor.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.commons.partition.executor.hash;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
public class JSHashExecutor extends SeriesPartitionExecutor {
@@ -30,7 +30,7 @@ public class JSHashExecutor extends SeriesPartitionExecutor {
}
@Override
- public SeriesPartitionSlot getSeriesPartitionSlot(String device) {
+ public TSeriesPartitionSlot getSeriesPartitionSlot(String device) {
int hash = base;
for (int i = 0; i < device.length(); i++) {
@@ -38,6 +38,6 @@ public class JSHashExecutor extends SeriesPartitionExecutor {
}
hash &= Integer.MAX_VALUE;
- return new SeriesPartitionSlot(hash % seriesPartitionSlotNum);
+ return new TSeriesPartitionSlot(hash % seriesPartitionSlotNum);
}
}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/SDBMHashExecutor.java b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/SDBMHashExecutor.java
index 0246d4f8de..105c256e3d 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/SDBMHashExecutor.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/partition/executor/hash/SDBMHashExecutor.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.commons.partition.executor.hash;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
public class SDBMHashExecutor extends SeriesPartitionExecutor {
@@ -28,7 +28,7 @@ public class SDBMHashExecutor extends SeriesPartitionExecutor {
}
@Override
- public SeriesPartitionSlot getSeriesPartitionSlot(String device) {
+ public TSeriesPartitionSlot getSeriesPartitionSlot(String device) {
int hash = 0;
for (int i = 0; i < device.length(); i++) {
@@ -36,6 +36,6 @@ public class SDBMHashExecutor extends SeriesPartitionExecutor {
}
hash &= Integer.MAX_VALUE;
- return new SeriesPartitionSlot(hash % seriesPartitionSlotNum);
+ return new TSeriesPartitionSlot(hash % seriesPartitionSlotNum);
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/util/SerializeDeserializeUtil.java b/node-commons/src/main/java/org/apache/iotdb/commons/utils/BasicStructureSerDeUtil.java
similarity index 98%
rename from confignode/src/main/java/org/apache/iotdb/confignode/util/SerializeDeserializeUtil.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/utils/BasicStructureSerDeUtil.java
index 7461827520..215829d396 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/util/SerializeDeserializeUtil.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/utils/BasicStructureSerDeUtil.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.iotdb.confignode.util;
+package org.apache.iotdb.commons.utils;
import java.nio.ByteBuffer;
import java.util.ArrayList;
@@ -25,10 +25,10 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
-public class SerializeDeserializeUtil {
+public class BasicStructureSerDeUtil {
public static final int INT_LEN = 4;
- private SerializeDeserializeUtil() {}
+ private BasicStructureSerDeUtil() {}
/** read string from byteBuffer. */
public static String readString(ByteBuffer buffer) {
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/utils/CommonUtils.java b/node-commons/src/main/java/org/apache/iotdb/commons/utils/CommonUtils.java
index b16d7692c8..f153933f43 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/utils/CommonUtils.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/utils/CommonUtils.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.commons.utils;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.commons.exception.BadNodeUrlException;
import org.slf4j.Logger;
@@ -31,17 +31,17 @@ import java.util.List;
public class CommonUtils {
private static final Logger logger = LoggerFactory.getLogger(CommonUtils.class);
- public static Endpoint parseNodeUrl(String nodeUrl) throws BadNodeUrlException {
+ public static TEndPoint parseNodeUrl(String nodeUrl) throws BadNodeUrlException {
String[] split = nodeUrl.split(":");
if (split.length != 2) {
logger.warn("Bad node url: {}", nodeUrl);
throw new BadNodeUrlException(String.format("Bad node url: %s", nodeUrl));
}
String ip = split[0];
- Endpoint result;
+ TEndPoint result;
try {
int port = Integer.parseInt(split[1]);
- result = new Endpoint(ip, port);
+ result = new TEndPoint(ip, port);
} catch (NumberFormatException e) {
logger.warn("Bad node url: {}", nodeUrl);
throw new BadNodeUrlException(String.format("Bad node url: %s", nodeUrl));
@@ -49,8 +49,8 @@ public class CommonUtils {
return result;
}
- public static List<Endpoint> parseNodeUrls(List<String> nodeUrls) throws BadNodeUrlException {
- List<Endpoint> result = new ArrayList<>();
+ public static List<TEndPoint> parseNodeUrls(List<String> nodeUrls) throws BadNodeUrlException {
+ List<TEndPoint> result = new ArrayList<>();
for (String url : nodeUrls) {
result.add(parseNodeUrl(url));
}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtils.java b/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtils.java
new file mode 100644
index 0000000000..1b40fa84a6
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtils.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.iotdb.commons.utils;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+// TODO: Serialize and deserialize by thrift structure
+/** Utils for serialize and deserialize all the data struct defined by thrift-commons */
+public class ThriftCommonsSerDeUtils {
+
+ private ThriftCommonsSerDeUtils() {
+ // Empty constructor
+ }
+
+ public static void writeTEndPoint(TEndPoint endPoint, ByteBuffer buffer) {
+ BasicStructureSerDeUtil.write(endPoint.getIp(), buffer);
+ buffer.putInt(endPoint.getPort());
+ }
+
+ public static TEndPoint readTEndPoint(ByteBuffer buffer) {
+ TEndPoint endPoint = new TEndPoint();
+ endPoint.setIp(BasicStructureSerDeUtil.readString(buffer));
+ endPoint.setPort(buffer.getInt());
+ return endPoint;
+ }
+
+ public static void writeTDataNodeLocation(TDataNodeLocation dataNodeLocation, ByteBuffer buffer) {
+ buffer.putInt(dataNodeLocation.getDataNodeId());
+
+ buffer.put(dataNodeLocation.isSetExternalEndPoint() ? (byte) 1 : (byte) 0);
+ if (dataNodeLocation.isSetExternalEndPoint()) {
+ writeTEndPoint(dataNodeLocation.getExternalEndPoint(), buffer);
+ }
+
+ buffer.put(dataNodeLocation.isSetInternalEndPoint() ? (byte) 1 : (byte) 0);
+ if (dataNodeLocation.isSetInternalEndPoint()) {
+ writeTEndPoint(dataNodeLocation.getInternalEndPoint(), buffer);
+ }
+
+ buffer.put(dataNodeLocation.isSetDataBlockManagerEndPoint() ? (byte) 1 : (byte) 0);
+ if (dataNodeLocation.isSetDataBlockManagerEndPoint()) {
+ writeTEndPoint(dataNodeLocation.getDataBlockManagerEndPoint(), buffer);
+ }
+
+ buffer.put(dataNodeLocation.isSetConsensusEndPoint() ? (byte) 1 : (byte) 0);
+ if (dataNodeLocation.isSetConsensusEndPoint()) {
+ writeTEndPoint(dataNodeLocation.getConsensusEndPoint(), buffer);
+ }
+ }
+
+ public static TDataNodeLocation readTDataNodeLocation(ByteBuffer buffer) {
+ TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+ dataNodeLocation.setDataNodeId(buffer.getInt());
+
+ if (buffer.get() > 0) {
+ dataNodeLocation.setExternalEndPoint(readTEndPoint(buffer));
+ }
+
+ if (buffer.get() > 0) {
+ dataNodeLocation.setInternalEndPoint(readTEndPoint(buffer));
+ }
+
+ if (buffer.get() > 0) {
+ dataNodeLocation.setDataBlockManagerEndPoint(readTEndPoint(buffer));
+ }
+
+ if (buffer.get() > 0) {
+ dataNodeLocation.setConsensusEndPoint(readTEndPoint(buffer));
+ }
+ return dataNodeLocation;
+ }
+
+ public static void writeTSeriesPartitionSlot(
+ TSeriesPartitionSlot seriesPartitionSlot, ByteBuffer buffer) {
+ buffer.putInt(seriesPartitionSlot.getSlotId());
+ }
+
+ public static TSeriesPartitionSlot readTSeriesPartitionSlot(ByteBuffer buffer) {
+ return new TSeriesPartitionSlot(buffer.getInt());
+ }
+
+ public static void writeTTimePartitionSlot(
+ TTimePartitionSlot timePartitionSlot, ByteBuffer buffer) {
+ buffer.putLong(timePartitionSlot.getStartTime());
+ }
+
+ public static TTimePartitionSlot readTTimePartitionSlot(ByteBuffer buffer) {
+ return new TTimePartitionSlot(buffer.getLong());
+ }
+
+ public static void writeTConsensusGroupId(TConsensusGroupId consensusGroupId, ByteBuffer buffer) {
+ buffer.putInt(consensusGroupId.getType().ordinal());
+ buffer.putInt(consensusGroupId.getId());
+ }
+
+ public static TConsensusGroupId readTConsensusGroupId(ByteBuffer buffer) {
+ TConsensusGroupType type = TConsensusGroupType.values()[buffer.getInt()];
+ return new TConsensusGroupId(type, buffer.getInt());
+ }
+
+ public static void writeTRegionReplicaSet(TRegionReplicaSet regionReplicaSet, ByteBuffer buffer) {
+ writeTConsensusGroupId(regionReplicaSet.getRegionId(), buffer);
+ buffer.putInt(regionReplicaSet.getDataNodeLocationsSize());
+ regionReplicaSet
+ .getDataNodeLocations()
+ .forEach(dataNodeLocation -> writeTDataNodeLocation(dataNodeLocation, buffer));
+ }
+
+ public static TRegionReplicaSet readTRegionReplicaSet(ByteBuffer buffer) {
+ TConsensusGroupId consensusGroupId = readTConsensusGroupId(buffer);
+ int dataNodeLocationNum = buffer.getInt();
+ List<TDataNodeLocation> dataNodeLocations = new ArrayList<>();
+ for (int i = 0; i < dataNodeLocationNum; i++) {
+ dataNodeLocations.add(readTDataNodeLocation(buffer));
+ }
+ return new TRegionReplicaSet(consensusGroupId, dataNodeLocations);
+ }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtils.java b/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtils.java
new file mode 100644
index 0000000000..85122f55ad
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtils.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.iotdb.commons.utils;
+
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+
+// TODO: Serialize and deserialize by thrift structure
+/** Utils for serialize and deserialize all the data struct defined by thrift-confignode */
+public class ThriftConfigNodeSerDeUtils {
+
+ private ThriftConfigNodeSerDeUtils() {
+ // Empty constructor
+ }
+
+ public static void writeTStorageGroupSchema(
+ TStorageGroupSchema storageGroupSchema, ByteBuffer buffer) {
+ BasicStructureSerDeUtil.write(storageGroupSchema.getName(), buffer);
+ buffer.putLong(storageGroupSchema.getTTL());
+ buffer.putInt(storageGroupSchema.getSchemaReplicationFactor());
+ buffer.putInt(storageGroupSchema.getDataReplicationFactor());
+ buffer.putLong(storageGroupSchema.getTimePartitionInterval());
+
+ buffer.putInt(storageGroupSchema.getSchemaRegionGroupIdsSize());
+ if (storageGroupSchema.getSchemaRegionGroupIdsSize() > 0) {
+ storageGroupSchema
+ .getSchemaRegionGroupIds()
+ .forEach(
+ schemaRegionGroupId ->
+ ThriftCommonsSerDeUtils.writeTConsensusGroupId(schemaRegionGroupId, buffer));
+ }
+
+ buffer.putInt(storageGroupSchema.getDataRegionGroupIdsSize());
+ if (storageGroupSchema.getDataRegionGroupIdsSize() > 0) {
+ storageGroupSchema
+ .getDataRegionGroupIds()
+ .forEach(
+ dataRegionGroupId ->
+ ThriftCommonsSerDeUtils.writeTConsensusGroupId(dataRegionGroupId, buffer));
+ }
+ }
+
+ public static TStorageGroupSchema readTStorageGroupSchema(ByteBuffer buffer) {
+ TStorageGroupSchema storageGroupSchema = new TStorageGroupSchema();
+ storageGroupSchema.setName(BasicStructureSerDeUtil.readString(buffer));
+ storageGroupSchema.setTTL(buffer.getLong());
+ storageGroupSchema.setSchemaReplicationFactor(buffer.getInt());
+ storageGroupSchema.setDataReplicationFactor(buffer.getInt());
+ storageGroupSchema.setTimePartitionInterval(buffer.getLong());
+
+ int groupIdNum = buffer.getInt();
+ storageGroupSchema.setSchemaRegionGroupIds(new ArrayList<>());
+ for (int i = 0; i < groupIdNum; i++) {
+ storageGroupSchema
+ .getSchemaRegionGroupIds()
+ .add(ThriftCommonsSerDeUtils.readTConsensusGroupId(buffer));
+ }
+
+ groupIdNum = buffer.getInt();
+ storageGroupSchema.setDataRegionGroupIds(new ArrayList<>());
+ for (int i = 0; i < groupIdNum; i++) {
+ storageGroupSchema
+ .getDataRegionGroupIds()
+ .add(ThriftCommonsSerDeUtils.readTConsensusGroupId(buffer));
+ }
+
+ return storageGroupSchema;
+ }
+}
diff --git a/node-commons/src/test/java/org/apache/iotdb/commons/ConsensusGroupIdTest.java b/node-commons/src/test/java/org/apache/iotdb/commons/ConsensusGroupIdTest.java
index e890449d2d..af19aea35e 100644
--- a/node-commons/src/test/java/org/apache/iotdb/commons/ConsensusGroupIdTest.java
+++ b/node-commons/src/test/java/org/apache/iotdb/commons/ConsensusGroupIdTest.java
@@ -19,35 +19,32 @@
package org.apache.iotdb.commons;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
import org.apache.iotdb.commons.consensus.ConsensusGroupId;
import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.consensus.GroupType;
import org.apache.iotdb.commons.consensus.SchemaRegionId;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
-import java.nio.ByteBuffer;
public class ConsensusGroupIdTest {
@Test
public void TestCreate() throws IOException {
- ConsensusGroupId dataRegionId = ConsensusGroupId.Factory.create(1, GroupType.DataRegion);
+ ConsensusGroupId dataRegionId =
+ ConsensusGroupId.Factory.convertFromTConsensusGroupId(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1));
Assert.assertTrue(dataRegionId instanceof DataRegionId);
Assert.assertEquals(1, dataRegionId.getId());
- Assert.assertEquals(GroupType.DataRegion, dataRegionId.getType());
+ Assert.assertEquals(TConsensusGroupType.DataRegion, dataRegionId.getType());
- ConsensusGroupId schemaRegionId = ConsensusGroupId.Factory.create(2, GroupType.SchemaRegion);
+ ConsensusGroupId schemaRegionId =
+ ConsensusGroupId.Factory.convertFromTConsensusGroupId(
+ new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 2));
Assert.assertTrue(schemaRegionId instanceof SchemaRegionId);
Assert.assertEquals(2, schemaRegionId.getId());
- Assert.assertEquals(GroupType.SchemaRegion, schemaRegionId.getType());
-
- ByteBuffer buffer = ByteBuffer.allocate(1024);
- schemaRegionId.serializeImpl(buffer);
- buffer.flip();
-
- ConsensusGroupId schemaRegionIdClone = ConsensusGroupId.Factory.create(buffer);
- Assert.assertEquals(schemaRegionId, schemaRegionIdClone);
+ Assert.assertEquals(TConsensusGroupType.SchemaRegion, schemaRegionId.getType());
}
}
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/utils/SerializeDeserializeUtilTest.java b/node-commons/src/test/java/org/apache/iotdb/commons/utils/BasicStructureSerDeUtilTest.java
similarity index 82%
rename from confignode/src/test/java/org/apache/iotdb/confignode/utils/SerializeDeserializeUtilTest.java
rename to node-commons/src/test/java/org/apache/iotdb/commons/utils/BasicStructureSerDeUtilTest.java
index 26b2d75aec..9e6a5f78fb 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/utils/SerializeDeserializeUtilTest.java
+++ b/node-commons/src/test/java/org/apache/iotdb/commons/utils/BasicStructureSerDeUtilTest.java
@@ -16,9 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.confignode.utils;
-
-import org.apache.iotdb.confignode.util.SerializeDeserializeUtil;
+package org.apache.iotdb.commons.utils;
import org.junit.Assert;
import org.junit.Test;
@@ -30,7 +28,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
-public class SerializeDeserializeUtilTest {
+public class BasicStructureSerDeUtilTest {
protected static final int DEFAULT_BUFFER_SIZE = 4096;
@Test
@@ -42,10 +40,10 @@ public class SerializeDeserializeUtilTest {
map.put(key, value);
ByteBuffer bf = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE);
- SerializeDeserializeUtil.write(map, bf);
+ BasicStructureSerDeUtil.write(map, bf);
byte[] b = bf.array();
bf.clear();
- Map<String, String> result = SerializeDeserializeUtil.readMap(ByteBuffer.wrap(b));
+ Map<String, String> result = BasicStructureSerDeUtil.readMap(ByteBuffer.wrap(b));
Assert.assertNotNull(result);
Assert.assertEquals(map, result);
}
@@ -61,11 +59,11 @@ public class SerializeDeserializeUtilTest {
stringMapLists.put(key, keyLists);
ByteBuffer bf = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE);
- SerializeDeserializeUtil.writeStringMapLists(stringMapLists, bf);
+ BasicStructureSerDeUtil.writeStringMapLists(stringMapLists, bf);
byte[] b = bf.array();
bf.clear();
Map<String, List<String>> stringMapListsResult =
- SerializeDeserializeUtil.readStringMapLists(ByteBuffer.wrap(b));
+ BasicStructureSerDeUtil.readStringMapLists(ByteBuffer.wrap(b));
Assert.assertNotNull(stringMapListsResult);
Assert.assertEquals(stringMapLists, stringMapListsResult);
}
@@ -81,9 +79,9 @@ public class SerializeDeserializeUtilTest {
ByteBuffer bf = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE);
byte[] b = bf.array();
- SerializeDeserializeUtil.writeIntMapLists(integerMapLists, bf);
+ BasicStructureSerDeUtil.writeIntMapLists(integerMapLists, bf);
Map<Integer, List<Integer>> intMapListsResult =
- SerializeDeserializeUtil.readIntMapLists(ByteBuffer.wrap(b));
+ BasicStructureSerDeUtil.readIntMapLists(ByteBuffer.wrap(b));
Assert.assertNotNull(intMapListsResult);
Assert.assertEquals(integerMapLists, intMapListsResult);
}
diff --git a/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtilsTest.java b/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtilsTest.java
new file mode 100644
index 0000000000..5ae5619dbb
--- /dev/null
+++ b/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtilsTest.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.iotdb.commons.utils;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+
+public class ThriftCommonsSerDeUtilsTest {
+
+ private static final ByteBuffer buffer = ByteBuffer.allocate(1024 * 10);
+
+ @After
+ public void cleanBuffer() {
+ buffer.clear();
+ }
+
+ @Test
+ public void readWriteTEndPointTest() {
+ TEndPoint endPoint0 = new TEndPoint("0.0.0.0", 6667);
+ ThriftCommonsSerDeUtils.writeTEndPoint(endPoint0, buffer);
+ buffer.flip();
+ TEndPoint endPoint1 = ThriftCommonsSerDeUtils.readTEndPoint(buffer);
+ Assert.assertEquals(endPoint0, endPoint1);
+ }
+
+ @Test
+ public void readWriteTDataNodeLocationTest() {
+ TDataNodeLocation dataNodeLocation0 = new TDataNodeLocation();
+ dataNodeLocation0.setDataNodeId(0);
+ dataNodeLocation0.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667));
+ dataNodeLocation0.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+ dataNodeLocation0.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777));
+ dataNodeLocation0.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010));
+ ThriftCommonsSerDeUtils.writeTDataNodeLocation(dataNodeLocation0, buffer);
+ buffer.flip();
+ TDataNodeLocation dataNodeLocation1 = ThriftCommonsSerDeUtils.readTDataNodeLocation(buffer);
+ Assert.assertEquals(dataNodeLocation0, dataNodeLocation1);
+ }
+
+ @Test
+ public void readWriteTSeriesPartitionSlotTest() {
+ TSeriesPartitionSlot seriesPartitionSlot0 = new TSeriesPartitionSlot(10);
+ ThriftCommonsSerDeUtils.writeTSeriesPartitionSlot(seriesPartitionSlot0, buffer);
+ buffer.flip();
+ TSeriesPartitionSlot seriesPartitionSlot1 =
+ ThriftCommonsSerDeUtils.readTSeriesPartitionSlot(buffer);
+ Assert.assertEquals(seriesPartitionSlot0, seriesPartitionSlot1);
+ }
+
+ @Test
+ public void writeTTimePartitionSlot() {
+ TTimePartitionSlot timePartitionSlot0 = new TTimePartitionSlot(100);
+ ThriftCommonsSerDeUtils.writeTTimePartitionSlot(timePartitionSlot0, buffer);
+ buffer.flip();
+ TTimePartitionSlot timePartitionSlot1 = ThriftCommonsSerDeUtils.readTTimePartitionSlot(buffer);
+ Assert.assertEquals(timePartitionSlot0, timePartitionSlot1);
+ }
+
+ @Test
+ public void readWriteTConsensusGroupIdTest() {
+ TConsensusGroupId consensusGroupId0 =
+ new TConsensusGroupId(TConsensusGroupType.PartitionRegion, 0);
+ ThriftCommonsSerDeUtils.writeTConsensusGroupId(consensusGroupId0, buffer);
+ buffer.flip();
+ TConsensusGroupId consensusGroupId1 = ThriftCommonsSerDeUtils.readTConsensusGroupId(buffer);
+ Assert.assertEquals(consensusGroupId0, consensusGroupId1);
+ }
+
+ @Test
+ public void readWriteTRegionReplicaSetTest() {
+ TRegionReplicaSet regionReplicaSet0 = new TRegionReplicaSet();
+ regionReplicaSet0.setRegionId(new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 0));
+ regionReplicaSet0.setDataNodeLocations(new ArrayList<>());
+ for (int i = 0; i < 3; i++) {
+ TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+ dataNodeLocation.setDataNodeId(i);
+ dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667 + i));
+ dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003 + i));
+ dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777 + i));
+ dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010 + i));
+ regionReplicaSet0.getDataNodeLocations().add(dataNodeLocation);
+ }
+ ThriftCommonsSerDeUtils.writeTRegionReplicaSet(regionReplicaSet0, buffer);
+ buffer.flip();
+ TRegionReplicaSet regionReplicaSet1 = ThriftCommonsSerDeUtils.readTRegionReplicaSet(buffer);
+ Assert.assertEquals(regionReplicaSet0, regionReplicaSet1);
+ }
+}
diff --git a/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtilsTest.java b/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtilsTest.java
new file mode 100644
index 0000000000..282ae59233
--- /dev/null
+++ b/node-commons/src/test/java/org/apache/iotdb/commons/utils/ThriftConfigNodeSerDeUtilsTest.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.commons.utils;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+
+public class ThriftConfigNodeSerDeUtilsTest {
+
+ private static final ByteBuffer buffer = ByteBuffer.allocate(1024 * 10);
+
+ @After
+ public void cleanBuffer() {
+ buffer.clear();
+ }
+
+ @Test
+ public void readWriteTStorageGroupSchemaTest() {
+ TStorageGroupSchema storageGroupSchema0 = new TStorageGroupSchema();
+ storageGroupSchema0.setName("root.sg");
+ storageGroupSchema0.setTTL(Long.MAX_VALUE);
+ storageGroupSchema0.setSchemaReplicationFactor(3);
+ storageGroupSchema0.setDataReplicationFactor(3);
+ storageGroupSchema0.setTimePartitionInterval(604800);
+
+ storageGroupSchema0.setSchemaRegionGroupIds(new ArrayList<>());
+ storageGroupSchema0.setDataRegionGroupIds(new ArrayList<>());
+ for (int i = 0; i < 3; i++) {
+ storageGroupSchema0
+ .getSchemaRegionGroupIds()
+ .add(new TConsensusGroupId(TConsensusGroupType.SchemaRegion, i * 2));
+ storageGroupSchema0
+ .getDataRegionGroupIds()
+ .add(new TConsensusGroupId(TConsensusGroupType.DataRegion, i * 2 + 1));
+ }
+
+ ThriftConfigNodeSerDeUtils.writeTStorageGroupSchema(storageGroupSchema0, buffer);
+ buffer.flip();
+ TStorageGroupSchema storageGroupSchema1 =
+ ThriftConfigNodeSerDeUtils.readTStorageGroupSchema(buffer);
+ Assert.assertEquals(storageGroupSchema0, storageGroupSchema1);
+ }
+}
diff --git a/pom.xml b/pom.xml
index 90a8dbccb4..06d6555fde 100644
--- a/pom.xml
+++ b/pom.xml
@@ -82,6 +82,7 @@
<module>tsfile</module>
<module>antlr</module>
<module>thrift</module>
+ <module>thrift-commons</module>
<module>thrift-confignode</module>
<module>thrift-cluster</module>
<module>thrift-sync</module>
@@ -742,6 +743,7 @@
<!-- put all source folders not in src/main/java here-->
<sourceDirectory>antlr/target/generated-sources/antlr4</sourceDirectory>
<sourceDirectory>thrift/target/generated-sources/thrift</sourceDirectory>
+ <sourceDirectory>thrift-commons/target/generated-sources/thrift</sourceDirectory>
<sourceDirectory>thrift-confignode/target/generated-sources/thrift</sourceDirectory>
<sourceDirectory>thrift-sync/target/generated-sources/thrift</sourceDirectory>
<sourceDirectory>thrift-cluster/target/generated-sources/thrift</sourceDirectory>
diff --git a/server/pom.xml b/server/pom.xml
index e6c9c29adf..cf18fa5054 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -50,11 +50,6 @@
<artifactId>iotdb-consensus</artifactId>
<version>${project.version}</version>
</dependency>
- <dependency>
- <groupId>org.apache.iotdb</groupId>
- <artifactId>node-commons</artifactId>
- <version>${project.version}</version>
- </dependency>
<dependency>
<groupId>org.apache.iotdb</groupId>
<artifactId>tsfile</artifactId>
@@ -82,11 +77,6 @@
<artifactId>influxdb-thrift</artifactId>
<version>${project.version}</version>
</dependency>
- <dependency>
- <groupId>org.apache.iotdb</groupId>
- <artifactId>iotdb-thrift-confignode</artifactId>
- <version>${project.version}</version>
- </dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-collections4</artifactId>
@@ -260,6 +250,11 @@
<artifactId>rocksdbjni</artifactId>
<version>6.27.3</version>
</dependency>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ <version>[${guava.version},)</version>
+ </dependency>
</dependencies>
<build>
<plugins>
diff --git a/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java b/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
index 4c2d0d7ed3..06bb8ff613 100644
--- a/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
+++ b/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
@@ -19,13 +19,13 @@
package org.apache.iotdb.db.client;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
import org.apache.iotdb.commons.exception.BadNodeUrlException;
import org.apache.iotdb.commons.utils.CommonUtils;
import org.apache.iotdb.confignode.rpc.thrift.ConfigIService;
import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerReq;
-import org.apache.iotdb.confignode.rpc.thrift.TDataNodeMessageResp;
+import org.apache.iotdb.confignode.rpc.thrift.TDataNodeLocationResp;
import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterReq;
import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRegisterResp;
import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq;
@@ -65,9 +65,9 @@ public class ConfigNodeClient {
private TTransport transport;
- private Endpoint configLeader;
+ private TEndPoint configLeader;
- private List<Endpoint> configNodes;
+ private List<TEndPoint> configNodes;
public ConfigNodeClient() throws BadNodeUrlException, IoTDBConnectionException {
// Read config nodes from configuration
@@ -76,12 +76,12 @@ public class ConfigNodeClient {
init();
}
- public ConfigNodeClient(List<Endpoint> configNodes) throws IoTDBConnectionException {
+ public ConfigNodeClient(List<TEndPoint> configNodes) throws IoTDBConnectionException {
this.configNodes = configNodes;
init();
}
- public ConfigNodeClient(List<Endpoint> configNodes, Endpoint configLeader)
+ public ConfigNodeClient(List<TEndPoint> configNodes, TEndPoint configLeader)
throws IoTDBConnectionException {
this.configNodes = configNodes;
this.configLeader = configLeader;
@@ -92,7 +92,7 @@ public class ConfigNodeClient {
reconnect();
}
- public void connect(Endpoint endpoint) throws IoTDBConnectionException {
+ public void connect(TEndPoint endpoint) throws IoTDBConnectionException {
try {
transport =
RpcTransportFactory.INSTANCE.getTransport(
@@ -131,7 +131,7 @@ public class ConfigNodeClient {
if (tryHostNum == configNodes.size()) {
break;
}
- Endpoint tryEndpoint = configNodes.get(j);
+ TEndPoint tryEndpoint = configNodes.get(j);
if (j == configNodes.size() - 1) {
j = -1;
}
@@ -154,7 +154,7 @@ public class ConfigNodeClient {
if (status.getCode() == TSStatusCode.NEED_REDIRECTION.getStatusCode()) {
if (status.isSetRedirectNode()) {
configLeader =
- new Endpoint(status.getRedirectNode().getIp(), status.getRedirectNode().getPort());
+ new TEndPoint(status.getRedirectNode().getIp(), status.getRedirectNode().getPort());
} else {
configLeader = null;
}
@@ -180,10 +180,11 @@ public class ConfigNodeClient {
throw new IoTDBConnectionException(MSG_RECONNECTION_FAIL);
}
- public TDataNodeMessageResp getDataNodesMessage(int dataNodeID) throws IoTDBConnectionException {
+ public TDataNodeLocationResp getDataNodeLocations(int dataNodeID)
+ throws IoTDBConnectionException {
for (int i = 0; i < RETRY_NUM; i++) {
try {
- TDataNodeMessageResp resp = client.getDataNodesMessage(dataNodeID);
+ TDataNodeLocationResp resp = client.getDataNodeLocations(dataNodeID);
if (!updateConfigNodeLeader(resp.status)) {
return resp;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/consensus/ConsensusImpl.java b/server/src/main/java/org/apache/iotdb/db/consensus/ConsensusImpl.java
index 972221609f..96fd26af89 100644
--- a/server/src/main/java/org/apache/iotdb/db/consensus/ConsensusImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/consensus/ConsensusImpl.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.consensus;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.commons.consensus.DataRegionId;
import org.apache.iotdb.commons.consensus.SchemaRegionId;
import org.apache.iotdb.consensus.ConsensusFactory;
@@ -51,7 +51,7 @@ public class ConsensusImpl {
private static final IConsensus INSTANCE =
ConsensusFactory.getConsensusImpl(
conf.getConsensusProtocolClass(),
- new Endpoint(conf.getInternalIp(), conf.getConsensusPort()),
+ new TEndPoint(conf.getInternalIp(), conf.getConsensusPort()),
new File(conf.getConsensusDir()),
gid -> {
switch (gid.getType()) {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
index 256278a020..5b89a48e71 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
@@ -19,10 +19,10 @@
package org.apache.iotdb.db.engine;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
import org.apache.iotdb.commons.concurrent.ThreadName;
import org.apache.iotdb.commons.exception.ShutdownException;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
import org.apache.iotdb.commons.service.IService;
import org.apache.iotdb.commons.service.ServiceType;
import org.apache.iotdb.commons.utils.TestOnly;
@@ -182,8 +182,8 @@ public class StorageEngine implements IService {
return enablePartition ? time / timePartitionInterval : 0;
}
- public static TimePartitionSlot getTimePartitionSlot(long time) {
- TimePartitionSlot timePartitionSlot = new TimePartitionSlot();
+ public static TTimePartitionSlot getTimePartitionSlot(long time) {
+ TTimePartitionSlot timePartitionSlot = new TTimePartitionSlot();
if (enablePartition) {
timePartitionSlot.setStartTime(time - time % timePartitionInterval);
} else {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockManager.java b/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockManager.java
index 573771fe81..6a8f34ccf5 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockManager.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.mpp.buffer;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.mpp.execution.FragmentInstanceContext;
import org.apache.iotdb.db.mpp.memory.LocalMemoryManager;
import org.apache.iotdb.mpp.rpc.thrift.DataBlockService;
@@ -258,7 +258,7 @@ public class DataBlockManager implements IDataBlockManager {
@Override
public ISinkHandle createSinkHandle(
TFragmentInstanceId localFragmentInstanceId,
- Endpoint endpoint,
+ TEndPoint endpoint,
TFragmentInstanceId remoteFragmentInstanceId,
String remotePlanNodeId,
FragmentInstanceContext instanceContext) {
@@ -291,7 +291,7 @@ public class DataBlockManager implements IDataBlockManager {
public ISourceHandle createSourceHandle(
TFragmentInstanceId localFragmentInstanceId,
String localPlanNodeId,
- Endpoint endpoint,
+ TEndPoint endpoint,
TFragmentInstanceId remoteFragmentInstanceId) {
if (sourceHandles.containsKey(localFragmentInstanceId)
&& sourceHandles.get(localFragmentInstanceId).containsKey(localPlanNodeId)) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockServiceClientFactory.java b/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockServiceClientFactory.java
index 397755cd4e..a363fed2bb 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockServiceClientFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/buffer/DataBlockServiceClientFactory.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.mpp.buffer;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.mpp.rpc.thrift.DataBlockService;
import org.apache.iotdb.rpc.RpcTransportFactory;
@@ -47,10 +47,10 @@ public class DataBlockServiceClientFactory {
private static final Logger logger = LoggerFactory.getLogger(DataBlockServiceClientFactory.class);
// TODO need to be replaced by mature client pool in the future
- private static final Map<Endpoint, DataBlockService.Iface> dataBlockServiceClientMap =
+ private static final Map<TEndPoint, DataBlockService.Iface> dataBlockServiceClientMap =
new ConcurrentHashMap<>();
- public DataBlockService.Iface getDataBlockServiceClient(Endpoint endpoint) {
+ public DataBlockService.Iface getDataBlockServiceClient(TEndPoint endpoint) {
return dataBlockServiceClientMap.computeIfAbsent(
endpoint,
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/buffer/IDataBlockManager.java b/server/src/main/java/org/apache/iotdb/db/mpp/buffer/IDataBlockManager.java
index 0eff0c6620..0d60d346db 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/buffer/IDataBlockManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/buffer/IDataBlockManager.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.mpp.buffer;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.mpp.execution.FragmentInstanceContext;
import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstanceId;
@@ -36,7 +36,7 @@ public interface IDataBlockManager {
*/
ISinkHandle createSinkHandle(
TFragmentInstanceId localFragmentInstanceId,
- Endpoint endpoint,
+ TEndPoint endpoint,
TFragmentInstanceId remoteFragmentInstanceId,
String remotePlanNodeId,
FragmentInstanceContext instanceContext);
@@ -55,7 +55,7 @@ public interface IDataBlockManager {
ISourceHandle createSourceHandle(
TFragmentInstanceId localFragmentInstanceId,
String localPlanNodeId,
- Endpoint endpoint,
+ TEndPoint endpoint,
TFragmentInstanceId remoteFragmentInstanceId);
/**
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/MPPQueryContext.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/MPPQueryContext.java
index b449bdf9e7..da8d42c6de 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/MPPQueryContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/MPPQueryContext.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.common;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.mpp.sql.analyze.QueryType;
/**
@@ -31,14 +31,14 @@ public class MPPQueryContext {
private SessionInfo session;
private QueryType queryType = QueryType.READ;
- private Endpoint hostEndpoint;
+ private TEndPoint hostEndpoint;
private ResultNodeContext resultNodeContext;
public MPPQueryContext(QueryId queryId) {
this.queryId = queryId;
}
- public MPPQueryContext(String sql, QueryId queryId, SessionInfo session, Endpoint hostEndpoint) {
+ public MPPQueryContext(String sql, QueryId queryId, SessionInfo session, TEndPoint hostEndpoint) {
this.sql = sql;
this.queryId = queryId;
this.session = session;
@@ -58,7 +58,7 @@ public class MPPQueryContext {
this.queryType = queryType;
}
- public Endpoint getHostEndpoint() {
+ public TEndPoint getHostEndpoint() {
return hostEndpoint;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/ResultNodeContext.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/ResultNodeContext.java
index 4c56646285..dcec963371 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/ResultNodeContext.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/ResultNodeContext.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.mpp.common;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
public class ResultNodeContext {
@@ -27,7 +27,7 @@ public class ResultNodeContext {
private final FragmentInstanceId virtualFragmentInstanceId;
private final PlanNodeId virtualResultNodeId;
- private Endpoint upStreamEndpoint;
+ private TEndPoint upStreamEndpoint;
private FragmentInstanceId upStreamFragmentInstanceId;
private PlanNodeId upStreamPlanNodeId;
@@ -37,7 +37,7 @@ public class ResultNodeContext {
}
public void setUpStream(
- Endpoint upStreamEndpoint,
+ TEndPoint upStreamEndpoint,
FragmentInstanceId upStreamFragmentInstanceId,
PlanNodeId upStreamPlanNodeId) {
this.upStreamEndpoint = upStreamEndpoint;
@@ -53,7 +53,7 @@ public class ResultNodeContext {
return virtualResultNodeId;
}
- public Endpoint getUpStreamEndpoint() {
+ public TEndPoint getUpStreamEndpoint() {
return upStreamEndpoint;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/Coordinator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/Coordinator.java
index a7d09b3ccf..ce589f8575 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/Coordinator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/Coordinator.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.execution;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.mpp.common.MPPQueryContext;
@@ -52,8 +52,8 @@ public class Coordinator {
private static final String COORDINATOR_SCHEDULED_EXECUTOR_NAME = "MPPCoordinatorScheduled";
private static final int COORDINATOR_SCHEDULED_EXECUTOR_SIZE = 1;
- private static final Endpoint LOCAL_HOST =
- new Endpoint(
+ private static final TEndPoint LOCAL_HOST =
+ new TEndPoint(
IoTDBDescriptor.getInstance().getConfig().getRpcAddress(),
IoTDBDescriptor.getInstance().getConfig().getInternalPort());
@@ -121,7 +121,7 @@ public class Coordinator {
}
// Get the hostname of current coordinator
- private Endpoint getHostEndpoint() {
+ private TEndPoint getHostEndpoint() {
return LOCAL_HOST;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/QueryExecution.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/QueryExecution.java
index a9bd605823..f362a48a72 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/QueryExecution.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/QueryExecution.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.execution;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.mpp.buffer.DataBlockService;
import org.apache.iotdb.db.mpp.buffer.ISourceHandle;
@@ -280,7 +280,7 @@ public class QueryExecution implements IQueryExecution {
.createSourceHandle(
context.getResultNodeContext().getVirtualFragmentInstanceId().toThrift(),
context.getResultNodeContext().getVirtualResultNodeId().getId(),
- new Endpoint(
+ new TEndPoint(
context.getResultNodeContext().getUpStreamEndpoint().getIp(),
IoTDBDescriptor.getInstance().getConfig().getDataBlockManagerPort()),
context.getResultNodeContext().getVirtualFragmentInstanceId().toThrift());
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/config/SetStorageGroupTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/config/SetStorageGroupTask.java
index 934d671e27..a28c7924c4 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/config/SetStorageGroupTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/config/SetStorageGroupTask.java
@@ -22,6 +22,7 @@ package org.apache.iotdb.db.mpp.execution.config;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
import org.apache.iotdb.commons.exception.BadNodeUrlException;
import org.apache.iotdb.confignode.rpc.thrift.TSetStorageGroupReq;
+import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
import org.apache.iotdb.db.client.ConfigNodeClient;
import org.apache.iotdb.db.mpp.sql.statement.metadata.SetStorageGroupStatement;
import org.apache.iotdb.rpc.IoTDBConnectionException;
@@ -45,8 +46,9 @@ public class SetStorageGroupTask implements IConfigTask {
public ListenableFuture<ConfigTaskResult> execute() {
SettableFuture<ConfigTaskResult> future = SettableFuture.create();
// Construct request using statement
- TSetStorageGroupReq req =
- new TSetStorageGroupReq(setStorageGroupStatement.getStorageGroupPath().getFullPath());
+ TStorageGroupSchema storageGroupSchema = new TStorageGroupSchema();
+ storageGroupSchema.setName(setStorageGroupStatement.getStorageGroupPath().getFullPath());
+ TSetStorageGroupReq req = new TSetStorageGroupReq(storageGroupSchema);
ConfigNodeClient configNodeClient = null;
try {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/AbstractFragInsStateTracker.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/AbstractFragInsStateTracker.java
index 82abc70c3f..71706b4857 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/AbstractFragInsStateTracker.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/AbstractFragInsStateTracker.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.mpp.execution.scheduler;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.mpp.execution.FragmentInstanceState;
import org.apache.iotdb.db.mpp.execution.QueryStateMachine;
@@ -61,7 +61,7 @@ public abstract class AbstractFragInsStateTracker implements IFragInstanceStateT
// TODO (jackie tien) change the port
InternalService.Iface client =
InternalServiceClientFactory.getInternalServiceClient(
- new Endpoint(
+ new TEndPoint(
instance.getHostEndpoint().getIp(),
IoTDBDescriptor.getInstance().getConfig().getInternalPort()));
TFragmentInstanceStateResp resp =
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/InternalServiceClientFactory.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/InternalServiceClientFactory.java
index c1c76f1e07..aa35c3854b 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/InternalServiceClientFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/InternalServiceClientFactory.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.mpp.execution.scheduler;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.mpp.rpc.thrift.InternalService;
import org.apache.iotdb.rpc.RpcTransportFactory;
@@ -44,10 +44,10 @@ public class InternalServiceClientFactory {
private static final Logger logger = LoggerFactory.getLogger(InternalServiceClientFactory.class);
// TODO need to be replaced by mature client pool in the future
- private static final Map<Endpoint, InternalService.Iface> internalServiceClientMap =
+ private static final Map<TEndPoint, InternalService.Iface> internalServiceClientMap =
new ConcurrentHashMap<>();
- public static InternalService.Iface getInternalServiceClient(Endpoint endpoint)
+ public static InternalService.Iface getInternalServiceClient(TEndPoint endpoint)
throws TTransportException {
return internalServiceClientMap.computeIfAbsent(
endpoint,
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleFragInstanceDispatcher.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleFragInstanceDispatcher.java
index 09ff1171b9..b2e9252f93 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleFragInstanceDispatcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleFragInstanceDispatcher.java
@@ -19,11 +19,11 @@
package org.apache.iotdb.db.mpp.execution.scheduler;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.mpp.sql.planner.plan.FragmentInstance;
import org.apache.iotdb.mpp.rpc.thrift.InternalService;
-import org.apache.iotdb.mpp.rpc.thrift.TConsensusGroupId;
import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstance;
import org.apache.iotdb.mpp.rpc.thrift.TSendFragmentInstanceReq;
import org.apache.iotdb.mpp.rpc.thrift.TSendFragmentInstanceResp;
@@ -50,19 +50,17 @@ public class SimpleFragInstanceDispatcher implements IFragInstanceDispatcher {
() -> {
TSendFragmentInstanceResp resp = new TSendFragmentInstanceResp(false);
for (FragmentInstance instance : instances) {
+ // TODO: (jackie tien) change the port
InternalService.Iface client =
InternalServiceClientFactory.getInternalServiceClient(
- new Endpoint(
+ new TEndPoint(
instance.getHostEndpoint().getIp(),
IoTDBDescriptor.getInstance().getConfig().getInternalPort()));
// TODO: (xingtanzjr) consider how to handle the buffer here
ByteBuffer buffer = ByteBuffer.allocate(1024 * 1024);
instance.serializeRequest(buffer);
buffer.flip();
- TConsensusGroupId groupId =
- new TConsensusGroupId(
- instance.getRegionReplicaSet().getConsensusGroupId().getId(),
- instance.getRegionReplicaSet().getConsensusGroupId().getType().toString());
+ TConsensusGroupId groupId = instance.getRegionReplicaSet().getRegionId();
TSendFragmentInstanceReq req =
new TSendFragmentInstanceReq(
new TFragmentInstance(buffer), groupId, instance.getType().toString());
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleQueryTerminator.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleQueryTerminator.java
index 5ffe02080a..856d8d6e6d 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleQueryTerminator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/scheduler/SimpleQueryTerminator.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.mpp.execution.scheduler;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.mpp.common.QueryId;
import org.apache.iotdb.db.mpp.sql.planner.plan.FragmentInstance;
@@ -50,16 +50,16 @@ public class SimpleQueryTerminator implements IQueryTerminator {
@Override
public Future<Boolean> terminate() {
- List<Endpoint> relatedHost = getRelatedHost(fragmentInstances);
+ List<TEndPoint> relatedHost = getRelatedHost(fragmentInstances);
return executor.submit(
() -> {
try {
- for (Endpoint endpoint : relatedHost) {
+ for (TEndPoint endpoint : relatedHost) {
// TODO (jackie tien) change the port
InternalService.Iface client =
InternalServiceClientFactory.getInternalServiceClient(
- new Endpoint(
+ new TEndPoint(
endpoint.getIp(),
IoTDBDescriptor.getInstance().getConfig().getInternalPort()));
client.cancelQuery(new TCancelQueryReq(queryId.getId()));
@@ -71,7 +71,7 @@ public class SimpleQueryTerminator implements IQueryTerminator {
});
}
- private List<Endpoint> getRelatedHost(List<FragmentInstance> instances) {
+ private List<TEndPoint> getRelatedHost(List<FragmentInstance> instances) {
return instances.stream()
.map(FragmentInstance::getHostEndpoint)
.distinct()
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/Analysis.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/Analysis.java
index a9e945ab4a..9b1aa3c651 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/Analysis.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/Analysis.java
@@ -19,8 +19,8 @@
package org.apache.iotdb.db.mpp.sql.analyze;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.commons.partition.DataPartition;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
import org.apache.iotdb.commons.partition.SchemaPartition;
import org.apache.iotdb.db.metadata.path.PartialPath;
import org.apache.iotdb.db.mpp.common.header.DatasetHeader;
@@ -58,7 +58,7 @@ public class Analysis {
public Analysis() {}
- public List<RegionReplicaSet> getPartitionInfo(PartialPath seriesPath, Filter timefilter) {
+ public List<TRegionReplicaSet> getPartitionInfo(PartialPath seriesPath, Filter timefilter) {
// TODO: (xingtanzjr) implement the calculation of timePartitionIdList
return dataPartition.getDataRegionReplicaSet(seriesPath.getDevice(), null);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterPartitionFetcher.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterPartitionFetcher.java
index f8a9d348be..b2ecaa612c 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterPartitionFetcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterPartitionFetcher.java
@@ -24,10 +24,7 @@ import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.commons.exception.BadNodeUrlException;
import org.apache.iotdb.commons.partition.DataPartition;
import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
import org.apache.iotdb.commons.partition.SchemaPartition;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq;
import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionResp;
@@ -195,17 +192,17 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
}
private SchemaPartition parseSchemaPartitionResp(TSchemaPartitionResp schemaPartitionResp) {
- Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> schemaPartitionMap = new HashMap<>();
+ Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap = new HashMap<>();
for (Map.Entry<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> sgEntry :
schemaPartitionResp.getSchemaRegionMap().entrySet()) {
// for each sg
String storageGroupName = sgEntry.getKey();
- Map<SeriesPartitionSlot, RegionReplicaSet> deviceToSchemaRegionMap = new HashMap<>();
+ Map<TSeriesPartitionSlot, TRegionReplicaSet> deviceToSchemaRegionMap = new HashMap<>();
for (Map.Entry<TSeriesPartitionSlot, TRegionReplicaSet> deviceEntry :
sgEntry.getValue().entrySet()) {
deviceToSchemaRegionMap.put(
- new SeriesPartitionSlot(deviceEntry.getKey().getSlotId()),
- new RegionReplicaSet(deviceEntry.getValue()));
+ new TSeriesPartitionSlot(deviceEntry.getKey().getSlotId()),
+ new TRegionReplicaSet(deviceEntry.getValue()));
}
schemaPartitionMap.put(storageGroupName, deviceToSchemaRegionMap);
}
@@ -216,7 +213,7 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
}
private DataPartition parseDataPartitionResp(TDataPartitionResp dataPartitionResp) {
- Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+ Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
dataPartitionMap = new HashMap<>();
for (Map.Entry<
String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
@@ -225,23 +222,24 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
String storageGroupName = sgEntry.getKey();
Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>
respDeviceToRegionsMap = sgEntry.getValue();
- Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>> deviceToRegionsMap =
- new HashMap<>();
+ Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>
+ deviceToRegionsMap = new HashMap<>();
for (Map.Entry<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>
deviceEntry : respDeviceToRegionsMap.entrySet()) {
// for each device
- Map<TimePartitionSlot, List<RegionReplicaSet>> timePartitionToRegionsMap = new HashMap<>();
+ Map<TTimePartitionSlot, List<TRegionReplicaSet>> timePartitionToRegionsMap =
+ new HashMap<>();
for (Map.Entry<TTimePartitionSlot, List<TRegionReplicaSet>> timePartitionEntry :
deviceEntry.getValue().entrySet()) {
// for each time partition
timePartitionToRegionsMap.put(
- new TimePartitionSlot(timePartitionEntry.getKey().getStartTime()),
+ new TTimePartitionSlot(timePartitionEntry.getKey().getStartTime()),
timePartitionEntry.getValue().stream()
- .map(RegionReplicaSet::new)
+ .map(TRegionReplicaSet::new)
.collect(Collectors.toList()));
}
deviceToRegionsMap.put(
- new SeriesPartitionSlot(deviceEntry.getKey().getSlotId()), timePartitionToRegionsMap);
+ new TSeriesPartitionSlot(deviceEntry.getKey().getSlotId()), timePartitionToRegionsMap);
}
dataPartitionMap.put(storageGroupName, deviceToRegionsMap);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterSchemaFetcher.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterSchemaFetcher.java
index 929ca01a55..969549998b 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterSchemaFetcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/ClusterSchemaFetcher.java
@@ -18,9 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.analyze;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
import org.apache.iotdb.commons.partition.SchemaPartition;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
import org.apache.iotdb.db.metadata.path.PartialPath;
import org.apache.iotdb.db.mpp.common.QueryId;
import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
@@ -62,7 +62,7 @@ public class ClusterSchemaFetcher implements ISchemaFetcher {
@Override
public SchemaTree fetchSchema(PathPatternTree patternTree) {
SchemaPartition schemaPartition = partitionFetcher.getSchemaPartition(patternTree);
- Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> schemaPartitionMap =
+ Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap =
schemaPartition.getSchemaPartitionMap();
List<String> storageGroups = new ArrayList<>(schemaPartitionMap.keySet());
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/FakePartitionFetcherImpl.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/FakePartitionFetcherImpl.java
index 3e72186310..eb0d8bec34 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/FakePartitionFetcherImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/analyze/FakePartitionFetcherImpl.java
@@ -19,14 +19,24 @@
package org.apache.iotdb.db.mpp.sql.analyze;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.*;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+import org.apache.iotdb.commons.partition.DataPartition;
+import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
+import org.apache.iotdb.commons.partition.SchemaPartition;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.mpp.common.schematree.PathPatternTree;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
public class FakePartitionFetcherImpl implements IPartitionFetcher {
@@ -51,56 +61,76 @@ public class FakePartitionFetcherImpl implements IPartitionFetcher {
new DataPartition(
IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionExecutorClass(),
IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionSlotNum());
- Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+ Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
dataPartitionMap = new HashMap<>();
- Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>> sgPartitionMap =
+ Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>> sgPartitionMap =
new HashMap<>();
- List<RegionReplicaSet> d1DataRegions = new ArrayList<>();
+ List<TRegionReplicaSet> d1DataRegions = new ArrayList<>();
d1DataRegions.add(
- new RegionReplicaSet(
- new DataRegionId(1),
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1),
Arrays.asList(
- new DataNodeLocation(11, new Endpoint("192.0.1.1", 9000)),
- new DataNodeLocation(12, new Endpoint("192.0.1.2", 9000)))));
+ new TDataNodeLocation()
+ .setDataNodeId(11)
+ .setExternalEndPoint(new TEndPoint("192.0.1.1", 9000)),
+ new TDataNodeLocation()
+ .setDataNodeId(12)
+ .setExternalEndPoint(new TEndPoint("192.0.1.2", 9000)))));
d1DataRegions.add(
- new RegionReplicaSet(
- new DataRegionId(2),
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 2),
Arrays.asList(
- new DataNodeLocation(21, new Endpoint("192.0.2.1", 9000)),
- new DataNodeLocation(22, new Endpoint("192.0.2.2", 9000)))));
- Map<TimePartitionSlot, List<RegionReplicaSet>> d1DataRegionMap = new HashMap<>();
- d1DataRegionMap.put(new TimePartitionSlot(), d1DataRegions);
-
- List<RegionReplicaSet> d2DataRegions = new ArrayList<>();
+ new TDataNodeLocation()
+ .setDataNodeId(21)
+ .setExternalEndPoint(new TEndPoint("192.0.2.1", 9000)),
+ new TDataNodeLocation()
+ .setDataNodeId(22)
+ .setExternalEndPoint(new TEndPoint("192.0.2.2", 9000)))));
+ Map<TTimePartitionSlot, List<TRegionReplicaSet>> d1DataRegionMap = new HashMap<>();
+ d1DataRegionMap.put(new TTimePartitionSlot(), d1DataRegions);
+
+ List<TRegionReplicaSet> d2DataRegions = new ArrayList<>();
d2DataRegions.add(
- new RegionReplicaSet(
- new DataRegionId(3),
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 3),
Arrays.asList(
- new DataNodeLocation(31, new Endpoint("192.0.3.1", 9000)),
- new DataNodeLocation(32, new Endpoint("192.0.3.2", 9000)))));
- Map<TimePartitionSlot, List<RegionReplicaSet>> d2DataRegionMap = new HashMap<>();
- d2DataRegionMap.put(new TimePartitionSlot(), d2DataRegions);
-
- List<RegionReplicaSet> d3DataRegions = new ArrayList<>();
+ new TDataNodeLocation()
+ .setDataNodeId(31)
+ .setExternalEndPoint(new TEndPoint("192.0.3.1", 9000)),
+ new TDataNodeLocation()
+ .setDataNodeId(32)
+ .setExternalEndPoint(new TEndPoint("192.0.3.2", 9000)))));
+ Map<TTimePartitionSlot, List<TRegionReplicaSet>> d2DataRegionMap = new HashMap<>();
+ d2DataRegionMap.put(new TTimePartitionSlot(), d2DataRegions);
+
+ List<TRegionReplicaSet> d3DataRegions = new ArrayList<>();
d3DataRegions.add(
- new RegionReplicaSet(
- new DataRegionId(1),
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1),
Arrays.asList(
- new DataNodeLocation(11, new Endpoint("192.0.1.1", 9000)),
- new DataNodeLocation(12, new Endpoint("192.0.1.2", 9000)))));
+ new TDataNodeLocation()
+ .setDataNodeId(11)
+ .setExternalEndPoint(new TEndPoint("192.0.1.1", 9000)),
+ new TDataNodeLocation()
+ .setDataNodeId(12)
+ .setExternalEndPoint(new TEndPoint("192.0.1.2", 9000)))));
d3DataRegions.add(
- new RegionReplicaSet(
- new DataRegionId(4),
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 4),
Arrays.asList(
- new DataNodeLocation(41, new Endpoint("192.0.4.1", 9000)),
- new DataNodeLocation(42, new Endpoint("192.0.4.2", 9000)))));
- Map<TimePartitionSlot, List<RegionReplicaSet>> d3DataRegionMap = new HashMap<>();
- d3DataRegionMap.put(new TimePartitionSlot(), d3DataRegions);
-
- sgPartitionMap.put(new SeriesPartitionSlot(device1.length()), d1DataRegionMap);
- sgPartitionMap.put(new SeriesPartitionSlot(device2.length()), d2DataRegionMap);
- sgPartitionMap.put(new SeriesPartitionSlot(device3.length()), d3DataRegionMap);
+ new TDataNodeLocation()
+ .setDataNodeId(41)
+ .setExternalEndPoint(new TEndPoint("192.0.4.1", 9000)),
+ new TDataNodeLocation()
+ .setDataNodeId(42)
+ .setExternalEndPoint(new TEndPoint("192.0.4.2", 9000)))));
+ Map<TTimePartitionSlot, List<TRegionReplicaSet>> d3DataRegionMap = new HashMap<>();
+ d3DataRegionMap.put(new TTimePartitionSlot(), d3DataRegions);
+
+ sgPartitionMap.put(new TSeriesPartitionSlot(device1.length()), d1DataRegionMap);
+ sgPartitionMap.put(new TSeriesPartitionSlot(device2.length()), d2DataRegionMap);
+ sgPartitionMap.put(new TSeriesPartitionSlot(device3.length()), d3DataRegionMap);
dataPartitionMap.put("root.sg", sgPartitionMap);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/DistributionPlanner.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/DistributionPlanner.java
index 83b5e950d6..b8a42f462b 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/DistributionPlanner.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/DistributionPlanner.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.sql.planner;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.mpp.common.MPPQueryContext;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
@@ -149,8 +149,8 @@ public class DistributionPlanner {
public PlanNode visitSchemaMerge(SchemaMergeNode node, DistributionPlanContext context) {
SchemaMergeNode root = (SchemaMergeNode) node.clone();
SchemaScanNode seed = (SchemaScanNode) node.getChildren().get(0);
- TreeSet<RegionReplicaSet> schemaRegions =
- new TreeSet<>(Comparator.comparingInt(region -> region.getConsensusGroupId().getId()));
+ TreeSet<TRegionReplicaSet> schemaRegions =
+ new TreeSet<>(Comparator.comparingInt(region -> region.getRegionId().getId()));
analysis
.getSchemaPartitionInfo()
.getSchemaPartitionMap()
@@ -177,7 +177,7 @@ public class DistributionPlanner {
// TODO: (xingtanzjr) a temporary way to resolve the distribution of single SeriesScanNode issue
@Override
public PlanNode visitSeriesScan(SeriesScanNode node, DistributionPlanContext context) {
- List<RegionReplicaSet> dataDistribution =
+ List<TRegionReplicaSet> dataDistribution =
analysis.getPartitionInfo(node.getSeriesPath(), node.getTimeFilter());
if (dataDistribution.size() == 1) {
node.setRegionReplicaSet(dataDistribution.get(0));
@@ -185,7 +185,7 @@ public class DistributionPlanner {
}
TimeJoinNode timeJoinNode =
new TimeJoinNode(context.queryContext.getQueryId().genPlanNodeId(), node.getScanOrder());
- for (RegionReplicaSet dataRegion : dataDistribution) {
+ for (TRegionReplicaSet dataRegion : dataDistribution) {
SeriesScanNode split = (SeriesScanNode) node.clone();
split.setPlanNodeId(context.queryContext.getQueryId().genPlanNodeId());
split.setRegionReplicaSet(dataRegion);
@@ -206,11 +206,11 @@ public class DistributionPlanner {
// If the child is SeriesScanNode, we need to check whether this node should be seperated
// into several splits.
SeriesScanNode handle = (SeriesScanNode) child;
- List<RegionReplicaSet> dataDistribution =
+ List<TRegionReplicaSet> dataDistribution =
analysis.getPartitionInfo(handle.getSeriesPath(), handle.getTimeFilter());
// If the size of dataDistribution is m, this SeriesScanNode should be seperated into m
// SeriesScanNode.
- for (RegionReplicaSet dataRegion : dataDistribution) {
+ for (TRegionReplicaSet dataRegion : dataDistribution) {
SeriesScanNode split = (SeriesScanNode) handle.clone();
split.setPlanNodeId(context.queryContext.getQueryId().genPlanNodeId());
split.setRegionReplicaSet(dataRegion);
@@ -230,7 +230,7 @@ public class DistributionPlanner {
}
// Step 2: For the source nodes, group them by the DataRegion.
- Map<RegionReplicaSet, List<SeriesScanNode>> sourceGroup =
+ Map<TRegionReplicaSet, List<SeriesScanNode>> sourceGroup =
sources.stream().collect(Collectors.groupingBy(SeriesScanNode::getRegionReplicaSet));
// Step 3: For the source nodes which belong to same data region, add a TimeJoinNode for them
// and make the
@@ -365,7 +365,7 @@ public class DistributionPlanner {
visitedChildren.add(visit(child, context));
});
- RegionReplicaSet dataRegion = calculateDataRegionByChildren(visitedChildren, context);
+ TRegionReplicaSet dataRegion = calculateDataRegionByChildren(visitedChildren, context);
NodeDistributionType distributionType =
nodeDistributionIsSame(visitedChildren, context)
? NodeDistributionType.SAME_WITH_ALL_CHILDREN
@@ -395,10 +395,10 @@ public class DistributionPlanner {
return newNode;
}
- private RegionReplicaSet calculateDataRegionByChildren(
+ private TRegionReplicaSet calculateDataRegionByChildren(
List<PlanNode> children, NodeGroupContext context) {
// Step 1: calculate the count of children group by DataRegion.
- Map<RegionReplicaSet, Long> groupByRegion =
+ Map<TRegionReplicaSet, Long> groupByRegion =
children.stream()
.collect(
Collectors.groupingBy(
@@ -408,7 +408,7 @@ public class DistributionPlanner {
return Collections.max(groupByRegion.entrySet(), Map.Entry.comparingByValue()).getKey();
}
- private RegionReplicaSet calculateSchemaRegionByChildren(
+ private TRegionReplicaSet calculateSchemaRegionByChildren(
List<PlanNode> children, NodeGroupContext context) {
// We always make the schemaRegion of MetaMergeNode to be the same as its first child.
return context.getNodeDistribution(children.get(0).getPlanNodeId()).region;
@@ -458,9 +458,9 @@ public class DistributionPlanner {
private class NodeDistribution {
private NodeDistributionType type;
- private RegionReplicaSet region;
+ private TRegionReplicaSet region;
- private NodeDistribution(NodeDistributionType type, RegionReplicaSet region) {
+ private NodeDistribution(NodeDistributionType type, TRegionReplicaSet region) {
this.type = type;
this.region = region;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/LocalExecutionPlanner.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/LocalExecutionPlanner.java
index e5d60439bd..51e7abc63d 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/LocalExecutionPlanner.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/LocalExecutionPlanner.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.sql.planner;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.storagegroup.DataRegion;
import org.apache.iotdb.db.metadata.path.PartialPath;
@@ -321,13 +321,13 @@ public class LocalExecutionPlanner {
SeriesScanOperator.class.getSimpleName());
FragmentInstanceId localInstanceId = context.instanceContext.getId();
FragmentInstanceId remoteInstanceId = node.getUpstreamInstanceId();
- Endpoint source = node.getUpstreamEndpoint();
+ TEndPoint source = node.getUpstreamEndpoint();
ISourceHandle sourceHandle =
DATA_BLOCK_MANAGER.createSourceHandle(
localInstanceId.toThrift(),
node.getPlanNodeId().getId(),
- new Endpoint(
+ new TEndPoint(
source.getIp(),
IoTDBDescriptor.getInstance().getConfig().getDataBlockManagerPort()),
remoteInstanceId.toThrift());
@@ -337,13 +337,13 @@ public class LocalExecutionPlanner {
@Override
public Operator visitFragmentSink(FragmentSinkNode node, LocalExecutionPlanContext context) {
Operator child = node.getChild().accept(this, context);
- Endpoint target = node.getDownStreamEndpoint();
+ TEndPoint target = node.getDownStreamEndpoint();
FragmentInstanceId localInstanceId = context.instanceContext.getId();
FragmentInstanceId targetInstanceId = node.getDownStreamInstanceId();
ISinkHandle sinkHandle =
DATA_BLOCK_MANAGER.createSinkHandle(
localInstanceId.toThrift(),
- new Endpoint(
+ new TEndPoint(
target.getIp(),
IoTDBDescriptor.getInstance().getConfig().getDataBlockManagerPort()),
targetInstanceId.toThrift(),
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/FragmentInstance.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/FragmentInstance.java
index 7058301ce8..b4a7819589 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/FragmentInstance.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/FragmentInstance.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.planner.plan;
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
import org.apache.iotdb.consensus.common.request.IConsensusRequest;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.sql.analyze.QueryType;
@@ -41,9 +42,9 @@ public class FragmentInstance implements IConsensusRequest {
private final PlanFragment fragment;
// The Region where the FragmentInstance should run
- private RegionReplicaSet regionReplicaSet;
+ private TRegionReplicaSet regionReplicaSet;
- private Endpoint hostEndpoint;
+ private TEndPoint hostEndpoint;
private Filter timeFilter;
@@ -58,26 +59,26 @@ public class FragmentInstance implements IConsensusRequest {
this.type = type;
}
- public RegionReplicaSet getDataRegionId() {
+ public TRegionReplicaSet getDataRegionId() {
return regionReplicaSet;
}
- public void setDataRegionAndHost(RegionReplicaSet regionReplicaSet) {
+ public void setDataRegionAndHost(TRegionReplicaSet regionReplicaSet) {
this.regionReplicaSet = regionReplicaSet;
// TODO: (xingtanzjr) We select the first Endpoint as the default target host for current
// instance
- this.hostEndpoint = regionReplicaSet.getDataNodeList().get(0).getEndPoint();
+ this.hostEndpoint = regionReplicaSet.getDataNodeLocations().get(0).getConsensusEndPoint();
}
- public RegionReplicaSet getRegionReplicaSet() {
+ public TRegionReplicaSet getRegionReplicaSet() {
return regionReplicaSet;
}
- public void setRegionReplicaSet(RegionReplicaSet regionReplicaSet) {
+ public void setRegionReplicaSet(TRegionReplicaSet regionReplicaSet) {
this.regionReplicaSet = regionReplicaSet;
}
- public Endpoint getHostEndpoint() {
+ public TEndPoint getHostEndpoint() {
return hostEndpoint;
}
@@ -122,9 +123,7 @@ public class FragmentInstance implements IConsensusRequest {
ret.append(
String.format(
"Region: %s",
- getRegionReplicaSet() == null
- ? "Not set"
- : getRegionReplicaSet().getConsensusGroupId()));
+ getRegionReplicaSet() == null ? "Not set" : getRegionReplicaSet().getRegionId()));
ret.append("---- Plan Node Tree ----\n");
ret.append(PlanNodeUtil.nodeToString(getFragment().getRoot()));
return ret.toString();
@@ -138,8 +137,8 @@ public class FragmentInstance implements IConsensusRequest {
QueryType queryType = QueryType.values()[ReadWriteIOUtils.readInt(buffer)];
FragmentInstance fragmentInstance =
new FragmentInstance(planFragment, id, timeFilter, queryType);
- fragmentInstance.regionReplicaSet = RegionReplicaSet.deserializeImpl(buffer);
- fragmentInstance.hostEndpoint = Endpoint.deserializeImpl(buffer);
+ fragmentInstance.regionReplicaSet = ThriftCommonsSerDeUtils.readTRegionReplicaSet(buffer);
+ fragmentInstance.hostEndpoint = ThriftCommonsSerDeUtils.readTEndPoint(buffer);
return fragmentInstance;
}
@@ -153,8 +152,8 @@ public class FragmentInstance implements IConsensusRequest {
timeFilter.serialize(buffer);
}
ReadWriteIOUtils.write(type.ordinal(), buffer);
- regionReplicaSet.serializeImpl(buffer);
- hostEndpoint.serializeImpl(buffer);
+ ThriftCommonsSerDeUtils.writeTRegionReplicaSet(regionReplicaSet, buffer);
+ ThriftCommonsSerDeUtils.writeTEndPoint(hostEndpoint, buffer);
}
@Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/PlanFragment.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/PlanFragment.java
index 38365ec4af..a287884a99 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/PlanFragment.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/PlanFragment.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.sql.planner.plan;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
@@ -61,16 +61,16 @@ public class PlanFragment {
// In current version, one PlanFragment should contain at least one SourceNode,
// and the DataRegions of all SourceNodes should be same in one PlanFragment.
// So we can use the DataRegion of one SourceNode as the PlanFragment's DataRegion.
- public RegionReplicaSet getTargetRegion() {
+ public TRegionReplicaSet getTargetRegion() {
return getNodeRegion(root);
}
- private RegionReplicaSet getNodeRegion(PlanNode root) {
+ private TRegionReplicaSet getNodeRegion(PlanNode root) {
if (root instanceof SourceNode) {
return ((SourceNode) root).getRegionReplicaSet();
}
for (PlanNode child : root.getChildren()) {
- RegionReplicaSet result = getNodeRegion(child);
+ TRegionReplicaSet result = getNodeRegion(child);
if (result != null) {
return result;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/SimpleFragmentParallelPlanner.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/SimpleFragmentParallelPlanner.java
index 631b217695..f3413df263 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/SimpleFragmentParallelPlanner.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/SimpleFragmentParallelPlanner.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.sql.planner.plan;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.mpp.common.MPPQueryContext;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
@@ -93,7 +93,7 @@ public class SimpleFragmentParallelPlanner implements IFragmentParallelPlaner {
// Get the target region for origin PlanFragment, then its instance will be distributed one
// of them.
- RegionReplicaSet regionReplicaSet = fragment.getTargetRegion();
+ TRegionReplicaSet regionReplicaSet = fragment.getTargetRegion();
// Set DataRegion and target host for the instance
// We need to store all the replica host in case of the scenario that the instance need to be
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/PlanGraphPrinter.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/PlanGraphPrinter.java
index d74a6c49f1..fbcef0ec72 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/PlanGraphPrinter.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/PlanGraphPrinter.java
@@ -60,7 +60,7 @@ public class PlanGraphPrinter extends PlanVisitor<List<String>, PlanGraphPrinter
List<String> boxValue = new ArrayList<>();
boxValue.add(String.format("SeriesScanNode-%s", node.getPlanNodeId().getId()));
boxValue.add(String.format("Series: %s", node.getSeriesPath()));
- boxValue.add(String.format("Partition: %s", node.getRegionReplicaSet().getConsensusGroupId()));
+ boxValue.add(String.format("Partition: %s", node.getRegionReplicaSet().getRegionId()));
return render(node, boxValue, context);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/WritePlanNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/WritePlanNode.java
index c37b8e1d7f..434b8bb9a5 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/WritePlanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/WritePlanNode.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.mpp.sql.planner.plan.node;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
import java.util.List;
@@ -30,7 +30,7 @@ public abstract class WritePlanNode extends PlanNode {
super(id);
}
- public abstract RegionReplicaSet getRegionReplicaSet();
+ public abstract TRegionReplicaSet getRegionReplicaSet();
public abstract List<WritePlanNode> splitByPartition(Analysis analysis);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/read/SchemaScanNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/read/SchemaScanNode.java
index 90812b842d..d1605ba8fd 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/read/SchemaScanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/read/SchemaScanNode.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.read;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.metadata.path.PartialPath;
import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
import org.apache.iotdb.db.mpp.sql.planner.plan.node.source.SourceNode;
@@ -32,7 +32,7 @@ public abstract class SchemaScanNode extends SourceNode {
private boolean hasLimit;
protected boolean isPrefixPath;
- private RegionReplicaSet schemaRegionReplicaSet;
+ private TRegionReplicaSet schemaRegionReplicaSet;
protected SchemaScanNode(PlanNodeId id) {
super(id);
@@ -81,12 +81,12 @@ public abstract class SchemaScanNode extends SourceNode {
}
@Override
- public RegionReplicaSet getRegionReplicaSet() {
+ public TRegionReplicaSet getRegionReplicaSet() {
return schemaRegionReplicaSet;
}
@Override
- public void setRegionReplicaSet(RegionReplicaSet schemaRegionReplicaSet) {
+ public void setRegionReplicaSet(TRegionReplicaSet schemaRegionReplicaSet) {
this.schemaRegionReplicaSet = schemaRegionReplicaSet;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/write/CreateTimeSeriesNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/write/CreateTimeSeriesNode.java
index e5d1959dc3..ddeefcef63 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/write/CreateTimeSeriesNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/metedata/write/CreateTimeSeriesNode.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.PartialPath;
import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
@@ -54,7 +54,7 @@ public class CreateTimeSeriesNode extends WritePlanNode {
private Map<String, String> attributes = null;
private long tagOffset = -1;
- private RegionReplicaSet regionReplicaSet;
+ private TRegionReplicaSet regionReplicaSet;
public CreateTimeSeriesNode(
PlanNodeId id,
@@ -314,19 +314,19 @@ public class CreateTimeSeriesNode extends WritePlanNode {
}
@Override
- public RegionReplicaSet getRegionReplicaSet() {
+ public TRegionReplicaSet getRegionReplicaSet() {
return regionReplicaSet;
}
@Override
public List<WritePlanNode> splitByPartition(Analysis analysis) {
- RegionReplicaSet regionReplicaSet =
+ TRegionReplicaSet regionReplicaSet =
analysis.getSchemaPartitionInfo().getSchemaRegionReplicaSet(path.getDevice());
setRegionReplicaSet(regionReplicaSet);
return ImmutableList.of(this);
}
- public void setRegionReplicaSet(RegionReplicaSet regionReplicaSet) {
+ public void setRegionReplicaSet(TRegionReplicaSet regionReplicaSet) {
this.regionReplicaSet = regionReplicaSet;
}
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/process/ExchangeNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/process/ExchangeNode.java
index d3f82fd328..dc61098d7a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/process/ExchangeNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/process/ExchangeNode.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.mpp.sql.planner.plan.node.process;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.sql.planner.plan.PlanFragment;
import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
@@ -44,7 +44,7 @@ public class ExchangeNode extends PlanNode {
// In current version, one ExchangeNode will only have one source.
// And the fragment which the sourceNode belongs to will only have one instance.
// Thus, by nodeId and endpoint, the ExchangeNode can know where its source from.
- private Endpoint upstreamEndpoint;
+ private TEndPoint upstreamEndpoint;
private FragmentInstanceId upstreamInstanceId;
private PlanNodeId upstreamPlanNodeId;
@@ -86,7 +86,7 @@ public class ExchangeNode extends PlanNode {
return CHILD_COUNT_NO_LIMIT;
}
- public void setUpstream(Endpoint endPoint, FragmentInstanceId instanceId, PlanNodeId nodeId) {
+ public void setUpstream(TEndPoint endPoint, FragmentInstanceId instanceId, PlanNodeId nodeId) {
this.upstreamEndpoint = endPoint;
this.upstreamInstanceId = instanceId;
this.upstreamPlanNodeId = nodeId;
@@ -95,8 +95,9 @@ public class ExchangeNode extends PlanNode {
public static ExchangeNode deserialize(ByteBuffer byteBuffer) {
FragmentSinkNode fragmentSinkNode =
(FragmentSinkNode) PlanFragment.deserializeHelper(byteBuffer);
- Endpoint endPoint =
- new Endpoint(ReadWriteIOUtils.readString(byteBuffer), ReadWriteIOUtils.readInt(byteBuffer));
+ TEndPoint endPoint =
+ new TEndPoint(
+ ReadWriteIOUtils.readString(byteBuffer), ReadWriteIOUtils.readInt(byteBuffer));
FragmentInstanceId fragmentInstanceId = FragmentInstanceId.deserialize(byteBuffer);
PlanNodeId upstreamPlanNodeId = PlanNodeId.deserialize(byteBuffer);
PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
@@ -151,7 +152,7 @@ public class ExchangeNode extends PlanNode {
this.child = null;
}
- public Endpoint getUpstreamEndpoint() {
+ public TEndPoint getUpstreamEndpoint() {
return upstreamEndpoint;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/sink/FragmentSinkNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/sink/FragmentSinkNode.java
index a738c8a502..c32b2b2062 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/sink/FragmentSinkNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/sink/FragmentSinkNode.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.sql.planner.plan.node.sink;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
@@ -36,7 +36,7 @@ import java.util.Objects;
public class FragmentSinkNode extends SinkNode {
private PlanNode child;
- private Endpoint downStreamEndpoint;
+ private TEndPoint downStreamEndpoint;
private FragmentInstanceId downStreamInstanceId;
private PlanNodeId downStreamPlanNodeId;
@@ -84,8 +84,9 @@ public class FragmentSinkNode extends SinkNode {
}
public static FragmentSinkNode deserialize(ByteBuffer byteBuffer) {
- Endpoint downStreamEndpoint =
- new Endpoint(ReadWriteIOUtils.readString(byteBuffer), ReadWriteIOUtils.readInt(byteBuffer));
+ TEndPoint downStreamEndpoint =
+ new TEndPoint(
+ ReadWriteIOUtils.readString(byteBuffer), ReadWriteIOUtils.readInt(byteBuffer));
FragmentInstanceId downStreamInstanceId = FragmentInstanceId.deserialize(byteBuffer);
PlanNodeId downStreamPlanNodeId = PlanNodeId.deserialize(byteBuffer);
PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
@@ -134,7 +135,7 @@ public class FragmentSinkNode extends SinkNode {
getDownStreamEndpoint().getIp(), getDownStreamInstanceId(), getDownStreamPlanNodeId());
}
- public void setDownStream(Endpoint endPoint, FragmentInstanceId instanceId, PlanNodeId nodeId) {
+ public void setDownStream(TEndPoint endPoint, FragmentInstanceId instanceId, PlanNodeId nodeId) {
this.downStreamEndpoint = endPoint;
this.downStreamInstanceId = instanceId;
this.downStreamPlanNodeId = nodeId;
@@ -144,7 +145,7 @@ public class FragmentSinkNode extends SinkNode {
this.downStreamPlanNodeId = downStreamPlanNodeId;
}
- public Endpoint getDownStreamEndpoint() {
+ public TEndPoint getDownStreamEndpoint() {
return downStreamEndpoint;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesAggregateScanNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesAggregateScanNode.java
index fde2b903a1..88bb0742bb 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesAggregateScanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesAggregateScanNode.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.planner.plan.node.source;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
import org.apache.iotdb.db.metadata.path.PartialPath;
import org.apache.iotdb.db.metadata.path.PathDeserializeUtil;
import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
@@ -90,7 +91,7 @@ public class SeriesAggregateScanNode extends SourceNode implements IOutputPlanNo
private List<ColumnHeader> columnHeaders;
// The id of DataRegion where the node will run
- private RegionReplicaSet regionReplicaSet;
+ private TRegionReplicaSet regionReplicaSet;
public SeriesAggregateScanNode(
PlanNodeId id,
@@ -169,12 +170,12 @@ public class SeriesAggregateScanNode extends SourceNode implements IOutputPlanNo
public void open() throws Exception {}
@Override
- public RegionReplicaSet getRegionReplicaSet() {
+ public TRegionReplicaSet getRegionReplicaSet() {
return this.regionReplicaSet;
}
@Override
- public void setRegionReplicaSet(RegionReplicaSet regionReplicaSet) {
+ public void setRegionReplicaSet(TRegionReplicaSet regionReplicaSet) {
this.regionReplicaSet = regionReplicaSet;
}
@@ -208,7 +209,7 @@ public class SeriesAggregateScanNode extends SourceNode implements IOutputPlanNo
ReadWriteIOUtils.write(scanOrder.ordinal(), byteBuffer);
timeFilter.serialize(byteBuffer);
groupByTimeParameter.serialize(byteBuffer);
- regionReplicaSet.serializeImpl(byteBuffer);
+ ThriftCommonsSerDeUtils.writeTRegionReplicaSet(regionReplicaSet, byteBuffer);
}
public static SeriesAggregateScanNode deserialize(ByteBuffer byteBuffer) {
@@ -227,7 +228,7 @@ public class SeriesAggregateScanNode extends SourceNode implements IOutputPlanNo
Filter timeFilter = FilterFactory.deserialize(byteBuffer);
GroupByTimeComponent groupByTimeComponent = GroupByTimeComponent.deserialize(byteBuffer);
- RegionReplicaSet regionReplicaSet = RegionReplicaSet.deserializeImpl(byteBuffer);
+ TRegionReplicaSet regionReplicaSet = ThriftCommonsSerDeUtils.readTRegionReplicaSet(byteBuffer);
PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
SeriesAggregateScanNode seriesAggregateScanNode =
new SeriesAggregateScanNode(
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesScanNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesScanNode.java
index bf0df051bc..76327e002f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesScanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SeriesScanNode.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.planner.plan.node.source;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
import org.apache.iotdb.db.metadata.path.PartialPath;
import org.apache.iotdb.db.metadata.path.PathDeserializeUtil;
import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
@@ -79,7 +80,7 @@ public class SeriesScanNode extends SourceNode implements IOutputPlanNode {
private ColumnHeader columnHeader;
// The id of DataRegion where the node will run
- private RegionReplicaSet regionReplicaSet;
+ private TRegionReplicaSet regionReplicaSet;
public SeriesScanNode(PlanNodeId id, PartialPath seriesPath) {
super(id);
@@ -95,7 +96,7 @@ public class SeriesScanNode extends SourceNode implements IOutputPlanNode {
this.columnHeader = new ColumnHeader(seriesPath.getFullPath(), seriesPath.getSeriesType());
}
- public SeriesScanNode(PlanNodeId id, PartialPath seriesPath, RegionReplicaSet regionReplicaSet) {
+ public SeriesScanNode(PlanNodeId id, PartialPath seriesPath, TRegionReplicaSet regionReplicaSet) {
this(id, seriesPath);
this.regionReplicaSet = regionReplicaSet;
}
@@ -115,12 +116,12 @@ public class SeriesScanNode extends SourceNode implements IOutputPlanNode {
public void open() throws Exception {}
@Override
- public RegionReplicaSet getRegionReplicaSet() {
+ public TRegionReplicaSet getRegionReplicaSet() {
return regionReplicaSet;
}
@Override
- public void setRegionReplicaSet(RegionReplicaSet dataRegion) {
+ public void setRegionReplicaSet(TRegionReplicaSet dataRegion) {
this.regionReplicaSet = dataRegion;
}
@@ -209,7 +210,8 @@ public class SeriesScanNode extends SourceNode implements IOutputPlanNode {
if (isNull == 1) valueFilter = FilterFactory.deserialize(byteBuffer);
int limit = ReadWriteIOUtils.readInt(byteBuffer);
int offset = ReadWriteIOUtils.readInt(byteBuffer);
- RegionReplicaSet dataRegionReplicaSet = RegionReplicaSet.deserializeImpl(byteBuffer);
+ TRegionReplicaSet dataRegionReplicaSet =
+ ThriftCommonsSerDeUtils.readTRegionReplicaSet(byteBuffer);
PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
SeriesScanNode seriesScanNode = new SeriesScanNode(planNodeId, partialPath);
seriesScanNode.allSensors = allSensors;
@@ -250,7 +252,7 @@ public class SeriesScanNode extends SourceNode implements IOutputPlanNode {
}
ReadWriteIOUtils.write(limit, byteBuffer);
ReadWriteIOUtils.write(offset, byteBuffer);
- regionReplicaSet.serializeImpl(byteBuffer);
+ ThriftCommonsSerDeUtils.writeTRegionReplicaSet(regionReplicaSet, byteBuffer);
}
public PartialPath getSeriesPath() {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SourceNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SourceNode.java
index 5d674371e9..9fe3e58140 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SourceNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/source/SourceNode.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.sql.planner.plan.node.source;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
@@ -30,7 +30,7 @@ public abstract class SourceNode extends PlanNode implements AutoCloseable {
public abstract void open() throws Exception;
- public abstract RegionReplicaSet getRegionReplicaSet();
+ public abstract TRegionReplicaSet getRegionReplicaSet();
- public abstract void setRegionReplicaSet(RegionReplicaSet regionReplicaSet);
+ public abstract void setRegionReplicaSet(TRegionReplicaSet regionReplicaSet);
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertMultiTabletsNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertMultiTabletsNode.java
index 3e2fd699d5..0aec9e691c 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertMultiTabletsNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertMultiTabletsNode.java
@@ -18,8 +18,8 @@
*/
package org.apache.iotdb.db.mpp.sql.planner.plan.node.write;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
import org.apache.iotdb.commons.utils.StatusUtils;
import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNode;
@@ -110,12 +110,12 @@ public class InsertMultiTabletsNode extends InsertNode {
@Override
public List<WritePlanNode> splitByPartition(Analysis analysis) {
- Map<RegionReplicaSet, InsertMultiTabletsNode> splitMap = new HashMap<>();
+ Map<TRegionReplicaSet, InsertMultiTabletsNode> splitMap = new HashMap<>();
for (int i = 0; i < insertTabletNodeList.size(); i++) {
InsertTabletNode insertTabletNode = insertTabletNodeList.get(i);
List<WritePlanNode> tmpResult = insertTabletNode.splitByPartition(analysis);
for (WritePlanNode subNode : tmpResult) {
- RegionReplicaSet dataRegionReplicaSet = ((InsertNode) subNode).getDataRegionReplicaSet();
+ TRegionReplicaSet dataRegionReplicaSet = ((InsertNode) subNode).getDataRegionReplicaSet();
if (splitMap.containsKey(dataRegionReplicaSet)) {
InsertMultiTabletsNode tmpNode = splitMap.get(dataRegionReplicaSet);
tmpNode.addInsertTabletNode((InsertTabletNode) subNode, i);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertNode.java
index 08c861913d..62cbffb55b 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertNode.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.sql.planner.plan.node.write;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
import org.apache.iotdb.db.metadata.path.PartialPath;
import org.apache.iotdb.db.mpp.sql.planner.plan.node.PlanNodeId;
@@ -53,7 +53,7 @@ public abstract class InsertNode extends WritePlanNode {
protected IDeviceID deviceID;
/** Physical address of data region after splitting */
- RegionReplicaSet dataRegionReplicaSet;
+ TRegionReplicaSet dataRegionReplicaSet;
protected InsertNode(PlanNodeId id) {
super(id);
@@ -79,11 +79,11 @@ public abstract class InsertNode extends WritePlanNode {
}
}
- public RegionReplicaSet getDataRegionReplicaSet() {
+ public TRegionReplicaSet getDataRegionReplicaSet() {
return dataRegionReplicaSet;
}
- public void setDataRegionReplicaSet(RegionReplicaSet dataRegionReplicaSet) {
+ public void setDataRegionReplicaSet(TRegionReplicaSet dataRegionReplicaSet) {
this.dataRegionReplicaSet = dataRegionReplicaSet;
}
@@ -131,7 +131,7 @@ public abstract class InsertNode extends WritePlanNode {
this.deviceID = deviceID;
}
- public RegionReplicaSet getRegionReplicaSet() {
+ public TRegionReplicaSet getRegionReplicaSet() {
return dataRegionReplicaSet;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowNode.java
index e999046ff8..cf144c9d51 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowNode.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.sql.planner.plan.node.write;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.db.engine.StorageEngine;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
@@ -76,7 +76,7 @@ public class InsertRowNode extends InsertNode implements WALEntryValue {
@Override
public List<WritePlanNode> splitByPartition(Analysis analysis) {
- TimePartitionSlot timePartitionSlot = StorageEngine.getTimePartitionSlot(time);
+ TTimePartitionSlot timePartitionSlot = StorageEngine.getTimePartitionSlot(time);
this.dataRegionReplicaSet =
analysis
.getDataPartitionInfo()
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsNode.java
index b82370ecfa..62fad82b81 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsNode.java
@@ -18,8 +18,8 @@
*/
package org.apache.iotdb.db.mpp.sql.planner.plan.node.write;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
import org.apache.iotdb.commons.utils.StatusUtils;
import org.apache.iotdb.db.engine.StorageEngine;
import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
@@ -132,11 +132,11 @@ public class InsertRowsNode extends InsertNode {
@Override
public List<WritePlanNode> splitByPartition(Analysis analysis) {
- Map<RegionReplicaSet, InsertRowsNode> splitMap = new HashMap<>();
+ Map<TRegionReplicaSet, InsertRowsNode> splitMap = new HashMap<>();
for (int i = 0; i < insertRowNodeList.size(); i++) {
InsertRowNode insertRowNode = insertRowNodeList.get(i);
// data region for insert row node
- RegionReplicaSet dataRegionReplicaSet =
+ TRegionReplicaSet dataRegionReplicaSet =
analysis
.getDataPartitionInfo()
.getDataRegionReplicaSetForWriting(
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsOfOneDeviceNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsOfOneDeviceNode.java
index a20beb3876..f83912efa9 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsOfOneDeviceNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertRowsOfOneDeviceNode.java
@@ -18,8 +18,8 @@
*/
package org.apache.iotdb.db.mpp.sql.planner.plan.node.write;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
import org.apache.iotdb.commons.utils.StatusUtils;
import org.apache.iotdb.db.engine.StorageEngine;
import org.apache.iotdb.db.mpp.sql.analyze.Analysis;
@@ -113,11 +113,11 @@ public class InsertRowsOfOneDeviceNode extends InsertNode {
@Override
public List<WritePlanNode> splitByPartition(Analysis analysis) {
- Map<RegionReplicaSet, InsertRowsNode> splitMap = new HashMap<>();
+ Map<TRegionReplicaSet, InsertRowsNode> splitMap = new HashMap<>();
for (int i = 0; i < insertRowNodeList.size(); i++) {
InsertRowNode insertRowNode = insertRowNodeList.get(i);
// data region for insert row node
- RegionReplicaSet dataRegionReplicaSet =
+ TRegionReplicaSet dataRegionReplicaSet =
analysis
.getDataPartitionInfo()
.getDataRegionReplicaSetForWriting(
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertTabletNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertTabletNode.java
index f53e716680..ef8a2e3b93 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertTabletNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/planner/plan/node/write/InsertTabletNode.java
@@ -18,8 +18,8 @@
*/
package org.apache.iotdb.db.mpp.sql.planner.plan.node.write;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.db.engine.StorageEngine;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.PartialPath;
@@ -552,10 +552,10 @@ public class InsertTabletNode extends InsertNode implements WALEntryValue {
(times[0] / StorageEngine.getTimePartitionInterval())
* StorageEngine.getTimePartitionInterval(); // included
long endTime = startTime + StorageEngine.getTimePartitionInterval(); // excluded
- TimePartitionSlot timePartitionSlot = StorageEngine.getTimePartitionSlot(times[0]);
+ TTimePartitionSlot timePartitionSlot = StorageEngine.getTimePartitionSlot(times[0]);
int startLoc = 0; // included
- List<TimePartitionSlot> timePartitionSlots = new ArrayList<>();
+ List<TTimePartitionSlot> timePartitionSlots = new ArrayList<>();
// for each List in split, they are range1.start, range1.end, range2.start, range2.end, ...
List<Integer> ranges = new ArrayList<>();
for (int i = 1; i < times.length; i++) { // times are sorted in session API.
@@ -580,12 +580,12 @@ public class InsertTabletNode extends InsertNode implements WALEntryValue {
timePartitionSlots.add(timePartitionSlot);
// data region for each time partition
- List<RegionReplicaSet> dataRegionReplicaSets =
+ List<TRegionReplicaSet> dataRegionReplicaSets =
analysis
.getDataPartitionInfo()
.getDataRegionReplicaSetForWriting(devicePath.getFullPath(), timePartitionSlots);
- Map<RegionReplicaSet, List<Integer>> splitMap = new HashMap<>();
+ Map<TRegionReplicaSet, List<Integer>> splitMap = new HashMap<>();
for (int i = 0; i < dataRegionReplicaSets.size(); i++) {
List<Integer> sub_ranges =
splitMap.computeIfAbsent(dataRegionReplicaSets.get(i), x -> new ArrayList<>());
@@ -594,7 +594,7 @@ public class InsertTabletNode extends InsertNode implements WALEntryValue {
}
List<Integer> locs;
- for (Map.Entry<RegionReplicaSet, List<Integer>> entry : splitMap.entrySet()) {
+ for (Map.Entry<TRegionReplicaSet, List<Integer>> entry : splitMap.entrySet()) {
// generate a new times and values
locs = entry.getValue();
int count = 0;
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowStatement.java
index 80862ab70e..858df6a94e 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowStatement.java
@@ -19,8 +19,8 @@
package org.apache.iotdb.db.mpp.sql.statement.crud;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.commons.conf.IoTDBConstant;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.StorageEngine;
import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
@@ -176,7 +176,7 @@ public class InsertRowStatement extends InsertBaseStatement {
dataTypes[index] = null;
}
- public List<TimePartitionSlot> getTimePartitionSlots() {
+ public List<TTimePartitionSlot> getTimePartitionSlots() {
return Collections.singletonList(StorageEngine.getTimePartitionSlot(time));
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowsOfOneDeviceStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowsOfOneDeviceStatement.java
index 2d4079b21c..f83f42c27f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowsOfOneDeviceStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertRowsOfOneDeviceStatement.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.mpp.sql.statement.crud;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.db.engine.StorageEngine;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.mpp.common.schematree.SchemaTree;
@@ -61,8 +61,8 @@ public class InsertRowsOfOneDeviceStatement extends InsertBaseStatement {
dataTypes = measurementsAndDataType.values().toArray(new TSDataType[0]);
}
- public List<TimePartitionSlot> getTimePartitionSlots() {
- Set<TimePartitionSlot> timePartitionSlotSet = new HashSet<>();
+ public List<TTimePartitionSlot> getTimePartitionSlots() {
+ Set<TTimePartitionSlot> timePartitionSlotSet = new HashSet<>();
for (InsertRowStatement insertRowStatement : insertRowStatementList) {
timePartitionSlotSet.add(StorageEngine.getTimePartitionSlot(insertRowStatement.getTime()));
}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertTabletStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertTabletStatement.java
index 23d740315e..12aa64db34 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertTabletStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/sql/statement/crud/InsertTabletStatement.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.sql.statement.crud;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.StorageEngine;
import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
@@ -82,13 +82,13 @@ public class InsertTabletStatement extends InsertBaseStatement {
bitMaps[index] = null;
}
- public List<TimePartitionSlot> getTimePartitionSlots() {
- List<TimePartitionSlot> result = new ArrayList<>();
+ public List<TTimePartitionSlot> getTimePartitionSlots() {
+ List<TTimePartitionSlot> result = new ArrayList<>();
long startTime =
(times[0] / StorageEngine.getTimePartitionInterval())
* StorageEngine.getTimePartitionInterval(); // included
long endTime = startTime + StorageEngine.getTimePartitionInterval(); // excluded
- TimePartitionSlot timePartitionSlot = StorageEngine.getTimePartitionSlot(times[0]);
+ TTimePartitionSlot timePartitionSlot = StorageEngine.getTimePartitionSlot(times[0]);
for (int i = 1; i < times.length; i++) { // times are sorted in session API.
if (times[i] >= endTime) {
result.add(timePartitionSlot);
diff --git a/server/src/main/java/org/apache/iotdb/db/service/DataNode.java b/server/src/main/java/org/apache/iotdb/db/service/DataNode.java
index f2285abbc3..7774b43826 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/DataNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/DataNode.java
@@ -18,8 +18,8 @@
*/
package org.apache.iotdb.db.service;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
import org.apache.iotdb.common.rpc.thrift.TEndPoint;
-import org.apache.iotdb.commons.cluster.Endpoint;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.commons.exception.BadNodeUrlException;
import org.apache.iotdb.commons.exception.ConfigurationException;
@@ -54,7 +54,7 @@ public class DataNode implements DataNodeMBean {
*/
private static final int DEFAULT_JOIN_RETRY = 10;
- private Endpoint thisNode = new Endpoint();
+ private TEndPoint thisNode = new TEndPoint();
private int dataNodeID;
@@ -123,14 +123,25 @@ public class DataNode implements DataNodeMBean {
while (retry > 0) {
logger.info("start joining the cluster.");
try {
- TDataNodeRegisterResp dataNodeRegisterResp =
- configNodeClient.registerDataNode(
- new TDataNodeRegisterReq(new TEndPoint(thisNode.getIp(), thisNode.getPort())));
+ IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+ TDataNodeRegisterReq req = new TDataNodeRegisterReq();
+ TDataNodeLocation location = new TDataNodeLocation();
+ location.setDataNodeId(-1);
+ location.setExternalEndPoint(new TEndPoint(config.getRpcAddress(), config.getRpcPort()));
+ location.setInternalEndPoint(
+ new TEndPoint(config.getInternalIp(), config.getInternalPort()));
+ location.setDataBlockManagerEndPoint(
+ new TEndPoint(config.getInternalIp(), config.getDataBlockManagerPort()));
+ location.setConsensusEndPoint(
+ new TEndPoint(config.getInternalIp(), config.getConsensusPort()));
+ req.setDataNodeLocation(location);
+
+ TDataNodeRegisterResp dataNodeRegisterResp = configNodeClient.registerDataNode(req);
if (dataNodeRegisterResp.getStatus().getCode()
== TSStatusCode.SUCCESS_STATUS.getStatusCode()
|| dataNodeRegisterResp.getStatus().getCode()
== TSStatusCode.DATANODE_ALREADY_REGISTERED.getStatusCode()) {
- dataNodeID = dataNodeRegisterResp.getDataNodeID();
+ dataNodeID = dataNodeRegisterResp.getDataNodeId();
IoTDBDescriptor.getInstance().loadGlobalConfig(dataNodeRegisterResp.globalConfig);
logger.info("Joined the cluster successfully");
return;
diff --git a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/InternalServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/InternalServiceImpl.java
index b101e9bd78..eb77ccb6cd 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/InternalServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/InternalServiceImpl.java
@@ -19,13 +19,12 @@
package org.apache.iotdb.db.service.thrift.impl;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.cluster.Endpoint;
import org.apache.iotdb.commons.consensus.ConsensusGroupId;
import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.consensus.GroupType;
import org.apache.iotdb.commons.consensus.SchemaRegionId;
import org.apache.iotdb.consensus.IConsensus;
import org.apache.iotdb.consensus.common.Peer;
@@ -66,7 +65,6 @@ import org.apache.thrift.TException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
@@ -85,8 +83,7 @@ public class InternalServiceImpl implements InternalService.Iface {
public TSendFragmentInstanceResp sendFragmentInstance(TSendFragmentInstanceReq req) {
QueryType type = QueryType.valueOf(req.queryType);
ConsensusGroupId groupId =
- ConsensusGroupId.Factory.create(
- req.consensusGroupId.id, GroupType.valueOf(req.consensusGroupId.type));
+ ConsensusGroupId.Factory.convertFromTConsensusGroupId(req.getConsensusGroupId());
switch (type) {
case READ:
ConsensusReadResponse readResp =
@@ -146,16 +143,14 @@ public class InternalServiceImpl implements InternalService.Iface {
try {
PartialPath storageGroupPartitionPath = new PartialPath(req.getStorageGroup());
TRegionReplicaSet regionReplicaSet = req.getRegionReplicaSet();
- SchemaRegionId schemaRegionId =
- (SchemaRegionId)
- ConsensusGroupId.Factory.create(ByteBuffer.wrap(regionReplicaSet.getRegionId()));
- LOGGER.info("SchemaRegionId: " + schemaRegionId.getId());
+ SchemaRegionId schemaRegionId = new SchemaRegionId(regionReplicaSet.getRegionId().getId());
schemaEngine.createSchemaRegion(storageGroupPartitionPath, schemaRegionId);
List<Peer> peers = new ArrayList<>();
- for (TEndPoint endPoint : regionReplicaSet.getEndpoint()) {
- Endpoint endpoint = new Endpoint(endPoint.getIp(), endPoint.getPort());
- // TODO: Expend Peer and RegisterDataNodeReq
- endpoint.setPort(endpoint.getPort() + 31007);
+ for (TDataNodeLocation dataNodeLocation : regionReplicaSet.getDataNodeLocations()) {
+ TEndPoint endpoint =
+ new TEndPoint(
+ dataNodeLocation.getConsensusEndPoint().getIp(),
+ dataNodeLocation.getConsensusEndPoint().getPort());
peers.add(new Peer(schemaRegionId, endpoint));
}
ConsensusGenericResponse consensusGenericResponse =
@@ -186,16 +181,14 @@ public class InternalServiceImpl implements InternalService.Iface {
TSStatus tsStatus;
try {
TRegionReplicaSet regionReplicaSet = req.getRegionReplicaSet();
- DataRegionId dataRegionId =
- (DataRegionId)
- ConsensusGroupId.Factory.create(ByteBuffer.wrap(regionReplicaSet.getRegionId()));
- LOGGER.info("DataRegionId: " + dataRegionId.getId());
+ DataRegionId dataRegionId = new DataRegionId(regionReplicaSet.getRegionId().getId());
storageEngine.createDataRegion(dataRegionId, req.storageGroup, req.ttl);
List<Peer> peers = new ArrayList<>();
- for (TEndPoint endPoint : regionReplicaSet.getEndpoint()) {
- Endpoint endpoint = new Endpoint(endPoint.getIp(), endPoint.getPort());
- // TODO: Expend Peer and RegisterDataNodeReq
- endpoint.setPort(endpoint.getPort() + 31007);
+ for (TDataNodeLocation dataNodeLocation : regionReplicaSet.getDataNodeLocations()) {
+ TEndPoint endpoint =
+ new TEndPoint(
+ dataNodeLocation.getConsensusEndPoint().getIp(),
+ dataNodeLocation.getConsensusEndPoint().getPort());
peers.add(new Peer(dataRegionId, endpoint));
}
ConsensusGenericResponse consensusGenericResponse =
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/DistributionPlannerTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/DistributionPlannerTest.java
index ee4d5dedc9..08d80476cb 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/DistributionPlannerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/DistributionPlannerTest.java
@@ -19,15 +19,15 @@
package org.apache.iotdb.db.mpp.sql.plan;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.consensus.SchemaRegionId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
+import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.commons.partition.DataPartition;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
import org.apache.iotdb.commons.partition.SchemaPartition;
-import org.apache.iotdb.commons.partition.SeriesPartitionSlot;
-import org.apache.iotdb.commons.partition.TimePartitionSlot;
import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
@@ -80,7 +80,7 @@ public class DistributionPlannerTest {
Analysis analysis = constructAnalysis();
- MPPQueryContext context = new MPPQueryContext("", queryId, null, new Endpoint());
+ MPPQueryContext context = new MPPQueryContext("", queryId, null, new TEndPoint());
DistributionPlanner planner =
new DistributionPlanner(analysis, new LogicalQueryPlan(context, root));
DistributedQueryPlan plan = planner.planFragments();
@@ -100,7 +100,7 @@ public class DistributionPlannerTest {
Analysis analysis = constructAnalysis();
- MPPQueryContext context = new MPPQueryContext("", queryId, null, new Endpoint());
+ MPPQueryContext context = new MPPQueryContext("", queryId, null, new TEndPoint());
DistributionPlanner planner =
new DistributionPlanner(analysis, new LogicalQueryPlan(context, root));
PlanNode rootAfterRewrite = planner.rewriteSource();
@@ -257,7 +257,7 @@ public class DistributionPlannerTest {
Analysis analysis = constructAnalysis();
- MPPQueryContext context = new MPPQueryContext("", queryId, null, new Endpoint());
+ MPPQueryContext context = new MPPQueryContext("", queryId, null, new TEndPoint());
DistributionPlanner planner =
new DistributionPlanner(analysis, new LogicalQueryPlan(context, root));
PlanNode rootAfterRewrite = planner.rewriteSource();
@@ -294,7 +294,7 @@ public class DistributionPlannerTest {
Analysis analysis = constructAnalysis();
- MPPQueryContext context = new MPPQueryContext("", queryId, null, new Endpoint());
+ MPPQueryContext context = new MPPQueryContext("", queryId, null, new TEndPoint());
DistributionPlanner planner =
new DistributionPlanner(analysis, new LogicalQueryPlan(context, root));
DistributedQueryPlan plan = planner.planFragments();
@@ -318,7 +318,7 @@ public class DistributionPlannerTest {
Analysis analysis = constructAnalysis();
- MPPQueryContext context = new MPPQueryContext("", queryId, null, new Endpoint());
+ MPPQueryContext context = new MPPQueryContext("", queryId, null, new TEndPoint());
context.setQueryType(QueryType.WRITE);
DistributionPlanner planner =
new DistributionPlanner(analysis, new LogicalQueryPlan(context, insertRowNode));
@@ -360,7 +360,7 @@ public class DistributionPlannerTest {
Analysis analysis = constructAnalysis();
- MPPQueryContext context = new MPPQueryContext("", queryId, null, new Endpoint());
+ MPPQueryContext context = new MPPQueryContext("", queryId, null, new TEndPoint());
context.setQueryType(QueryType.WRITE);
DistributionPlanner planner =
new DistributionPlanner(analysis, new LogicalQueryPlan(context, node));
@@ -385,56 +385,73 @@ public class DistributionPlannerTest {
new DataPartition(
IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionExecutorClass(),
IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionSlotNum());
- Map<String, Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>>>
+
+ Map<String, Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>>>
dataPartitionMap = new HashMap<>();
- Map<SeriesPartitionSlot, Map<TimePartitionSlot, List<RegionReplicaSet>>> sgPartitionMap =
+ Map<TSeriesPartitionSlot, Map<TTimePartitionSlot, List<TRegionReplicaSet>>> sgPartitionMap =
new HashMap<>();
- List<RegionReplicaSet> d1DataRegion1 = new ArrayList<>();
- d1DataRegion1.add(
- new RegionReplicaSet(
- new DataRegionId(1),
+ List<TRegionReplicaSet> d1DataRegions = new ArrayList<>();
+ d1DataRegions.add(
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1),
Arrays.asList(
- new DataNodeLocation(11, new Endpoint("192.0.1.1", 9000)),
- new DataNodeLocation(12, new Endpoint("192.0.1.2", 9000)))));
-
- List<RegionReplicaSet> d1DataRegion2 = new ArrayList<>();
- d1DataRegion2.add(
- new RegionReplicaSet(
- new DataRegionId(2),
+ new TDataNodeLocation()
+ .setDataNodeId(11)
+ .setExternalEndPoint(new TEndPoint("192.0.1.1", 9000)),
+ new TDataNodeLocation()
+ .setDataNodeId(12)
+ .setExternalEndPoint(new TEndPoint("192.0.1.2", 9000)))));
+ d1DataRegions.add(
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 2),
Arrays.asList(
- new DataNodeLocation(21, new Endpoint("192.0.2.1", 9000)),
- new DataNodeLocation(22, new Endpoint("192.0.2.2", 9000)))));
-
- Map<TimePartitionSlot, List<RegionReplicaSet>> d1DataRegionMap = new HashMap<>();
- d1DataRegionMap.put(new TimePartitionSlot(0), d1DataRegion1);
- d1DataRegionMap.put(new TimePartitionSlot(1), d1DataRegion2);
-
- List<RegionReplicaSet> d2DataRegions = new ArrayList<>();
+ new TDataNodeLocation()
+ .setDataNodeId(21)
+ .setExternalEndPoint(new TEndPoint("192.0.2.1", 9000)),
+ new TDataNodeLocation()
+ .setDataNodeId(22)
+ .setExternalEndPoint(new TEndPoint("192.0.2.2", 9000)))));
+ Map<TTimePartitionSlot, List<TRegionReplicaSet>> d1DataRegionMap = new HashMap<>();
+ d1DataRegionMap.put(new TTimePartitionSlot(), d1DataRegions);
+
+ List<TRegionReplicaSet> d2DataRegions = new ArrayList<>();
d2DataRegions.add(
- new RegionReplicaSet(
- new DataRegionId(3),
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 3),
Arrays.asList(
- new DataNodeLocation(31, new Endpoint("192.0.3.1", 9000)),
- new DataNodeLocation(32, new Endpoint("192.0.3.2", 9000)))));
- Map<TimePartitionSlot, List<RegionReplicaSet>> d2DataRegionMap = new HashMap<>();
- d2DataRegionMap.put(new TimePartitionSlot(0), d2DataRegions);
-
- List<RegionReplicaSet> d3DataRegions = new ArrayList<>();
+ new TDataNodeLocation()
+ .setDataNodeId(31)
+ .setExternalEndPoint(new TEndPoint("192.0.3.1", 9000)),
+ new TDataNodeLocation()
+ .setDataNodeId(32)
+ .setExternalEndPoint(new TEndPoint("192.0.3.2", 9000)))));
+ Map<TTimePartitionSlot, List<TRegionReplicaSet>> d2DataRegionMap = new HashMap<>();
+ d2DataRegionMap.put(new TTimePartitionSlot(), d2DataRegions);
+
+ List<TRegionReplicaSet> d3DataRegions = new ArrayList<>();
d3DataRegions.add(
- new RegionReplicaSet(
- new DataRegionId(1),
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1),
Arrays.asList(
- new DataNodeLocation(11, new Endpoint("192.0.1.1", 9000)),
- new DataNodeLocation(12, new Endpoint("192.0.1.2", 9000)))));
+ new TDataNodeLocation()
+ .setDataNodeId(11)
+ .setExternalEndPoint(new TEndPoint("192.0.1.1", 9000)),
+ new TDataNodeLocation()
+ .setDataNodeId(12)
+ .setExternalEndPoint(new TEndPoint("192.0.1.2", 9000)))));
d3DataRegions.add(
- new RegionReplicaSet(
- new DataRegionId(4),
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 4),
Arrays.asList(
- new DataNodeLocation(41, new Endpoint("192.0.4.1", 9000)),
- new DataNodeLocation(42, new Endpoint("192.0.4.2", 9000)))));
- Map<TimePartitionSlot, List<RegionReplicaSet>> d3DataRegionMap = new HashMap<>();
- d3DataRegionMap.put(new TimePartitionSlot(0), d3DataRegions);
+ new TDataNodeLocation()
+ .setDataNodeId(41)
+ .setExternalEndPoint(new TEndPoint("192.0.4.1", 9000)),
+ new TDataNodeLocation()
+ .setDataNodeId(42)
+ .setExternalEndPoint(new TEndPoint("192.0.4.2", 9000)))));
+ Map<TTimePartitionSlot, List<TRegionReplicaSet>> d3DataRegionMap = new HashMap<>();
+ d3DataRegionMap.put(new TTimePartitionSlot(), d3DataRegions);
sgPartitionMap.put(executor.getSeriesPartitionSlot(device1), d1DataRegionMap);
sgPartitionMap.put(executor.getSeriesPartitionSlot(device2), d2DataRegionMap);
@@ -451,22 +468,30 @@ public class DistributionPlannerTest {
new SchemaPartition(
IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionExecutorClass(),
IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionSlotNum());
- Map<String, Map<SeriesPartitionSlot, RegionReplicaSet>> schemaPartitionMap = new HashMap<>();
+ Map<String, Map<TSeriesPartitionSlot, TRegionReplicaSet>> schemaPartitionMap = new HashMap<>();
- RegionReplicaSet schemaRegion1 =
- new RegionReplicaSet(
- new SchemaRegionId(11),
+ TRegionReplicaSet schemaRegion1 =
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 11),
Arrays.asList(
- new DataNodeLocation(11, new Endpoint("192.0.1.1", 9000)),
- new DataNodeLocation(12, new Endpoint("192.0.1.2", 9000))));
- Map<SeriesPartitionSlot, RegionReplicaSet> schemaRegionMap = new HashMap<>();
-
- RegionReplicaSet schemaRegion2 =
- new RegionReplicaSet(
- new SchemaRegionId(21),
+ new TDataNodeLocation()
+ .setDataNodeId(11)
+ .setExternalEndPoint(new TEndPoint("192.0.1.1", 9000)),
+ new TDataNodeLocation()
+ .setDataNodeId(12)
+ .setExternalEndPoint(new TEndPoint("192.0.1.2", 9000))));
+ Map<TSeriesPartitionSlot, TRegionReplicaSet> schemaRegionMap = new HashMap<>();
+
+ TRegionReplicaSet schemaRegion2 =
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 21),
Arrays.asList(
- new DataNodeLocation(21, new Endpoint("192.0.1.1", 9000)),
- new DataNodeLocation(22, new Endpoint("192.0.1.2", 9000))));
+ new TDataNodeLocation()
+ .setDataNodeId(21)
+ .setExternalEndPoint(new TEndPoint("192.0.1.1", 9000)),
+ new TDataNodeLocation()
+ .setDataNodeId(22)
+ .setExternalEndPoint(new TEndPoint("192.0.1.2", 9000))));
schemaRegionMap.put(executor.getSeriesPartitionSlot(device1), schemaRegion1);
schemaRegionMap.put(executor.getSeriesPartitionSlot(device2), schemaRegion2);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/FragmentInstanceSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/FragmentInstanceSerdeTest.java
index 4bf6a477e2..5bf598cf14 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/FragmentInstanceSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/FragmentInstanceSerdeTest.java
@@ -18,10 +18,11 @@
*/
package org.apache.iotdb.db.mpp.sql.plan;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.MeasurementPath;
import org.apache.iotdb.db.mpp.common.PlanFragmentId;
@@ -56,6 +57,13 @@ public class FragmentInstanceSerdeTest {
@Test
public void TestSerializeAndDeserializeForTree1() throws IllegalPathException {
+ TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+ dataNodeLocation.setDataNodeId(0);
+ dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667));
+ dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+ dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777));
+ dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010));
+
PlanFragmentId planFragmentId = new PlanFragmentId("test", -1);
FragmentInstance fragmentInstance =
new FragmentInstance(
@@ -63,10 +71,10 @@ public class FragmentInstanceSerdeTest {
planFragmentId.genFragmentInstanceId(),
new GroupByFilter(1, 2, 3, 4),
QueryType.READ);
- RegionReplicaSet regionReplicaSet =
- new RegionReplicaSet(
- new DataRegionId(1),
- ImmutableList.of(new DataNodeLocation(0, new Endpoint("127.0.0.1", 6666))));
+ TRegionReplicaSet regionReplicaSet =
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1),
+ ImmutableList.of(dataNodeLocation));
fragmentInstance.setDataRegionAndHost(regionReplicaSet);
ByteBuffer byteBuffer = ByteBuffer.allocate(1024);
@@ -78,6 +86,13 @@ public class FragmentInstanceSerdeTest {
@Test
public void TestSerializeAndDeserializeWithNullFilter() throws IllegalPathException {
+ TDataNodeLocation dataNodeLocation = new TDataNodeLocation();
+ dataNodeLocation.setDataNodeId(0);
+ dataNodeLocation.setExternalEndPoint(new TEndPoint("0.0.0.0", 6667));
+ dataNodeLocation.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+ dataNodeLocation.setDataBlockManagerEndPoint(new TEndPoint("0.0.0.0", 8777));
+ dataNodeLocation.setConsensusEndPoint(new TEndPoint("0.0.0.0", 40010));
+
PlanFragmentId planFragmentId = new PlanFragmentId("test2", 1);
FragmentInstance fragmentInstance =
new FragmentInstance(
@@ -85,10 +100,10 @@ public class FragmentInstanceSerdeTest {
planFragmentId.genFragmentInstanceId(),
null,
QueryType.READ);
- RegionReplicaSet regionReplicaSet =
- new RegionReplicaSet(
- new DataRegionId(1),
- ImmutableList.of(new DataNodeLocation(0, new Endpoint("127.0.0.2", 6667))));
+ TRegionReplicaSet regionReplicaSet =
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1),
+ ImmutableList.of(dataNodeLocation));
fragmentInstance.setDataRegionAndHost(regionReplicaSet);
ByteBuffer byteBuffer = ByteBuffer.allocate(1024);
@@ -125,17 +140,20 @@ public class FragmentInstanceSerdeTest {
SeriesScanNode seriesScanNode1 =
new SeriesScanNode(new PlanNodeId("SeriesScanNode1"), new MeasurementPath("root.sg.d1.s2"));
seriesScanNode1.setRegionReplicaSet(
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
seriesScanNode1.setScanOrder(OrderBy.TIMESTAMP_DESC);
SeriesScanNode seriesScanNode2 =
new SeriesScanNode(new PlanNodeId("SeriesScanNode2"), new MeasurementPath("root.sg.d2.s1"));
seriesScanNode2.setRegionReplicaSet(
- new RegionReplicaSet(new DataRegionId(2), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 2), new ArrayList<>()));
seriesScanNode2.setScanOrder(OrderBy.TIMESTAMP_DESC);
SeriesScanNode seriesScanNode3 =
new SeriesScanNode(new PlanNodeId("SeriesScanNode3"), new MeasurementPath("root.sg.d2.s2"));
seriesScanNode3.setRegionReplicaSet(
- new RegionReplicaSet(new DataRegionId(3), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 3), new ArrayList<>()));
seriesScanNode3.setScanOrder(OrderBy.TIMESTAMP_DESC);
// build tree
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/QueryPlannerTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/QueryPlannerTest.java
index a0b28041d4..f10596f775 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/QueryPlannerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/QueryPlannerTest.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.mpp.sql.plan;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
import org.apache.iotdb.db.mpp.common.MPPQueryContext;
import org.apache.iotdb.db.mpp.common.QueryId;
@@ -50,7 +50,8 @@ public class QueryPlannerTest {
QueryExecution queryExecution =
new QueryExecution(
stmt,
- new MPPQueryContext(querySql, new QueryId("query1"), new SessionInfo(), new Endpoint()),
+ new MPPQueryContext(
+ querySql, new QueryId("query1"), new SessionInfo(), new TEndPoint()),
IoTDBThreadPoolFactory.newSingleThreadExecutor("test_query"),
IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("test_query_scheduled"),
new FakePartitionFetcherImpl(),
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java
index eadfb2258c..185ba5a4bc 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.metadata.read;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.PartialPath;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
@@ -56,13 +56,13 @@ public class DeviceSchemaScanNodeSerdeTest {
FragmentSinkNode fragmentSinkNode = new FragmentSinkNode(new PlanNodeId("fragmentSink"));
fragmentSinkNode.addChild(devicesSchemaScanNode);
fragmentSinkNode.setDownStream(
- new Endpoint("127.0.0.1", 6667),
+ new TEndPoint("127.0.0.1", 6667),
new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
new PlanNodeId("test"));
exchangeNode.addChild(schemaMergeNode);
exchangeNode.setRemoteSourceNode(fragmentSinkNode);
exchangeNode.setUpstream(
- new Endpoint("127.0.0.1", 6667),
+ new TEndPoint("127.0.0.1", 6667),
new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
new PlanNodeId("test"));
offsetNode.addChild(exchangeNode);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java
index 3680c12251..e3c49ffa92 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.metadata.read;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.PartialPath;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
@@ -59,13 +59,13 @@ public class TimeSeriesSchemaScanNodeSerdeTest {
FragmentSinkNode fragmentSinkNode = new FragmentSinkNode(new PlanNodeId("fragmentSink"));
fragmentSinkNode.addChild(timeSeriesSchemaScanNode);
fragmentSinkNode.setDownStream(
- new Endpoint("127.0.0.1", 6667),
+ new TEndPoint("127.0.0.1", 6667),
new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
new PlanNodeId("test"));
exchangeNode.addChild(schemaMergeNode);
exchangeNode.setRemoteSourceNode(fragmentSinkNode);
exchangeNode.setUpstream(
- new Endpoint("127.0.0.1", 6667),
+ new TEndPoint("127.0.0.1", 6667),
new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
new PlanNodeId("test"));
offsetNode.addChild(exchangeNode);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/AggregateNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/AggregateNodeSerdeTest.java
index ab20e99e8e..4f1c52508d 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/AggregateNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/AggregateNodeSerdeTest.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.process;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.AlignedPath;
import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -57,7 +58,8 @@ public class AggregateNodeSerdeTest {
new SeriesScanNode(
new PlanNodeId("TestSeriesScanNode"),
new AlignedPath("s1"),
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
aggregateNode.addChild(seriesScanNode);
ByteBuffer byteBuffer = ByteBuffer.allocate(1024);
aggregateNode.serialize(byteBuffer);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/DeviceMergeNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/DeviceMergeNodeSerdeTest.java
index 66463f1aa5..dc8646a993 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/DeviceMergeNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/DeviceMergeNodeSerdeTest.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.process;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.AlignedPath;
import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -59,7 +60,8 @@ public class DeviceMergeNodeSerdeTest {
new SeriesScanNode(
new PlanNodeId("TestSeriesScanNode"),
new AlignedPath("s1"),
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
aggregateNode.addChild(seriesScanNode);
DeviceMergeNode deviceMergeNode =
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/ExchangeNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/ExchangeNodeSerdeTest.java
index 3dc47fb4f5..76563e29b9 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/ExchangeNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/ExchangeNodeSerdeTest.java
@@ -18,9 +18,10 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.process;
-import org.apache.iotdb.commons.cluster.Endpoint;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.AlignedPath;
import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -65,7 +66,8 @@ public class ExchangeNodeSerdeTest {
new SeriesScanNode(
new PlanNodeId("TestSeriesScanNode"),
new AlignedPath("s1"),
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
aggregateNode.addChild(seriesScanNode);
DeviceMergeNode deviceMergeNode =
@@ -91,14 +93,14 @@ public class ExchangeNodeSerdeTest {
FragmentSinkNode fragmentSinkNode =
new FragmentSinkNode(new PlanNodeId("TestFragmentSinkNode"));
fragmentSinkNode.setDownStream(
- new Endpoint("127.0.0.1", 6666),
+ new TEndPoint("127.0.0.1", 6666),
new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
new PlanNodeId("test"));
fragmentSinkNode.addChild(seriesScanNode);
exchangeNode.setRemoteSourceNode(fragmentSinkNode);
exchangeNode.addChild(deviceMergeNode);
exchangeNode.setUpstream(
- new Endpoint("127.0.0.1", 6666),
+ new TEndPoint("127.0.0.1", 6666),
new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
new PlanNodeId("test"));
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FillNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FillNodeSerdeTest.java
index c282d26626..df3a0f3d95 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FillNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FillNodeSerdeTest.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.process;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.AlignedPath;
import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -69,7 +70,8 @@ public class FillNodeSerdeTest {
new SeriesScanNode(
new PlanNodeId("TestSeriesScanNode"),
new AlignedPath("s1"),
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
aggregateNode.addChild(seriesScanNode);
deviceMergeNode.addChildDeviceNode("device", aggregateNode);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNodeSerdeTest.java
index 74f416e2c1..b33fe9088b 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNodeSerdeTest.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.process;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.AlignedPath;
import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -80,7 +81,8 @@ public class FilterNodeSerdeTest {
new SeriesScanNode(
new PlanNodeId("TestSeriesScanNode"),
new AlignedPath("s1"),
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
aggregateNode.addChild(seriesScanNode);
deviceMergeNode.addChildDeviceNode("device", aggregateNode);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNullNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNullNodeSerdeTest.java
index 9740b0d530..f34d1ae761 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNullNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/FilterNullNodeSerdeTest.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.process;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.AlignedPath;
import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -82,7 +83,8 @@ public class FilterNullNodeSerdeTest {
new SeriesScanNode(
new PlanNodeId("TestSeriesScanNode"),
new AlignedPath("s1"),
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
aggregateNode.addChild(seriesScanNode);
deviceMergeNode.addChildDeviceNode("device", aggregateNode);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/GroupByLevelNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/GroupByLevelNodeSerdeTest.java
index cdbd5015cc..8f4dc9a71a 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/GroupByLevelNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/GroupByLevelNodeSerdeTest.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.process;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.AlignedPath;
import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -84,7 +85,8 @@ public class GroupByLevelNodeSerdeTest {
new SeriesScanNode(
new PlanNodeId("TestSeriesScanNode"),
new AlignedPath("s1"),
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
aggregateNode.addChild(seriesScanNode);
deviceMergeNode.addChildDeviceNode("device", aggregateNode);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/LimitNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/LimitNodeSerdeTest.java
index c693aa735e..2e6e84f4b1 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/LimitNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/LimitNodeSerdeTest.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.process;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.AlignedPath;
import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -85,7 +86,8 @@ public class LimitNodeSerdeTest {
new SeriesScanNode(
new PlanNodeId("TestSeriesScanNode"),
new AlignedPath("s1"),
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
aggregateNode.addChild(seriesScanNode);
deviceMergeNode.addChildDeviceNode("device", aggregateNode);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/OffsetNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/OffsetNodeSerdeTest.java
index eb2f482b99..5cfa8dc015 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/OffsetNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/OffsetNodeSerdeTest.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.process;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.AlignedPath;
import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -91,7 +92,8 @@ public class OffsetNodeSerdeTest {
new SeriesScanNode(
new PlanNodeId("TestSeriesScanNode"),
new AlignedPath("s1"),
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
aggregateNode.addChild(seriesScanNode);
deviceMergeNode.addChildDeviceNode("device", aggregateNode);
@@ -161,17 +163,20 @@ public class OffsetNodeSerdeTest {
SeriesScanNode seriesScanNode1 =
new SeriesScanNode(new PlanNodeId("SeriesScanNode1"), new MeasurementPath("root.sg.d1.s2"));
seriesScanNode1.setRegionReplicaSet(
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
seriesScanNode1.setScanOrder(OrderBy.TIMESTAMP_DESC);
SeriesScanNode seriesScanNode2 =
new SeriesScanNode(new PlanNodeId("SeriesScanNode2"), new MeasurementPath("root.sg.d2.s1"));
seriesScanNode2.setRegionReplicaSet(
- new RegionReplicaSet(new DataRegionId(2), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 2), new ArrayList<>()));
seriesScanNode2.setScanOrder(OrderBy.TIMESTAMP_DESC);
SeriesScanNode seriesScanNode3 =
new SeriesScanNode(new PlanNodeId("SeriesScanNode3"), new MeasurementPath("root.sg.d2.s2"));
seriesScanNode3.setRegionReplicaSet(
- new RegionReplicaSet(new DataRegionId(3), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 3), new ArrayList<>()));
seriesScanNode3.setScanOrder(OrderBy.TIMESTAMP_DESC);
// build tree
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/SortNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/SortNodeSerdeTest.java
index d2385f117a..98f733b41d 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/SortNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/SortNodeSerdeTest.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.process;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.AlignedPath;
import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -89,7 +90,8 @@ public class SortNodeSerdeTest {
new SeriesScanNode(
new PlanNodeId("TestSeriesScanNode"),
new AlignedPath("s1"),
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
aggregateNode.addChild(seriesScanNode);
deviceMergeNode.addChildDeviceNode("device", aggregateNode);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/TimeJoinNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/TimeJoinNodeSerdeTest.java
index 9c99507611..c98ddb2ef2 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/TimeJoinNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/process/TimeJoinNodeSerdeTest.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.process;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.AlignedPath;
import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -90,7 +91,8 @@ public class TimeJoinNodeSerdeTest {
new SeriesScanNode(
new PlanNodeId("TestSeriesScanNode"),
new AlignedPath("s1"),
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
aggregateNode.addChild(seriesScanNode);
deviceMergeNode.addChildDeviceNode("device", aggregateNode);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/sink/FragmentSinkNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/sink/FragmentSinkNodeSerdeTest.java
index a2804b8c2e..cde85f4e97 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/sink/FragmentSinkNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/sink/FragmentSinkNodeSerdeTest.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.sink;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.metadata.path.PartialPath;
import org.apache.iotdb.db.mpp.common.FragmentInstanceId;
@@ -44,7 +44,7 @@ public class FragmentSinkNodeSerdeTest {
new DevicesSchemaScanNode(
new PlanNodeId("deviceSchema"), new PartialPath("root.sg.device"), 0, 0, false, false));
fragmentSinkNode.setDownStream(
- new Endpoint("127.0.0.1", 6666),
+ new TEndPoint("127.0.0.1", 6666),
new FragmentInstanceId(new PlanFragmentId("q", 1), "ds"),
new PlanNodeId("test"));
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesAggregateScanNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesAggregateScanNodeSerdeTest.java
index 67fb28feab..258968d6da 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesAggregateScanNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesAggregateScanNodeSerdeTest.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.source;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.metadata.path.MeasurementPath;
@@ -64,7 +65,8 @@ public class SeriesAggregateScanNodeSerdeTest {
new In<String>(st, VALUE_FILTER, true),
groupByTimeComponent);
seriesAggregateScanNode.setRegionReplicaSet(
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
ByteBuffer byteBuffer = ByteBuffer.allocate(2048);
seriesAggregateScanNode.serialize(byteBuffer);
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesScanNodeSerdeTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesScanNodeSerdeTest.java
index 44e93c9629..8ddc7ebf5b 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesScanNodeSerdeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/sql/plan/node/source/SeriesScanNodeSerdeTest.java
@@ -18,8 +18,9 @@
*/
package org.apache.iotdb.db.mpp.sql.plan.node.source;
-import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.db.exception.metadata.IllegalPathException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.metadata.path.AlignedPath;
@@ -44,7 +45,8 @@ public class SeriesScanNodeSerdeTest {
new SeriesScanNode(
new PlanNodeId("TestSeriesScanNode"),
new AlignedPath("s1"),
- new RegionReplicaSet(new DataRegionId(1), new ArrayList<>()));
+ new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new ArrayList<>()));
seriesScanNode.setTimeFilter(new GroupByFilter(1, 2, 3, 4));
seriesScanNode.setScanOrder(OrderBy.TIMESTAMP_ASC);
ByteBuffer byteBuffer = ByteBuffer.allocate(2048);
diff --git a/server/src/test/java/org/apache/iotdb/db/service/InternalServiceImplTest.java b/server/src/test/java/org/apache/iotdb/db/service/InternalServiceImplTest.java
index 5def9d7f05..bf0dbad703 100644
--- a/server/src/test/java/org/apache/iotdb/db/service/InternalServiceImplTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/service/InternalServiceImplTest.java
@@ -19,10 +19,13 @@
package org.apache.iotdb.db.service;
-import org.apache.iotdb.commons.cluster.DataNodeLocation;
-import org.apache.iotdb.commons.cluster.Endpoint;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.commons.consensus.ConsensusGroupId;
import org.apache.iotdb.commons.consensus.SchemaRegionId;
-import org.apache.iotdb.commons.partition.RegionReplicaSet;
import org.apache.iotdb.consensus.common.Peer;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -40,7 +43,6 @@ import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write.CreateAligne
import org.apache.iotdb.db.mpp.sql.planner.plan.node.metedata.write.CreateTimeSeriesNode;
import org.apache.iotdb.db.service.thrift.impl.InternalServiceImpl;
import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.mpp.rpc.thrift.TConsensusGroupId;
import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstance;
import org.apache.iotdb.mpp.rpc.thrift.TSendFragmentInstanceReq;
import org.apache.iotdb.mpp.rpc.thrift.TSendFragmentInstanceResp;
@@ -80,16 +82,20 @@ public class InternalServiceImplTest {
@Before
public void setUp() throws Exception {
- RegionReplicaSet regionReplicaSet = genRegionReplicaSet();
+ TRegionReplicaSet regionReplicaSet = genRegionReplicaSet();
ConsensusImpl.getInstance()
- .addConsensusGroup(regionReplicaSet.getConsensusGroupId(), genPeerList(regionReplicaSet));
+ .addConsensusGroup(
+ ConsensusGroupId.Factory.convertFromTConsensusGroupId(regionReplicaSet.getRegionId()),
+ genPeerList(regionReplicaSet));
internalServiceImpl = new InternalServiceImpl();
}
@After
public void tearDown() throws Exception {
- RegionReplicaSet regionReplicaSet = genRegionReplicaSet();
- ConsensusImpl.getInstance().removeConsensusGroup(regionReplicaSet.getConsensusGroupId());
+ TRegionReplicaSet regionReplicaSet = genRegionReplicaSet();
+ ConsensusImpl.getInstance()
+ .removeConsensusGroup(
+ ConsensusGroupId.Factory.convertFromTConsensusGroupId(regionReplicaSet.getRegionId()));
FileUtils.deleteFully(new File(conf.getConsensusDir()));
}
@@ -128,7 +134,7 @@ public class InternalServiceImplTest {
},
"meter1");
- RegionReplicaSet regionReplicaSet = genRegionReplicaSet();
+ TRegionReplicaSet regionReplicaSet = genRegionReplicaSet();
PlanFragment planFragment = new PlanFragment(new PlanFragmentId("2", 3), createTimeSeriesNode);
FragmentInstance fragmentInstance =
new FragmentInstance(
@@ -148,10 +154,7 @@ public class InternalServiceImplTest {
TFragmentInstance tFragmentInstance = new TFragmentInstance();
tFragmentInstance.setBody(byteBuffer);
request.setFragmentInstance(tFragmentInstance);
- request.setConsensusGroupId(
- new TConsensusGroupId(
- regionReplicaSet.getConsensusGroupId().getId(),
- regionReplicaSet.getConsensusGroupId().getType().toString()));
+ request.setConsensusGroupId(regionReplicaSet.getRegionId());
request.setQueryType(QueryType.WRITE.toString());
// Use consensus layer to execute request
@@ -219,7 +222,7 @@ public class InternalServiceImplTest {
}
});
- RegionReplicaSet regionReplicaSet = genRegionReplicaSet();
+ TRegionReplicaSet regionReplicaSet = genRegionReplicaSet();
PlanFragment planFragment =
new PlanFragment(new PlanFragmentId("2", 3), createAlignedTimeSeriesNode);
FragmentInstance fragmentInstance =
@@ -240,10 +243,7 @@ public class InternalServiceImplTest {
TFragmentInstance tFragmentInstance = new TFragmentInstance();
tFragmentInstance.setBody(byteBuffer);
request.setFragmentInstance(tFragmentInstance);
- request.setConsensusGroupId(
- new TConsensusGroupId(
- regionReplicaSet.getConsensusGroupId().getId(),
- regionReplicaSet.getConsensusGroupId().getType().toString()));
+ request.setConsensusGroupId(regionReplicaSet.getRegionId());
request.setQueryType(QueryType.WRITE.toString());
// Use consensus layer to execute request
@@ -252,21 +252,24 @@ public class InternalServiceImplTest {
Assert.assertTrue(response.accepted);
}
- private RegionReplicaSet genRegionReplicaSet() {
- List<DataNodeLocation> dataNodeList = new ArrayList<>();
+ private TRegionReplicaSet genRegionReplicaSet() {
+ List<TDataNodeLocation> dataNodeList = new ArrayList<>();
dataNodeList.add(
- new DataNodeLocation(
- conf.getConsensusPort(), new Endpoint(conf.getInternalIp(), conf.getConsensusPort())));
+ new TDataNodeLocation()
+ .setConsensusEndPoint(new TEndPoint(conf.getInternalIp(), conf.getConsensusPort())));
// construct fragmentInstance
- SchemaRegionId schemaRegionId = new SchemaRegionId(0);
- return new RegionReplicaSet(schemaRegionId, dataNodeList);
+ return new TRegionReplicaSet(
+ new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 0), dataNodeList);
}
- private List<Peer> genPeerList(RegionReplicaSet regionReplicaSet) {
+ private List<Peer> genPeerList(TRegionReplicaSet regionReplicaSet) {
List<Peer> peerList = new ArrayList<>();
- for (DataNodeLocation node : regionReplicaSet.getDataNodeList()) {
- peerList.add(new Peer(regionReplicaSet.getConsensusGroupId(), node.getEndPoint()));
+ for (TDataNodeLocation node : regionReplicaSet.getDataNodeLocations()) {
+ peerList.add(
+ new Peer(
+ new SchemaRegionId(regionReplicaSet.getRegionId().getId()),
+ node.getConsensusEndPoint()));
}
return peerList;
}
diff --git a/thrift-cluster/pom.xml b/thrift-cluster/pom.xml
index 236417bd87..ff6aeca0ed 100644
--- a/thrift-cluster/pom.xml
+++ b/thrift-cluster/pom.xml
@@ -40,6 +40,11 @@
<artifactId>iotdb-thrift</artifactId>
<version>${project.version}</version>
</dependency>
+ <dependency>
+ <groupId>org.apache.iotdb</groupId>
+ <artifactId>iotdb-thrift-commons</artifactId>
+ <version>${project.version}</version>
+ </dependency>
</dependencies>
<build>
<plugins>
diff --git a/thrift-cluster/pom.xml b/thrift-commons/pom.xml
similarity index 87%
copy from thrift-cluster/pom.xml
copy to thrift-commons/pom.xml
index 236417bd87..942b35df8a 100644
--- a/thrift-cluster/pom.xml
+++ b/thrift-commons/pom.xml
@@ -27,19 +27,14 @@
<version>0.14.0-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
- <artifactId>iotdb-thrift-cluster</artifactId>
- <name>rpc-thrift-cluster</name>
- <description>RPC (Thrift) framework among servers.</description>
+ <artifactId>iotdb-thrift-commons</artifactId>
+ <name>rpc-thrift-commons</name>
+ <description>RPC (Thrift) common framework.</description>
<dependencies>
<dependency>
<groupId>org.apache.thrift</groupId>
<artifactId>libthrift</artifactId>
</dependency>
- <dependency>
- <groupId>org.apache.iotdb</groupId>
- <artifactId>iotdb-thrift</artifactId>
- <version>${project.version}</version>
- </dependency>
</dependencies>
<build>
<plugins>
diff --git a/thrift/src/main/thrift/common.thrift b/thrift-commons/src/main/thrift/common.thrift
similarity index 65%
rename from thrift/src/main/thrift/common.thrift
rename to thrift-commons/src/main/thrift/common.thrift
index 0d20e7d6f3..9b09bc3743 100644
--- a/thrift/src/main/thrift/common.thrift
+++ b/thrift-commons/src/main/thrift/common.thrift
@@ -33,9 +33,15 @@ struct TSStatus {
4: optional TEndPoint redirectNode
}
-struct TRegionReplicaSet {
- 1: required binary regionId
- 2: required list<TEndPoint> endpoint
+enum TConsensusGroupType {
+ PartitionRegion,
+ DataRegion,
+ SchemaRegion
+}
+
+struct TConsensusGroupId {
+ 1: required TConsensusGroupType type
+ 2: required i32 id
}
struct TSeriesPartitionSlot {
@@ -44,4 +50,21 @@ struct TSeriesPartitionSlot {
struct TTimePartitionSlot {
1: required i64 startTime
+}
+
+struct TRegionReplicaSet {
+ 1: required TConsensusGroupId regionId
+ 2: required list<TDataNodeLocation> dataNodeLocations
+}
+
+struct TDataNodeLocation {
+ 1: required i32 dataNodeId
+ // TEndPoint for DataNode's external rpc
+ 2: required TEndPoint externalEndPoint
+ // TEndPoint for DataNode's internal rpc
+ 3: required TEndPoint internalEndPoint
+ // TEndPoint for transfering data between DataNodes
+ 4: required TEndPoint dataBlockManagerEndPoint
+ // TEndPoint for DataNode's ConsensusLayer
+ 5: required TEndPoint consensusEndPoint
}
\ No newline at end of file
diff --git a/thrift-confignode/pom.xml b/thrift-confignode/pom.xml
index a3501f8095..fb3f5e4467 100644
--- a/thrift-confignode/pom.xml
+++ b/thrift-confignode/pom.xml
@@ -37,7 +37,7 @@
</dependency>
<dependency>
<groupId>org.apache.iotdb</groupId>
- <artifactId>iotdb-thrift</artifactId>
+ <artifactId>iotdb-thrift-commons</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
diff --git a/thrift-confignode/src/main/thrift/confignode.thrift b/thrift-confignode/src/main/thrift/confignode.thrift
index ac4d8d9d96..7ac920f073 100644
--- a/thrift-confignode/src/main/thrift/confignode.thrift
+++ b/thrift-confignode/src/main/thrift/confignode.thrift
@@ -23,7 +23,7 @@ namespace py iotdb.thrift.confignode
// DataNode
struct TDataNodeRegisterReq {
- 1: required common.TEndPoint endPoint
+ 1: required common.TDataNodeLocation dataNodeLocation
// Map<StorageGroupName, TStorageGroupSchema>
// DataNode can use statusMap to report its status to the ConfigNode when restart
2: optional map<string, TStorageGroupSchema> statusMap
@@ -38,25 +38,19 @@ struct TGlobalConfig {
struct TDataNodeRegisterResp {
1: required common.TSStatus status
- 2: optional i32 dataNodeID
+ 2: optional i32 dataNodeId
3: optional TGlobalConfig globalConfig
}
-struct TDataNodeMessageResp {
+struct TDataNodeLocationResp {
1: required common.TSStatus status
- // map<DataNodeId, DataNodeMessage>
- 2: optional map<i32, TDataNodeMessage> dataNodeMessageMap
-}
-
-struct TDataNodeMessage {
- 1: required i32 dataNodeId
- 2: required common.TEndPoint endPoint
+ // map<DataNodeId, DataNodeLocation>
+ 2: optional map<i32, common.TDataNodeLocation> dataNodeLocationMap
}
// StorageGroup
struct TSetStorageGroupReq {
- 1: required string storageGroup
- 2: optional i64 TTL
+ 1: required TStorageGroupSchema storageGroup
}
struct TDeleteStorageGroupReq {
@@ -68,6 +62,11 @@ struct TSetTTLReq {
2: required i64 TTL
}
+struct TSetTimePartitionIntervalReq {
+ 1: required string storageGroup
+ 2: required i64 timePartitionInterval
+}
+
struct TStorageGroupSchemaResp {
1: required common.TSStatus status
// map<string, StorageGroupMessage>
@@ -75,12 +74,13 @@ struct TStorageGroupSchemaResp {
}
struct TStorageGroupSchema {
- 1: required string storageGroup
+ 1: required string name
2: optional i64 TTL
- // list<DataRegionId>
- 3: optional list<binary> dataRegionGroupIds
- // list<SchemaRegionId>
- 4: optional list<binary> schemaRegionGroupIds
+ 3: optional i32 schemaReplicationFactor
+ 4: optional i32 dataReplicationFactor
+ 5: optional i64 timePartitionInterval
+ 6: optional list<common.TConsensusGroupId> dataRegionGroupIds
+ 7: optional list<common.TConsensusGroupId> schemaRegionGroupIds
}
// Schema
@@ -128,7 +128,7 @@ service ConfigIService {
TDataNodeRegisterResp registerDataNode(TDataNodeRegisterReq req)
- TDataNodeMessageResp getDataNodesMessage(i32 dataNodeID)
+ TDataNodeLocationResp getDataNodeLocations(i32 dataNodeId)
/* StorageGroup */
@@ -138,6 +138,8 @@ service ConfigIService {
common.TSStatus setTTL(TSetTTLReq req)
+ common.TSStatus setTimePartitionInterval(TSetTimePartitionIntervalReq req)
+
TStorageGroupSchemaResp getStorageGroupsSchema()
/* Schema */
diff --git a/thrift/pom.xml b/thrift/pom.xml
index 59adc6b3f9..92306b53a1 100644
--- a/thrift/pom.xml
+++ b/thrift/pom.xml
@@ -35,6 +35,11 @@
<groupId>org.apache.thrift</groupId>
<artifactId>libthrift</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.iotdb</groupId>
+ <artifactId>iotdb-thrift-commons</artifactId>
+ <version>${project.version}</version>
+ </dependency>
</dependencies>
<build>
<plugins>
diff --git a/thrift/src/main/thrift/mpp.thrift b/thrift/src/main/thrift/mpp.thrift
index d7339e4100..0c2acf6cb2 100644
--- a/thrift/src/main/thrift/mpp.thrift
+++ b/thrift/src/main/thrift/mpp.thrift
@@ -83,14 +83,9 @@ struct TFragmentInstance {
1: required binary body
}
-struct TConsensusGroupId {
- 1: required i32 id
- 2: required string type
-}
-
struct TSendFragmentInstanceReq {
1: required TFragmentInstance fragmentInstance
- 2: required TConsensusGroupId consensusGroupId
+ 2: required common.TConsensusGroupId consensusGroupId
3: required string queryType
}