You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ta...@apache.org on 2022/03/08 03:07:57 UTC

[iotdb] branch new_cluster updated: ConfigNode basic framework (#5174)

This is an automated email from the ASF dual-hosted git repository.

tanxinyu pushed a commit to branch new_cluster
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/new_cluster by this push:
     new c4e385d  ConfigNode basic framework (#5174)
c4e385d is described below

commit c4e385dd71dfb62cfec53da5dc8c8095db294fdf
Author: CRZbulabula <33...@users.noreply.github.com>
AuthorDate: Tue Mar 8 10:55:04 2022 +0800

    ConfigNode basic framework (#5174)
    
    Co-authored-by: CRZbulabula <cr...@gmail.com>
---
 confignode/pom.xml                                 | 49 +++++++++++++
 .../iotdb/confignode/manager/ConfigManager.java    | 54 +++++++++++++++
 .../confignode/partition/DataPartitionRule.java    | 50 +++++++++++++
 .../iotdb/confignode/partition/PartitionTable.java | 59 ++++++++++++++++
 .../confignode/service/balancer/LoadBalancer.java  | 49 +++++++++++++
 .../service/basic/ConfigServiceProvider.java       | 45 ++++++++++++
 .../service/thrift/impl/ConfigServiceImpl.java     | 81 ++++++++++++++++++++++
 pom.xml                                            |  3 +
 thrift-confignode/pom.xml                          | 67 ++++++++++++++++++
 .../src/main/thrift/confignode.thrift              | 38 ++++++++++
 10 files changed, 495 insertions(+)

diff --git a/confignode/pom.xml b/confignode/pom.xml
new file mode 100644
index 0000000..bfc5af2
--- /dev/null
+++ b/confignode/pom.xml
@@ -0,0 +1,49 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>iotdb-parent</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <artifactId>iotdb-confignode</artifactId>
+    <name>IoTDB ConfigNode</name>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>service-rpc</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-server</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift-confignode</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
+</project>
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
new file mode 100644
index 0000000..21d29b9
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.confignode.manager;
+
+import org.apache.iotdb.confignode.partition.PartitionTable;
+import org.apache.iotdb.confignode.service.balancer.LoadBalancer;
+
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * ConfigManager Maintains consistency between PartitionTables in the ConfigNodeGroup. Expose the
+ * query interface for the PartitionTable
+ */
+public class ConfigManager {
+
+  private final Lock partitionTableLock;
+  private final PartitionTable partitionTable;
+
+  private final LoadBalancer loadBalancer;
+
+  public ConfigManager() {
+    this.partitionTableLock = new ReentrantLock();
+    this.partitionTable = new PartitionTable();
+
+    this.loadBalancer = new LoadBalancer(partitionTableLock, partitionTable);
+  }
+
+  public int getDeviceGroupID(String device) {
+    return -1;
+  }
+
+  // TODO: Interfaces for metadata operations
+
+  // TODO: Interfaces for data operations
+
+  // TODO: Interfaces for LoadBalancer control
+}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/partition/DataPartitionRule.java b/confignode/src/main/java/org/apache/iotdb/confignode/partition/DataPartitionRule.java
new file mode 100644
index 0000000..bc9132e
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/partition/DataPartitionRule.java
@@ -0,0 +1,50 @@
+/*
+ * 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.tsfile.utils.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * DataPartitionRule is used to hold real-time write-load allocation rules i.e. rules = [(0, 0.3),
+ * (1, 0.2), (2. 0.5)] means allocate 30% of the write-load to VSGGroup-0 and 20% to vsgGroup-1 and
+ * 50% to VSGGroup-2
+ */
+public class DataPartitionRule {
+  // List<Pair<VSGGroupID, Write allocation ratio>>
+  private final List<Pair<Integer, Double>> rules;
+
+  public DataPartitionRule() {
+    this.rules = new ArrayList<>();
+  }
+
+  public DataPartitionRule(List<Pair<Integer, Double>> rules) {
+    this.rules = rules;
+  }
+
+  public void addDataPartitionRule(int vsgGroupID, double ratio) {
+    this.rules.add(new Pair<>(vsgGroupID, ratio));
+  }
+
+  public List<Pair<Integer, Double>> getDataPartitionRule() {
+    return this.rules;
+  }
+}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/partition/PartitionTable.java b/confignode/src/main/java/org/apache/iotdb/confignode/partition/PartitionTable.java
new file mode 100644
index 0000000..f6a6ec3
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/partition/PartitionTable.java
@@ -0,0 +1,59 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * PartitionTable stores metadata partition table, data partition table, and real-time write load
+ * allocation rules
+ */
+public class PartitionTable {
+
+  // Map<StorageGroup, Map<DeviceGroupID, MManagerGroupID>>
+  private final Map<String, Map<Integer, Integer>> metadataPartitionTable;
+  // Map<MManagerGroupID, List<PhysicalNodeIDs>>
+  private final Map<Integer, List<Integer>> mManagerGroupPhysicalNodesMap;
+
+  // Map<StorageGroup, Map<DeviceGroupID, Map<TimeInterval, List<VSGGroupID>>>>
+  private final Map<String, Map<Integer, Map<Long, List<Integer>>>> dataPartitionTable;
+  // Map<VSGGroupID, List<PhysicalNodeIDs>>
+  private final Map<Integer, List<Integer>> vsgGroupPhysicalNodesMap;
+
+  // Map<StorageGroup, Map<DeviceGroupID, DataPartitionRule>>
+  private final Map<String, Map<Integer, DataPartitionRule>> dataPartitionRuleTable;
+
+  public PartitionTable() {
+    this.metadataPartitionTable = new HashMap<>();
+    this.mManagerGroupPhysicalNodesMap = new HashMap<>();
+
+    this.dataPartitionTable = new HashMap<>();
+    this.vsgGroupPhysicalNodesMap = new HashMap<>();
+
+    this.dataPartitionRuleTable = new HashMap<>();
+  }
+
+  // TODO: Interfaces for metadata operations
+
+  // TODO: Interfaces for data operations
+
+  // TODO: Interfaces for data partition rules
+}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/balancer/LoadBalancer.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/balancer/LoadBalancer.java
new file mode 100644
index 0000000..4cfb8b3
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/balancer/LoadBalancer.java
@@ -0,0 +1,49 @@
+/*
+ * 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.service.balancer;
+
+import org.apache.iotdb.confignode.partition.PartitionTable;
+
+import java.util.concurrent.locks.Lock;
+
+/**
+ * The LoadBalancer at ConfigNodeGroup-Leader is active for cluster dynamic load balancing
+ * scheduling
+ */
+public class LoadBalancer implements Runnable {
+
+  private final Lock partitionTableLock;
+  private final PartitionTable partitionTable;
+
+  public LoadBalancer(Lock partitionTableLock, PartitionTable partitionTable) {
+    this.partitionTableLock = partitionTableLock;
+    this.partitionTable = partitionTable;
+  }
+
+  @Override
+  public void run() {}
+
+  private void metadataLoadBalance() {}
+
+  private void dataLoadBalance() {}
+
+  private void cleanDataSlice() {}
+
+  // TODO: Interfaces for active, interrupt and reset LoadBalancer
+}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/basic/ConfigServiceProvider.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/basic/ConfigServiceProvider.java
new file mode 100644
index 0000000..fca11e8
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/basic/ConfigServiceProvider.java
@@ -0,0 +1,45 @@
+/*
+ * 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.service.basic;
+
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.executor.PlanExecutor;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.service.basic.ServiceProvider;
+
+public class ConfigServiceProvider extends ServiceProvider {
+  public ConfigServiceProvider(PlanExecutor executor) throws QueryProcessException {
+    super(executor);
+  }
+
+  @Override
+  public QueryContext genQueryContext(
+      long queryId, boolean debug, long startTime, String statement, long timeout) {
+    return null;
+  }
+
+  @Override
+  public boolean executeNonQuery(PhysicalPlan plan)
+      throws QueryProcessException, StorageGroupNotSetException, StorageEngineException {
+    return false;
+  }
+}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/impl/ConfigServiceImpl.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/impl/ConfigServiceImpl.java
new file mode 100644
index 0000000..76c3c1e
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/impl/ConfigServiceImpl.java
@@ -0,0 +1,81 @@
+/*
+ * 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.service.thrift.impl;
+
+import org.apache.iotdb.confignode.manager.ConfigManager;
+import org.apache.iotdb.confignode.rpc.thrift.ConfigIService;
+import org.apache.iotdb.confignode.service.basic.ConfigServiceProvider;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.service.rpc.thrift.TSCreateAlignedTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateMultiTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSCreateTimeseriesReq;
+import org.apache.iotdb.service.rpc.thrift.TSStatus;
+
+import org.apache.thrift.TException;
+
+import java.util.List;
+
+/** ConfigServiceImpl exposes the interface that interacts with the DataNode */
+public class ConfigServiceImpl implements ConfigIService.Iface {
+
+  private ConfigServiceProvider configServiceProvider;
+  private ConfigManager configManager;
+
+  public ConfigServiceImpl() throws QueryProcessException {
+    this.configServiceProvider = new ConfigServiceProvider(null);
+    this.configManager = new ConfigManager();
+  }
+
+  @Override
+  public TSStatus setStorageGroup(long sessionId, String storageGroup) throws TException {
+    return null;
+  }
+
+  @Override
+  public TSStatus deleteStorageGroup(long sessionId, List<String> storageGroups) throws TException {
+    return null;
+  }
+
+  @Override
+  public TSStatus createTimeseries(TSCreateTimeseriesReq req) throws TException {
+    return null;
+  }
+
+  @Override
+  public TSStatus createAlignedTimeseries(TSCreateAlignedTimeseriesReq req) throws TException {
+    return null;
+  }
+
+  @Override
+  public TSStatus createMultiTimeseries(TSCreateMultiTimeseriesReq req) throws TException {
+    return null;
+  }
+
+  @Override
+  public TSStatus deleteTimeSeries(long sessionId, List<String> paths) throws TException {
+    return null;
+  }
+
+  @Override
+  public int getDeviceGroupID(long sessionId, String device) throws TException {
+    return -1;
+  }
+
+  // TODO: Interfaces for data operations
+}
diff --git a/pom.xml b/pom.xml
index a358dcb..e560ec2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -82,6 +82,7 @@
         <module>tsfile</module>
         <module>antlr</module>
         <module>thrift</module>
+        <module>thrift-confignode</module>
         <module>thrift-cluster</module>
         <module>thrift-sync</module>
         <module>thrift-influxdb</module>
@@ -109,6 +110,7 @@
         <module>client-cpp</module>
         <module>metrics</module>
         <module>integration</module>
+        <module>confignode</module>
         <!--        <module>library-udf</module>-->
     </modules>
     <!-- Properties Management -->
@@ -735,6 +737,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-confignode/target/generated-sources/thrift</sourceDirectory>
                         <sourceDirectory>thrift-sync/target/generated-sources/thrift</sourceDirectory>
                         <sourceDirectory>thrift-cluster/target/generated-sources/thrift</sourceDirectory>
                         <sourceDirectory>thrift-influxdb/target/generated-sources/thrift</sourceDirectory>
diff --git a/thrift-confignode/pom.xml b/thrift-confignode/pom.xml
new file mode 100644
index 0000000..bdae393
--- /dev/null
+++ b/thrift-confignode/pom.xml
@@ -0,0 +1,67 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>iotdb-parent</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <artifactId>iotdb-thrift-confignode</artifactId>
+    <name>ConfigNode RPC thrift</name>
+    <description>RPC (Thrift) framework among ConfigNodes.</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>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>3.2.0</version>
+                <executions>
+                    <execution>
+                        <id>add-source</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>${project.build.directory}/generated-sources/thrift</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>
diff --git a/thrift-confignode/src/main/thrift/confignode.thrift b/thrift-confignode/src/main/thrift/confignode.thrift
new file mode 100644
index 0000000..ba71a0e
--- /dev/null
+++ b/thrift-confignode/src/main/thrift/confignode.thrift
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+include "rpc.thrift"
+namespace java org.apache.iotdb.confignode.rpc.thrift
+namespace py iotdb.thrift.confignode
+
+service ConfigIService {
+  rpc.TSStatus setStorageGroup(1:i64 sessionId, 2:string storageGroup)
+
+  rpc.TSStatus deleteStorageGroup(1:i64 sessionId, 2:list<string> storageGroups)
+
+  rpc.TSStatus createTimeseries(1:rpc.TSCreateTimeseriesReq req)
+
+  rpc.TSStatus createAlignedTimeseries(1:rpc.TSCreateAlignedTimeseriesReq req)
+
+  rpc.TSStatus createMultiTimeseries(1:rpc.TSCreateMultiTimeseriesReq req)
+
+  rpc.TSStatus deleteTimeSeries(1:i64 sessionId, 2:list<string> paths)
+
+  i32 getDeviceGroupID(1:i64 sessionId, 2:string device)
+}
\ No newline at end of file