You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2021/07/28 10:45:59 UTC

[GitHub] [iotdb] CloudWise-Lukemiao opened a new pull request #3644: iotdb add openapi

CloudWise-Lukemiao opened a new pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644


   iotdb add  OpenAPI module


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] qiaojialin commented on a change in pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#discussion_r686562968



##########
File path: docs/zh/UserGuide/Communication-Service-Protocol/OpenApi.md
##########
@@ -0,0 +1,442 @@
+<!--
+
+    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.
+
+-->
+
+# Rest API
+IoTDB的restApi设计用于支持与Grafana和Prometheus的集成同时也提供了查询、插入和non-query接口,它使用OpenAPI标准来定义接口和生成框架源代码。
+为了IoTDB数据库安全我们建议使用一台非IoTDB的服务器安装反向代理服务(例如nginx等)把http请求转发到IoTDB,当然你可以不使用反向代理服务直接使用openapi的rest,如果你正在使用grafana服务同时使用nginx则需要在nginx.conf添加 add_header 'Access-Control-Max-Age' xx 来保证可以正常使用
+OpenApi 接口使用了基础(basic)鉴权,每次url请求都需要在header中携带 'Authorization': 'Basic ' + base64.encode(username + ':' + password)例如:
+
+```
+location /rest/ {
+   proxy_pass  http://ip:port/rest/;  
+   add_header 'Access-Control-Max-Age' 20;
+}
+```
+
+### Configuration
+配置位于“iotdb-engines.properties”中,将“enable_openApi”设置为“true”以启用该模块,而将“false”设置为禁用该模块。
+默认情况下,该值为“false”。
+```
+enable_openApi=true
+```
+
+仅在“enable_openApi=true”时生效。将“openApi_port”设置为数字(1025~65535),以自定义rest服务套接字端口。
+默认情况下,值为“18080”。
+
+```
+openApi_port=18080
+```
+
+设置Prometheus数据存储时的存储组数量
+
+```
+sg_count=5
+```
+
+openApi 开启ssl配置,将“enable_https”设置为“true”以启用该模块,而将“false”设置为禁用该模块。
+默认情况下,该值为“false”。
+
+```
+enable_https=false
+```
+
+keyStore所在路径
+
+```
+key_store_path=/xxx/xxx.keystore
+```
+keystore密码
+
+```
+key_store_pwd=xxxx
+```
+trustStore所在路径(非必填)
+
+```
+trust_store_path=xxxx
+```
+
+trustStore密码
+```
+trust_store_pwd=xxxx
+```
+ssl 超时时间单位为秒
+
+```
+idle_timeout=5000
+```
+
+## grafana接口
+
+## 检查iotdb服务是否在运行
+请求方式:get
+请求url:http://ip:port/ping
+```
+$ curl -H "Authorization:Basic cm9vdDpyb2901" http://127.0.0.1:18080/ping
+$ {"code":4,"type":"ok","message":"login success!"}
+```
+响应示例
+```json
+{
+"code": 4,
+"type": "ok",
+"message": "login success!"
+}
+```
+用户名密码认证失败示例
+```json
+{
+  "code": 1,
+  "type": "error",
+  "message": "username or passowrd is incorrect!"
+}
+```
+
+##用于通过Grafana逐级获取时间序列名称
+请求方式:post
+请求头:application/json
+请求url:http://ip:port/rest/grafana/node
+```
+$ curl -H "Content-Type:application/json" -H "Authorization:Basic cm9vdDpyb290" -X POST --data '["root","sg5"]' http://127.0.0.1:18080/rest/grafana/node
+$ ["wf01","wf02","wf03"]
+```
+请求示例:
+```json
+["root","sg5"]
+```
+
+响应示例:
+```json
+["wf01","wf02","wf03"]
+```
+
+##为Grafana提供自动降采样数据查询
+请求方式:post
+请求头:application/json
+请求url:http://ip:port/rest/grafana/query/json
+```
+$ curl -H "Content-Type:application/json" -H "Authorization:Basic cm9vdDpyb290" -X POST --data '{"paths":["root","ln","wf02"],"aggregation":"AVG","groupBy":{"samplingInterval":"1s","step":"1s"},"stime":1627286097811,"etime":1627286397811}' http://127.0.0.1:18080/rest/grafana/query/json
+$ [{"datapoints":[[1.2,1627285095273],[0.75,1627285096273],[0.45,1627285097273],[0.15,1627285098273],[0.9,1627285099273],[0.0,1627285100273],[0.15,1627285101273]],"target":"root.ln.wf02"}]
+```
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  interval | string | 是  |  间隔 |
+| stime  |  number |  是 |  开始时间(时间戳) |
+| etime | number|  是 |  结束时间(时间戳) |
+| paths  |  array|  是 |  timeseries 为root.sg 转换成path为["root","sg"] |
+| aggregation  |  string | 否  | 函数 |
+| groupBy  |  object | 否  | 分组 |
+| samplingInterval  |  string | 否  | 降采样间隔 |
+| step  |  string | 否  | 降采样步长 |
+| limitAll  |  object | 否  | 限制 |
+| slimit  |  string | 否(默认值为10)  |  列数 |
+| limit  |  string | 否  |  行数 |
+| fills  |  array | 否  |  填充 |
+| dataType  |  string | 否  |  填充函数 |
+| previous  |  string |  否 |  填充类型 |
+| duration  |  string |  否 |  时间范围 |
+请求示例:
+```json
+{"paths":["root","ln"],"limitAll":{"slimit":"1","limit":""},"aggregation":"AVG","groupBy":{"samplingInterval":"1s","step":"1s"},"stime":1627286097811,"etime":1627286397811}
+```
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------ |
+|  datapoints |  array | 返回数组第一个为值,第二个为时间,依次类推  |
+| target  |  string |  返回查询timeseries |
+
+响应示例:
+```json
+[
+  {
+    "datapoints":[
+      [
+        0.8999999761581421,
+        1627286097811
+      ],
+      [
+        0.75,
+        1627286098811
+      ],
+      [
+        0.75,
+        1627286099811
+      ],
+      [
+        1.0499999523162842,
+        1627286100811
+      ],
+      [
+        1.0499999523162842,
+        1627286101811
+      ],
+      [
+        0.6000000238418579,
+        1627286102811
+      ]
+    ],
+    "target":"root.ln.wf03"
+  }
+]
+```
+##为Grafana提供自动降采样数据查询(DataFrame)
+请求方式:post
+请求头:application/json
+请求url:http://ip:port/rest/grafana/query/frame
+```
+$ curl -H "Content-Type:application/json" -H "Authorization:Basic cm9vdDpyb290" -X POST --data '{"interval":"1s","stime":"1616554359000","etime":"1616554369000","paths":["root","sg7"]}' http://127.0.0.1:18080/rest/grafana/query/frame
+$ [{"values":[1616554359000,1616554360000,1616554361000,1616554362000,1616554363000,1616554364000,1616554365000,1616554366000,1616554367000,1616554368000],"name":"Time","type":"time"},{"values":[5.0,7.0,7.0,null,7.0,7.0,null,null,null],"name":"root.sg7.val02","type":"DOUBLE"},{"values":[5.0,null,null,null,null,null,null,null,null],"name":"root.sg7.val03","type":"DOUBLE"}]
+```
+
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  interval | string | 是  |  间隔 |
+| stime  |  number |  是 |  开始时间(时间戳) |
+| etime | number|  是 |  结束时间(时间戳) |
+| paths  |  array|  是 |  timeseries 为root.sg 转换成path为["root","sg"] |
+| aggregation  |  string | 否  | 函数 |
+| groupBy  |  object | 否  | 分组 |
+| samplingInterval  |  string | 否  | 降采样间隔 |
+| step  |  string | 否  | 降采样步长 |
+| limitAll  |  object | 否  | 限制 |
+| slimit  |  string | 否(默认值为10)  |  列数 |
+| limit  |  string | 否  |  行数 |
+| fills  |  array | 否  |  填充 |
+| dataType  |  string | 否  |  填充函数 |
+| previous  |  string |  否 |  填充类型 |
+| duration  |  string |  否 |  时间范围 |
+请求示例:
+```json
+{"paths":["root","ln"],"limitAll":{"slimit":"1","limit":""},"aggregation":"AVG","groupBy":{"samplingInterval":"1s","step":"1s"},"stime":1627286097811,"etime":1627286397811}
+```
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------ |
+|  name |  String | 字段名称  |
+| type  |  string |  字段类型|
+| values  |  array |  字段值|
+
+响应示例:
+```json
+[
+  {
+    "values":[
+      1616554359000,
+      1616554360000,
+      1616554361000,
+      1616554362000,
+      1616554363000,
+      1616554364000,
+      1616554365000,
+      1616554366000,
+      1616554367000,
+      1616554368000
+    ],
+    "name":"Time",
+    "type":"INT64"
+  },
+  {
+    "values":[
+      5,
+      7,
+      7,
+      null,
+      7,
+      7,
+      null,
+      null,
+      null
+    ],
+    "name":"root.sg7.val02",
+    "type":"DOUBLE"
+  },
+  {
+    "values":[
+      5,
+      null,
+      null,
+      null,
+      null,
+      null,
+      null,
+      null,
+      null
+    ],
+    "name":"root.sg7.val03",
+    "type":"DOUBLE"
+  }
+]
+```
+## prometheus接口
+
+### prometheus wirte接口
+prometheus 数据经过Protobuf (3.12.3)编码和snappy 压缩后传输
+
+请求方式:post
+请求头:application/x-protobuf
+请求url:http://ip:port/rest/prometheus/write
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  Timeseries | array | 是  |   |
+| Labels  |  array |  是 |  tag信息 |
+|  Name | String|  是 |  tagKey |
+| Value  |  array|  是 | tagValue |
+| Samples  |  array | 是  |   |
+| Timestamp  |  number |  是 |  时间 |
+| Value  |  number |  是 |  值 |
+
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------ |
+|  code |  number | 200 成功,500 失败,401没有权限  |
+| message  |  string | 说明|
+
+
+响应示例:
+```json
+{"code": 200,
+"message": "write data success"
+}
+```
+
+### prometheus read 接口
+prometheus 数据经过Protobuf (3.12.3)编码和snappy 压缩后传输
+
+请求方式:post
+请求头:application/x-protobuf
+请求url:http://ip:port/rest/prometheus/query
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  Queries | array | 是  |   |
+| StartTimestampMs  |  number |  是 |  开始时间 |
+| EndTimestampMs | number|  是 |  结束时间 |
+| Matchers  |  array|  是 |  |
+| Name  |  String |  是 |  名称 |
+| Value  |  String |  是 |  值 |
+| Hints  |  array|  是 |  |
+| StepMs  |  number |  否 |  时间间隔 |
+| Func  |  String |  否 |  函数 |
+| StartMs  |  number |  是 |  开始时间 |
+| EndMs  |  number |  否 |  结束时间 |
+| Grouping  |  String |  否 |  聚合字段 |
+| By  |  boolean |  否 |  是否启用group by |
+| RangeMs  |  number |  否 |  滑动步长 |
+
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------|
+|  Timeseries | array |   |
+| Labels  |  array | tag信息 |
+|  Name | String|  tagKey |
+| Value  |  array| tagValue |
+| Samples  |  array |   |
+| Timestamp  |  number|  时间 |
+| Value  |  number | 值 |
+
+## 其他接口
+
+###  read 接口
+
+请求方式:post
+请求头:application/json
+请求url:http://ip:port/rest/read
+```
+$ curl -H "Content-Type:application/json" -H "Authorization:Basic cm9vdDpyb290" -X POST --data '{"sql":"select * from root limit 1 slimit 2"}' http://127.0.0.1:18080/rest/read
+$ [{"values":[1],"name":"Time","type":"INT64"},{"values":[1.1],"name":"root.ln.wf02","type":""},{"values":[2.0],"name":"root.ln.wf03","type":""}]
+```
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  sql | string | 是  |   |
+
+
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------|
+| values | array |  值 |
+| name  |  string | 测点名称 |
+| type | String| 数据类型 |
+
+###  nonQuery 接口

Review comment:
       ```suggestion
   ###  SQL 非查询接口
   ```

##########
File path: docs/zh/UserGuide/Communication-Service-Protocol/OpenApi.md
##########
@@ -0,0 +1,442 @@
+<!--
+
+    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.
+
+-->
+
+# Rest API
+IoTDB的restApi设计用于支持与Grafana和Prometheus的集成同时也提供了查询、插入和non-query接口,它使用OpenAPI标准来定义接口和生成框架源代码。
+为了IoTDB数据库安全我们建议使用一台非IoTDB的服务器安装反向代理服务(例如nginx等)把http请求转发到IoTDB,当然你可以不使用反向代理服务直接使用openapi的rest,如果你正在使用grafana服务同时使用nginx则需要在nginx.conf添加 add_header 'Access-Control-Max-Age' xx 来保证可以正常使用
+OpenApi 接口使用了基础(basic)鉴权,每次url请求都需要在header中携带 'Authorization': 'Basic ' + base64.encode(username + ':' + password)例如:
+
+```
+location /rest/ {
+   proxy_pass  http://ip:port/rest/;  
+   add_header 'Access-Control-Max-Age' 20;
+}
+```
+
+### Configuration
+配置位于“iotdb-engines.properties”中,将“enable_openApi”设置为“true”以启用该模块,而将“false”设置为禁用该模块。
+默认情况下,该值为“false”。
+```
+enable_openApi=true
+```
+
+仅在“enable_openApi=true”时生效。将“openApi_port”设置为数字(1025~65535),以自定义rest服务套接字端口。
+默认情况下,值为“18080”。
+
+```
+openApi_port=18080
+```
+
+设置Prometheus数据存储时的存储组数量
+
+```
+sg_count=5
+```
+
+openApi 开启ssl配置,将“enable_https”设置为“true”以启用该模块,而将“false”设置为禁用该模块。
+默认情况下,该值为“false”。
+
+```
+enable_https=false
+```
+
+keyStore所在路径
+
+```
+key_store_path=/xxx/xxx.keystore
+```
+keystore密码
+
+```
+key_store_pwd=xxxx
+```
+trustStore所在路径(非必填)
+
+```
+trust_store_path=xxxx
+```
+
+trustStore密码
+```
+trust_store_pwd=xxxx
+```
+ssl 超时时间单位为秒
+
+```
+idle_timeout=5000
+```
+
+## grafana接口
+
+## 检查iotdb服务是否在运行
+请求方式:get
+请求url:http://ip:port/ping
+```
+$ curl -H "Authorization:Basic cm9vdDpyb2901" http://127.0.0.1:18080/ping
+$ {"code":4,"type":"ok","message":"login success!"}
+```
+响应示例
+```json
+{
+"code": 4,
+"type": "ok",
+"message": "login success!"
+}
+```
+用户名密码认证失败示例
+```json
+{
+  "code": 1,
+  "type": "error",
+  "message": "username or passowrd is incorrect!"
+}
+```
+
+##用于通过Grafana逐级获取时间序列名称
+请求方式:post
+请求头:application/json
+请求url:http://ip:port/rest/grafana/node
+```
+$ curl -H "Content-Type:application/json" -H "Authorization:Basic cm9vdDpyb290" -X POST --data '["root","sg5"]' http://127.0.0.1:18080/rest/grafana/node
+$ ["wf01","wf02","wf03"]
+```
+请求示例:
+```json
+["root","sg5"]
+```
+
+响应示例:
+```json
+["wf01","wf02","wf03"]
+```
+
+##为Grafana提供自动降采样数据查询
+请求方式:post
+请求头:application/json
+请求url:http://ip:port/rest/grafana/query/json
+```
+$ curl -H "Content-Type:application/json" -H "Authorization:Basic cm9vdDpyb290" -X POST --data '{"paths":["root","ln","wf02"],"aggregation":"AVG","groupBy":{"samplingInterval":"1s","step":"1s"},"stime":1627286097811,"etime":1627286397811}' http://127.0.0.1:18080/rest/grafana/query/json
+$ [{"datapoints":[[1.2,1627285095273],[0.75,1627285096273],[0.45,1627285097273],[0.15,1627285098273],[0.9,1627285099273],[0.0,1627285100273],[0.15,1627285101273]],"target":"root.ln.wf02"}]
+```
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  interval | string | 是  |  间隔 |
+| stime  |  number |  是 |  开始时间(时间戳) |
+| etime | number|  是 |  结束时间(时间戳) |
+| paths  |  array|  是 |  timeseries 为root.sg 转换成path为["root","sg"] |
+| aggregation  |  string | 否  | 函数 |
+| groupBy  |  object | 否  | 分组 |
+| samplingInterval  |  string | 否  | 降采样间隔 |
+| step  |  string | 否  | 降采样步长 |
+| limitAll  |  object | 否  | 限制 |
+| slimit  |  string | 否(默认值为10)  |  列数 |
+| limit  |  string | 否  |  行数 |
+| fills  |  array | 否  |  填充 |
+| dataType  |  string | 否  |  填充函数 |
+| previous  |  string |  否 |  填充类型 |
+| duration  |  string |  否 |  时间范围 |
+请求示例:
+```json
+{"paths":["root","ln"],"limitAll":{"slimit":"1","limit":""},"aggregation":"AVG","groupBy":{"samplingInterval":"1s","step":"1s"},"stime":1627286097811,"etime":1627286397811}
+```
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------ |
+|  datapoints |  array | 返回数组第一个为值,第二个为时间,依次类推  |
+| target  |  string |  返回查询timeseries |
+
+响应示例:
+```json
+[
+  {
+    "datapoints":[
+      [
+        0.8999999761581421,
+        1627286097811
+      ],
+      [
+        0.75,
+        1627286098811
+      ],
+      [
+        0.75,
+        1627286099811
+      ],
+      [
+        1.0499999523162842,
+        1627286100811
+      ],
+      [
+        1.0499999523162842,
+        1627286101811
+      ],
+      [
+        0.6000000238418579,
+        1627286102811
+      ]
+    ],
+    "target":"root.ln.wf03"
+  }
+]
+```
+##为Grafana提供自动降采样数据查询(DataFrame)
+请求方式:post
+请求头:application/json
+请求url:http://ip:port/rest/grafana/query/frame
+```
+$ curl -H "Content-Type:application/json" -H "Authorization:Basic cm9vdDpyb290" -X POST --data '{"interval":"1s","stime":"1616554359000","etime":"1616554369000","paths":["root","sg7"]}' http://127.0.0.1:18080/rest/grafana/query/frame
+$ [{"values":[1616554359000,1616554360000,1616554361000,1616554362000,1616554363000,1616554364000,1616554365000,1616554366000,1616554367000,1616554368000],"name":"Time","type":"time"},{"values":[5.0,7.0,7.0,null,7.0,7.0,null,null,null],"name":"root.sg7.val02","type":"DOUBLE"},{"values":[5.0,null,null,null,null,null,null,null,null],"name":"root.sg7.val03","type":"DOUBLE"}]
+```
+
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  interval | string | 是  |  间隔 |
+| stime  |  number |  是 |  开始时间(时间戳) |
+| etime | number|  是 |  结束时间(时间戳) |
+| paths  |  array|  是 |  timeseries 为root.sg 转换成path为["root","sg"] |
+| aggregation  |  string | 否  | 函数 |
+| groupBy  |  object | 否  | 分组 |
+| samplingInterval  |  string | 否  | 降采样间隔 |
+| step  |  string | 否  | 降采样步长 |
+| limitAll  |  object | 否  | 限制 |
+| slimit  |  string | 否(默认值为10)  |  列数 |
+| limit  |  string | 否  |  行数 |
+| fills  |  array | 否  |  填充 |
+| dataType  |  string | 否  |  填充函数 |
+| previous  |  string |  否 |  填充类型 |
+| duration  |  string |  否 |  时间范围 |
+请求示例:
+```json
+{"paths":["root","ln"],"limitAll":{"slimit":"1","limit":""},"aggregation":"AVG","groupBy":{"samplingInterval":"1s","step":"1s"},"stime":1627286097811,"etime":1627286397811}
+```
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------ |
+|  name |  String | 字段名称  |
+| type  |  string |  字段类型|
+| values  |  array |  字段值|
+
+响应示例:
+```json
+[
+  {
+    "values":[
+      1616554359000,
+      1616554360000,
+      1616554361000,
+      1616554362000,
+      1616554363000,
+      1616554364000,
+      1616554365000,
+      1616554366000,
+      1616554367000,
+      1616554368000
+    ],
+    "name":"Time",
+    "type":"INT64"
+  },
+  {
+    "values":[
+      5,
+      7,
+      7,
+      null,
+      7,
+      7,
+      null,
+      null,
+      null
+    ],
+    "name":"root.sg7.val02",
+    "type":"DOUBLE"
+  },
+  {
+    "values":[
+      5,
+      null,
+      null,
+      null,
+      null,
+      null,
+      null,
+      null,
+      null
+    ],
+    "name":"root.sg7.val03",
+    "type":"DOUBLE"
+  }
+]
+```
+## prometheus接口
+
+### prometheus wirte接口
+prometheus 数据经过Protobuf (3.12.3)编码和snappy 压缩后传输
+
+请求方式:post
+请求头:application/x-protobuf
+请求url:http://ip:port/rest/prometheus/write
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  Timeseries | array | 是  |   |
+| Labels  |  array |  是 |  tag信息 |
+|  Name | String|  是 |  tagKey |
+| Value  |  array|  是 | tagValue |
+| Samples  |  array | 是  |   |
+| Timestamp  |  number |  是 |  时间 |
+| Value  |  number |  是 |  值 |
+
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------ |
+|  code |  number | 200 成功,500 失败,401没有权限  |
+| message  |  string | 说明|
+
+
+响应示例:
+```json
+{"code": 200,
+"message": "write data success"
+}
+```
+
+### prometheus read 接口
+prometheus 数据经过Protobuf (3.12.3)编码和snappy 压缩后传输
+
+请求方式:post
+请求头:application/x-protobuf
+请求url:http://ip:port/rest/prometheus/query
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  Queries | array | 是  |   |
+| StartTimestampMs  |  number |  是 |  开始时间 |
+| EndTimestampMs | number|  是 |  结束时间 |
+| Matchers  |  array|  是 |  |
+| Name  |  String |  是 |  名称 |
+| Value  |  String |  是 |  值 |
+| Hints  |  array|  是 |  |
+| StepMs  |  number |  否 |  时间间隔 |
+| Func  |  String |  否 |  函数 |
+| StartMs  |  number |  是 |  开始时间 |
+| EndMs  |  number |  否 |  结束时间 |
+| Grouping  |  String |  否 |  聚合字段 |
+| By  |  boolean |  否 |  是否启用group by |
+| RangeMs  |  number |  否 |  滑动步长 |
+
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------|
+|  Timeseries | array |   |
+| Labels  |  array | tag信息 |
+|  Name | String|  tagKey |
+| Value  |  array| tagValue |
+| Samples  |  array |   |
+| Timestamp  |  number|  时间 |
+| Value  |  number | 值 |
+
+## 其他接口
+
+###  read 接口

Review comment:
       ```suggestion
   ###  SQL 查询接口
   ```

##########
File path: docs/zh/UserGuide/Communication-Service-Protocol/OpenApi.md
##########
@@ -0,0 +1,442 @@
+<!--
+
+    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.
+
+-->
+
+# Rest API
+IoTDB的restApi设计用于支持与Grafana和Prometheus的集成同时也提供了查询、插入和non-query接口,它使用OpenAPI标准来定义接口和生成框架源代码。
+为了IoTDB数据库安全我们建议使用一台非IoTDB的服务器安装反向代理服务(例如nginx等)把http请求转发到IoTDB,当然你可以不使用反向代理服务直接使用openapi的rest,如果你正在使用grafana服务同时使用nginx则需要在nginx.conf添加 add_header 'Access-Control-Max-Age' xx 来保证可以正常使用
+OpenApi 接口使用了基础(basic)鉴权,每次url请求都需要在header中携带 'Authorization': 'Basic ' + base64.encode(username + ':' + password)例如:
+
+```
+location /rest/ {
+   proxy_pass  http://ip:port/rest/;  
+   add_header 'Access-Control-Max-Age' 20;
+}
+```
+
+### Configuration
+配置位于“iotdb-engines.properties”中,将“enable_openApi”设置为“true”以启用该模块,而将“false”设置为禁用该模块。
+默认情况下,该值为“false”。
+```
+enable_openApi=true
+```
+
+仅在“enable_openApi=true”时生效。将“openApi_port”设置为数字(1025~65535),以自定义rest服务套接字端口。
+默认情况下,值为“18080”。
+
+```
+openApi_port=18080
+```
+
+设置Prometheus数据存储时的存储组数量
+
+```
+sg_count=5
+```
+
+openApi 开启ssl配置,将“enable_https”设置为“true”以启用该模块,而将“false”设置为禁用该模块。
+默认情况下,该值为“false”。
+
+```
+enable_https=false
+```
+
+keyStore所在路径
+
+```
+key_store_path=/xxx/xxx.keystore
+```
+keystore密码
+
+```
+key_store_pwd=xxxx
+```
+trustStore所在路径(非必填)
+
+```
+trust_store_path=xxxx
+```
+
+trustStore密码
+```
+trust_store_pwd=xxxx
+```
+ssl 超时时间单位为秒
+
+```
+idle_timeout=5000
+```
+
+## grafana接口
+
+## 检查iotdb服务是否在运行
+请求方式:get
+请求url:http://ip:port/ping
+```
+$ curl -H "Authorization:Basic cm9vdDpyb2901" http://127.0.0.1:18080/ping
+$ {"code":4,"type":"ok","message":"login success!"}
+```
+响应示例
+```json
+{
+"code": 4,
+"type": "ok",
+"message": "login success!"
+}
+```
+用户名密码认证失败示例
+```json
+{
+  "code": 1,
+  "type": "error",
+  "message": "username or passowrd is incorrect!"
+}
+```
+
+##用于通过Grafana逐级获取时间序列名称
+请求方式:post
+请求头:application/json
+请求url:http://ip:port/rest/grafana/node
+```
+$ curl -H "Content-Type:application/json" -H "Authorization:Basic cm9vdDpyb290" -X POST --data '["root","sg5"]' http://127.0.0.1:18080/rest/grafana/node
+$ ["wf01","wf02","wf03"]
+```
+请求示例:
+```json
+["root","sg5"]
+```
+
+响应示例:
+```json
+["wf01","wf02","wf03"]
+```
+
+##为Grafana提供自动降采样数据查询
+请求方式:post
+请求头:application/json
+请求url:http://ip:port/rest/grafana/query/json
+```
+$ curl -H "Content-Type:application/json" -H "Authorization:Basic cm9vdDpyb290" -X POST --data '{"paths":["root","ln","wf02"],"aggregation":"AVG","groupBy":{"samplingInterval":"1s","step":"1s"},"stime":1627286097811,"etime":1627286397811}' http://127.0.0.1:18080/rest/grafana/query/json
+$ [{"datapoints":[[1.2,1627285095273],[0.75,1627285096273],[0.45,1627285097273],[0.15,1627285098273],[0.9,1627285099273],[0.0,1627285100273],[0.15,1627285101273]],"target":"root.ln.wf02"}]
+```
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  interval | string | 是  |  间隔 |
+| stime  |  number |  是 |  开始时间(时间戳) |
+| etime | number|  是 |  结束时间(时间戳) |
+| paths  |  array|  是 |  timeseries 为root.sg 转换成path为["root","sg"] |
+| aggregation  |  string | 否  | 函数 |
+| groupBy  |  object | 否  | 分组 |
+| samplingInterval  |  string | 否  | 降采样间隔 |
+| step  |  string | 否  | 降采样步长 |
+| limitAll  |  object | 否  | 限制 |
+| slimit  |  string | 否(默认值为10)  |  列数 |
+| limit  |  string | 否  |  行数 |
+| fills  |  array | 否  |  填充 |
+| dataType  |  string | 否  |  填充函数 |
+| previous  |  string |  否 |  填充类型 |
+| duration  |  string |  否 |  时间范围 |
+请求示例:
+```json
+{"paths":["root","ln"],"limitAll":{"slimit":"1","limit":""},"aggregation":"AVG","groupBy":{"samplingInterval":"1s","step":"1s"},"stime":1627286097811,"etime":1627286397811}
+```
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------ |
+|  datapoints |  array | 返回数组第一个为值,第二个为时间,依次类推  |
+| target  |  string |  返回查询timeseries |
+
+响应示例:
+```json
+[
+  {
+    "datapoints":[
+      [
+        0.8999999761581421,
+        1627286097811
+      ],
+      [
+        0.75,
+        1627286098811
+      ],
+      [
+        0.75,
+        1627286099811
+      ],
+      [
+        1.0499999523162842,
+        1627286100811
+      ],
+      [
+        1.0499999523162842,
+        1627286101811
+      ],
+      [
+        0.6000000238418579,
+        1627286102811
+      ]
+    ],
+    "target":"root.ln.wf03"
+  }
+]
+```
+##为Grafana提供自动降采样数据查询(DataFrame)
+请求方式:post
+请求头:application/json
+请求url:http://ip:port/rest/grafana/query/frame
+```
+$ curl -H "Content-Type:application/json" -H "Authorization:Basic cm9vdDpyb290" -X POST --data '{"interval":"1s","stime":"1616554359000","etime":"1616554369000","paths":["root","sg7"]}' http://127.0.0.1:18080/rest/grafana/query/frame
+$ [{"values":[1616554359000,1616554360000,1616554361000,1616554362000,1616554363000,1616554364000,1616554365000,1616554366000,1616554367000,1616554368000],"name":"Time","type":"time"},{"values":[5.0,7.0,7.0,null,7.0,7.0,null,null,null],"name":"root.sg7.val02","type":"DOUBLE"},{"values":[5.0,null,null,null,null,null,null,null,null],"name":"root.sg7.val03","type":"DOUBLE"}]
+```
+
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  interval | string | 是  |  间隔 |
+| stime  |  number |  是 |  开始时间(时间戳) |
+| etime | number|  是 |  结束时间(时间戳) |
+| paths  |  array|  是 |  timeseries 为root.sg 转换成path为["root","sg"] |
+| aggregation  |  string | 否  | 函数 |
+| groupBy  |  object | 否  | 分组 |
+| samplingInterval  |  string | 否  | 降采样间隔 |
+| step  |  string | 否  | 降采样步长 |
+| limitAll  |  object | 否  | 限制 |
+| slimit  |  string | 否(默认值为10)  |  列数 |
+| limit  |  string | 否  |  行数 |
+| fills  |  array | 否  |  填充 |
+| dataType  |  string | 否  |  填充函数 |
+| previous  |  string |  否 |  填充类型 |
+| duration  |  string |  否 |  时间范围 |
+请求示例:
+```json
+{"paths":["root","ln"],"limitAll":{"slimit":"1","limit":""},"aggregation":"AVG","groupBy":{"samplingInterval":"1s","step":"1s"},"stime":1627286097811,"etime":1627286397811}
+```
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------ |
+|  name |  String | 字段名称  |
+| type  |  string |  字段类型|
+| values  |  array |  字段值|
+
+响应示例:
+```json
+[
+  {
+    "values":[
+      1616554359000,
+      1616554360000,
+      1616554361000,
+      1616554362000,
+      1616554363000,
+      1616554364000,
+      1616554365000,
+      1616554366000,
+      1616554367000,
+      1616554368000
+    ],
+    "name":"Time",
+    "type":"INT64"
+  },
+  {
+    "values":[
+      5,
+      7,
+      7,
+      null,
+      7,
+      7,
+      null,
+      null,
+      null
+    ],
+    "name":"root.sg7.val02",
+    "type":"DOUBLE"
+  },
+  {
+    "values":[
+      5,
+      null,
+      null,
+      null,
+      null,
+      null,
+      null,
+      null,
+      null
+    ],
+    "name":"root.sg7.val03",
+    "type":"DOUBLE"
+  }
+]
+```
+## prometheus接口
+
+### prometheus wirte接口
+prometheus 数据经过Protobuf (3.12.3)编码和snappy 压缩后传输
+
+请求方式:post
+请求头:application/x-protobuf
+请求url:http://ip:port/rest/prometheus/write
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  Timeseries | array | 是  |   |
+| Labels  |  array |  是 |  tag信息 |
+|  Name | String|  是 |  tagKey |
+| Value  |  array|  是 | tagValue |
+| Samples  |  array | 是  |   |
+| Timestamp  |  number |  是 |  时间 |
+| Value  |  number |  是 |  值 |
+
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------ |
+|  code |  number | 200 成功,500 失败,401没有权限  |
+| message  |  string | 说明|
+
+
+响应示例:
+```json
+{"code": 200,
+"message": "write data success"
+}
+```
+
+### prometheus read 接口
+prometheus 数据经过Protobuf (3.12.3)编码和snappy 压缩后传输
+
+请求方式:post
+请求头:application/x-protobuf
+请求url:http://ip:port/rest/prometheus/query
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  Queries | array | 是  |   |
+| StartTimestampMs  |  number |  是 |  开始时间 |
+| EndTimestampMs | number|  是 |  结束时间 |
+| Matchers  |  array|  是 |  |
+| Name  |  String |  是 |  名称 |
+| Value  |  String |  是 |  值 |
+| Hints  |  array|  是 |  |
+| StepMs  |  number |  否 |  时间间隔 |
+| Func  |  String |  否 |  函数 |
+| StartMs  |  number |  是 |  开始时间 |
+| EndMs  |  number |  否 |  结束时间 |
+| Grouping  |  String |  否 |  聚合字段 |
+| By  |  boolean |  否 |  是否启用group by |
+| RangeMs  |  number |  否 |  滑动步长 |
+
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------|
+|  Timeseries | array |   |
+| Labels  |  array | tag信息 |
+|  Name | String|  tagKey |
+| Value  |  array| tagValue |
+| Samples  |  array |   |
+| Timestamp  |  number|  时间 |
+| Value  |  number | 值 |
+
+## 其他接口
+
+###  read 接口
+
+请求方式:post
+请求头:application/json
+请求url:http://ip:port/rest/read
+```
+$ curl -H "Content-Type:application/json" -H "Authorization:Basic cm9vdDpyb290" -X POST --data '{"sql":"select * from root limit 1 slimit 2"}' http://127.0.0.1:18080/rest/read
+$ [{"values":[1],"name":"Time","type":"INT64"},{"values":[1.1],"name":"root.ln.wf02","type":""},{"values":[2.0],"name":"root.ln.wf03","type":""}]
+```
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  sql | string | 是  |   |
+
+
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------|
+| values | array |  值 |
+| name  |  string | 测点名称 |
+| type | String| 数据类型 |
+
+###  nonQuery 接口
+
+请求方式:post
+请求头:application/json
+请求url:http://ip:port/rest/nonQuery
+
+```
+$ curl -H "Content-Type:application/json" -H "Authorization:Basic cm9vdDpyb290" -X POST --data '{"sql":"set storage group to root.ln"}' http://127.0.0.1:18080/rest/nonQuery
+$ {"code":200,"message":"execute sucessfully"}
+```
+参数说明:
+
+|参数名称  |参数类型  |是否必填|参数描述|
+| ------------ | ------------ | ------------ |------------ |
+|  sql | string | 是  |   |
+
+
+返回参数:
+
+|参数名称  |参数类型  |参数描述|
+| ------------ | ------------ | ------------|
+| code | integer |  状态码 |
+| message  |  string | 信息提示 |
+
+###  write 接口

Review comment:
       ```suggestion
   ###  写入接口
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #3644: [IOTDB-1762] IoTDB REST Data Services : OpenAPI Support

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/43580787/badge)](https://coveralls.io/builds/43580787)
   
   Coverage increased (+0.2%) to 67.457% when pulling **e56e5492a3947811c5a96262f509539ef91d9460 on CloudWise-Lukemiao:openapi-rest** into **8ec3e12865699064958e36a37a50083d85e11f9c on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] CloudWise-Lukemiao commented on a change in pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
CloudWise-Lukemiao commented on a change in pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#discussion_r684263775



##########
File path: openapi/java-rest/src/main/java/org/apache/iotdb/openapi/gen/handler/impl/ProtobufMessageBodyWriter.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.openapi.gen.handler.impl;
+
+import com.google.protobuf.Message;
+
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyWriter;
+import javax.ws.rs.ext.Provider;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+import java.util.Map;
+import java.util.WeakHashMap;
+
+@Provider
+@Produces("application/x-protobuf")
+public class ProtobufMessageBodyWriter implements MessageBodyWriter<Message> {
+
+  @Override
+  public boolean isWriteable(
+      Class<?> aClass, Type type, Annotation[] annotations, MediaType mediaType) {
+    return Message.class.isAssignableFrom(aClass);
+    // return false;
+  }
+
+  private Map<Object, byte[]> buffer = new WeakHashMap<Object, byte[]>();
+
+  @Override
+  public long getSize(
+      Message message, Class<?> aClass, Type type, Annotation[] annotations, MediaType mediaType) {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    try {
+      message.writeTo(baos);
+    } catch (IOException e) {
+      return -1;
+    }
+    byte[] bytes = baos.toByteArray();
+    buffer.put(message, bytes);
+    return bytes.length;
+    // return 0;

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] qiaojialin commented on a change in pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#discussion_r686554262



##########
File path: docs/zh/UserGuide/Communication-Service-Protocol/OpenApi.md
##########
@@ -0,0 +1,442 @@
+<!--
+
+    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.
+
+-->
+
+# Rest API
+IoTDB的restApi设计用于支持与Grafana和Prometheus的集成同时也提供了查询、插入和non-query接口,它使用OpenAPI标准来定义接口和生成框架源代码。

Review comment:
       ```suggestion
   IoTDB 的 restApi 设计用于支持与 Grafana 和 Prometheus 的集成同时也提供了读写等接口,它使用 OpenAPI 标准来定义接口和生成框架源代码。
   ```

##########
File path: docs/zh/UserGuide/Communication-Service-Protocol/OpenApi.md
##########
@@ -0,0 +1,442 @@
+<!--
+
+    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.
+
+-->
+
+# Rest API
+IoTDB的restApi设计用于支持与Grafana和Prometheus的集成同时也提供了查询、插入和non-query接口,它使用OpenAPI标准来定义接口和生成框架源代码。
+为了IoTDB数据库安全我们建议使用一台非IoTDB的服务器安装反向代理服务(例如nginx等)把http请求转发到IoTDB,当然你可以不使用反向代理服务直接使用openapi的rest,如果你正在使用grafana服务同时使用nginx则需要在nginx.conf添加 add_header 'Access-Control-Max-Age' xx 来保证可以正常使用
+OpenApi 接口使用了基础(basic)鉴权,每次url请求都需要在header中携带 'Authorization': 'Basic ' + base64.encode(username + ':' + password)例如:

Review comment:
       ```suggestion
   OpenApi 接口使用了基础(basic)鉴权,每次url请求都需要在 header 中携带 'Authorization': 'Basic ' + base64.encode(username + ':' + password)例如:
   ```

##########
File path: server/src/test/java/org/apache/iotdb/openapi/apache/iotdb/OpenApiServiceTest.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.openapi.apache.iotdb;
+
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.openapi.gen.handler.NotFoundException;
+import org.apache.iotdb.openapi.gen.handler.filter.BasicSecurityContext;
+import org.apache.iotdb.openapi.gen.handler.impl.RestApiServiceImpl;
+import org.apache.iotdb.openapi.gen.handler.model.User;
+import org.apache.iotdb.openapi.gen.model.GroupByFillPlan;
+import org.apache.iotdb.openapi.gen.model.ReadData;
+import org.apache.iotdb.openapi.gen.model.WriteData;
+
+import com.google.gson.Gson;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.ws.rs.core.Response;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class OpenApiServiceTest {
+  GroupByFillPlan groupByFillPlan = new GroupByFillPlan();
+  User user = new User();
+  BasicSecurityContext basicSecurityContext = new BasicSecurityContext(user, false);
+  RestApiServiceImpl v1ApiService = new RestApiServiceImpl();
+
+  @Before
+  public void setUp() {
+    EnvironmentUtils.envSetUp();
+    user.setUsername("root");
+    user.setPassword("root");
+    BigDecimal stime = BigDecimal.valueOf(1);
+    BigDecimal etime = BigDecimal.valueOf(10);
+    List<String> path = new ArrayList<String>();
+    path.add("root");
+    path.add("sg");
+    path.add("aa");
+    groupByFillPlan.setStime(stime);
+    groupByFillPlan.setEtime(etime);
+    groupByFillPlan.setPaths(path);
+    // groupByFillPlan.set("1ms");

Review comment:
       remove

##########
File path: docs/zh/UserGuide/Communication-Service-Protocol/OpenApi.md
##########
@@ -0,0 +1,442 @@
+<!--
+
+    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.
+
+-->
+
+# Rest API
+IoTDB的restApi设计用于支持与Grafana和Prometheus的集成同时也提供了查询、插入和non-query接口,它使用OpenAPI标准来定义接口和生成框架源代码。
+为了IoTDB数据库安全我们建议使用一台非IoTDB的服务器安装反向代理服务(例如nginx等)把http请求转发到IoTDB,当然你可以不使用反向代理服务直接使用openapi的rest,如果你正在使用grafana服务同时使用nginx则需要在nginx.conf添加 add_header 'Access-Control-Max-Age' xx 来保证可以正常使用

Review comment:
       ```suggestion
   为了 IoTDB 数据库安全我们建议使用一台非 IoTDB 的服务器安装反向代理服务(例如 nginx 等)把 http 请求转发到 IoTDB,当然你可以不使用反向代理服务直接使用 openapi 的 rest,如果你正在使用 grafana 服务同时使用 nginx 则需要在 nginx.conf 添加 add_header 'Access-Control-Max-Age' xx 来保证可以正常使用
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] qiaojialin commented on a change in pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#discussion_r683102792



##########
File path: openapi/java-rest/pom.xml
##########
@@ -0,0 +1,597 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>openapi</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>java-rest</artifactId>
+    <!--<packaging>war</packaging>-->
+    <dependencies>
+        <dependency>
+            <groupId>org.xerial.snappy</groupId>
+            <artifactId>snappy-java</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+            <version>9.4.35.v20201120</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <profiles>
+        <profile>
+            <id>unix</id>
+            <activation>
+                <os>
+                    <family>unix</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>linux-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>
+                    <artifactId>jackson-jaxrs-json-provider</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <!-- Base64 encoding that works in both JVM and Android -->
+                <dependency>
+                    <groupId>com.brsanthu</groupId>
+                    <artifactId>migbase64</artifactId>
+                    <version>2.2</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                    <version>3.5.1</version>
+                </dependency>
+                <!-- Bean Validation API support -->
+                <!--             <dependency>
+                    <groupId>javax.validation</groupId>
+                    <artifactId>validation-api</artifactId>
+                    <version>${beanvalidation-version}</version>
+                    <scope>provided</scope>
+                </dependency>-->
+            </dependencies>
+        </profile>
+        <profile>
+            <id>mac</id>
+            <activation>
+                <os>
+                    <family>mac</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>osx-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>

Review comment:
       please check if the license is forbidden by apache.
   http://www.apache.org/legal/resolved.html#category-x

##########
File path: openapi/java-rest/src/main/java/org/apache/iotdb/openapi/gen/handler/factories/V1ApiServiceFactory.java
##########
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// this file is generated by openapi generator
+
+package org.apache.iotdb.openapi.gen.handler.factories;
+
+import org.apache.iotdb.openapi.gen.handler.V1ApiService;
+import org.apache.iotdb.openapi.gen.handler.impl.V1ApiServiceImpl;
+
+@javax.annotation.Generated(

Review comment:
       is this needed?

##########
File path: openapi/java-rest/pom.xml
##########
@@ -0,0 +1,597 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>openapi</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>java-rest</artifactId>
+    <!--<packaging>war</packaging>-->
+    <dependencies>
+        <dependency>
+            <groupId>org.xerial.snappy</groupId>
+            <artifactId>snappy-java</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+            <version>9.4.35.v20201120</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <profiles>
+        <profile>
+            <id>unix</id>
+            <activation>
+                <os>
+                    <family>unix</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>linux-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>
+                    <artifactId>jackson-jaxrs-json-provider</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <!-- Base64 encoding that works in both JVM and Android -->
+                <dependency>
+                    <groupId>com.brsanthu</groupId>
+                    <artifactId>migbase64</artifactId>
+                    <version>2.2</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                    <version>3.5.1</version>
+                </dependency>
+                <!-- Bean Validation API support -->
+                <!--             <dependency>
+                    <groupId>javax.validation</groupId>
+                    <artifactId>validation-api</artifactId>
+                    <version>${beanvalidation-version}</version>
+                    <scope>provided</scope>
+                </dependency>-->
+            </dependencies>
+        </profile>
+        <profile>
+            <id>mac</id>
+            <activation>
+                <os>
+                    <family>mac</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>osx-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>

Review comment:
       check license

##########
File path: server/src/main/java/org/apache/iotdb/openapi/gen/handler/impl/V1ApiServiceImpl.java
##########
@@ -0,0 +1,1025 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// this file is generated by openapi generator
+
+package org.apache.iotdb.openapi.gen.handler.impl;
+
+import org.apache.iotdb.db.auth.AuthException;
+import org.apache.iotdb.db.auth.AuthorityChecker;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.Planner;
+import org.apache.iotdb.db.qp.executor.PlanExecutor;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.GroupByTimePlan;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowChildPathsPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowPlan.ShowContentType;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.executor.QueryRouter;
+import org.apache.iotdb.openapi.gen.handler.NotFoundException;
+import org.apache.iotdb.openapi.gen.handler.V1ApiService;
+import org.apache.iotdb.openapi.gen.model.GroupByFillPlan;
+import org.apache.iotdb.openapi.gen.model.ReadData;
+import org.apache.iotdb.openapi.gen.model.WriteData;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+
+import com.google.common.base.Joiner;
+import com.google.gson.Gson;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ProtocolStringList;
+import org.apache.commons.lang3.StringUtils;
+import org.xerial.snappy.Snappy;
+import prometheus.Remote;
+import prometheus.Remote.Query;
+import prometheus.Remote.ReadRequest;
+import prometheus.Remote.ReadResponse;
+import prometheus.Types;
+import prometheus.Types.Label;
+import prometheus.Types.LabelMatcher;
+import prometheus.Types.Sample;
+import prometheus.Types.TimeSeries;
+import prometheus.Types.TimeSeries.Builder;
+
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@javax.annotation.Generated(
+    value = "org.openapitools.codegen.languages.JavaJerseyServerCodegen",
+    date = "2021-01-13T21:45:03.765+08:00[Asia/Shanghai]")
+public class V1ApiServiceImpl extends V1ApiService {

Review comment:
       why  V1?

##########
File path: openapi/java-rest/src/main/java/org/apache/iotdb/openapi/gen/handler/impl/ProtobufMessageBodyWriter.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.openapi.gen.handler.impl;
+
+import com.google.protobuf.Message;
+
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyWriter;
+import javax.ws.rs.ext.Provider;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+import java.util.Map;
+import java.util.WeakHashMap;
+
+@Provider
+@Produces("application/x-protobuf")
+public class ProtobufMessageBodyWriter implements MessageBodyWriter<Message> {
+
+  @Override
+  public boolean isWriteable(
+      Class<?> aClass, Type type, Annotation[] annotations, MediaType mediaType) {
+    return Message.class.isAssignableFrom(aClass);
+    // return false;

Review comment:
       remove 

##########
File path: openapi/java-rest/src/main/java/org/apache/iotdb/openapi/gen/handler/impl/ProtobufMessageBodyWriter.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.openapi.gen.handler.impl;
+
+import com.google.protobuf.Message;
+
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyWriter;
+import javax.ws.rs.ext.Provider;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+import java.util.Map;
+import java.util.WeakHashMap;
+
+@Provider
+@Produces("application/x-protobuf")
+public class ProtobufMessageBodyWriter implements MessageBodyWriter<Message> {
+
+  @Override
+  public boolean isWriteable(
+      Class<?> aClass, Type type, Annotation[] annotations, MediaType mediaType) {
+    return Message.class.isAssignableFrom(aClass);
+    // return false;
+  }
+
+  private Map<Object, byte[]> buffer = new WeakHashMap<Object, byte[]>();
+
+  @Override
+  public long getSize(
+      Message message, Class<?> aClass, Type type, Annotation[] annotations, MediaType mediaType) {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    try {
+      message.writeTo(baos);
+    } catch (IOException e) {
+      return -1;
+    }
+    byte[] bytes = baos.toByteArray();
+    buffer.put(message, bytes);
+    return bytes.length;
+    // return 0;

Review comment:
       remove

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
##########
@@ -729,6 +729,100 @@
 
   private String adminPassword = "root";
 
+  /** if the startOpenApi is true, we will start OpenApi */
+  private boolean startOpenApi = true;

Review comment:
       the default could be false, be consistent with iotdb-engine.properties

##########
File path: openapi/java-rest/pom.xml
##########
@@ -0,0 +1,597 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>openapi</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>java-rest</artifactId>
+    <!--<packaging>war</packaging>-->
+    <dependencies>
+        <dependency>
+            <groupId>org.xerial.snappy</groupId>
+            <artifactId>snappy-java</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+            <version>9.4.35.v20201120</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <profiles>
+        <profile>
+            <id>unix</id>
+            <activation>
+                <os>
+                    <family>unix</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>linux-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>
+                    <artifactId>jackson-jaxrs-json-provider</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <!-- Base64 encoding that works in both JVM and Android -->
+                <dependency>
+                    <groupId>com.brsanthu</groupId>
+                    <artifactId>migbase64</artifactId>
+                    <version>2.2</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                    <version>3.5.1</version>
+                </dependency>
+                <!-- Bean Validation API support -->
+                <!--             <dependency>
+                    <groupId>javax.validation</groupId>
+                    <artifactId>validation-api</artifactId>
+                    <version>${beanvalidation-version}</version>
+                    <scope>provided</scope>
+                </dependency>-->
+            </dependencies>
+        </profile>
+        <profile>
+            <id>mac</id>
+            <activation>
+                <os>
+                    <family>mac</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>osx-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>
+                    <artifactId>jackson-jaxrs-json-provider</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <!-- Base64 encoding that works in both JVM and Android -->
+                <dependency>
+                    <groupId>com.brsanthu</groupId>
+                    <artifactId>migbase64</artifactId>
+                    <version>2.2</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                    <version>3.5.1</version>
+                </dependency>
+                <!-- Bean Validation API support -->

Review comment:
       remove unused dependency

##########
File path: openapi/java-rest/src/main/java/org/apache/iotdb/openapi/gen/handler/impl/V1ApiServiceImpl.java
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// this file is generated by openapi generator
+
+package org.apache.iotdb.openapi.gen.handler.impl;
+
+import org.apache.iotdb.openapi.gen.handler.*;
+import org.apache.iotdb.openapi.gen.handler.NotFoundException;
+import org.apache.iotdb.openapi.gen.model.*;
+import org.apache.iotdb.openapi.gen.model.GroupByFillPlan;
+
+import javax.validation.constraints.*;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+
+import java.util.List;
+
+@javax.annotation.Generated(
+    value = "org.openapitools.codegen.languages.JavaJerseyServerCodegen",
+    date = "2021-01-13T21:45:03.765+08:00[Asia/Shanghai]")
+public class V1ApiServiceImpl extends V1ApiService {
+  @Override
+  public Response postV1GrafanaData(
+      GroupByFillPlan groupByFillPlan, SecurityContext securityContext) throws NotFoundException {
+    // do some magic!
+    return Response.ok().entity(new ApiResponseMessage(ApiResponseMessage.OK, "magic!")).build();
+  }
+
+  @Override
+  public Response postV1GrafanaDataSimplejson(
+      GroupByFillPlan groupByFillPlan, SecurityContext securityContext) throws NotFoundException {
+    // do some magic!

Review comment:
       remove

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
##########
@@ -729,6 +729,100 @@
 
   private String adminPassword = "root";
 
+  /** if the startOpenApi is true, we will start OpenApi */
+  private boolean startOpenApi = true;
+
+  /** set the OpenApi reset port. */
+  private int OpenApiPort = 18080;
+
+  /** set the OpenApi storegroup number. */
+  private int sgCount = 5;
+
+  /** enable the OpenApi ssl. */
+  private boolean enable_https = false;
+  /** openapi ssl key Store Path */

Review comment:
       add an empty line




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] CloudWise-Lukemiao commented on a change in pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
CloudWise-Lukemiao commented on a change in pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#discussion_r683289880



##########
File path: server/src/main/java/org/apache/iotdb/openapi/gen/handler/impl/V1ApiServiceImpl.java
##########
@@ -0,0 +1,1025 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// this file is generated by openapi generator
+
+package org.apache.iotdb.openapi.gen.handler.impl;
+
+import org.apache.iotdb.db.auth.AuthException;
+import org.apache.iotdb.db.auth.AuthorityChecker;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.Planner;
+import org.apache.iotdb.db.qp.executor.PlanExecutor;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.GroupByTimePlan;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowChildPathsPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowPlan.ShowContentType;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.executor.QueryRouter;
+import org.apache.iotdb.openapi.gen.handler.NotFoundException;
+import org.apache.iotdb.openapi.gen.handler.V1ApiService;
+import org.apache.iotdb.openapi.gen.model.GroupByFillPlan;
+import org.apache.iotdb.openapi.gen.model.ReadData;
+import org.apache.iotdb.openapi.gen.model.WriteData;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+
+import com.google.common.base.Joiner;
+import com.google.gson.Gson;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ProtocolStringList;
+import org.apache.commons.lang3.StringUtils;
+import org.xerial.snappy.Snappy;
+import prometheus.Remote;
+import prometheus.Remote.Query;
+import prometheus.Remote.ReadRequest;
+import prometheus.Remote.ReadResponse;
+import prometheus.Types;
+import prometheus.Types.Label;
+import prometheus.Types.LabelMatcher;
+import prometheus.Types.Sample;
+import prometheus.Types.TimeSeries;
+import prometheus.Types.TimeSeries.Builder;
+
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@javax.annotation.Generated(
+    value = "org.openapitools.codegen.languages.JavaJerseyServerCodegen",
+    date = "2021-01-13T21:45:03.765+08:00[Asia/Shanghai]")
+public class V1ApiServiceImpl extends V1ApiService {

Review comment:
       The method name is the code automatically generated from the configuration file  iotdb-rest.yaml .It has been modified to  RestApiServiceImpl




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu closed pull request #3644: [IOTDB-1762] IoTDB REST Data Services : OpenAPI Support

Posted by GitBox <gi...@apache.org>.
SteveYurongSu closed pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/42058591/badge)](https://coveralls.io/builds/42058591)
   
   Coverage decreased (-1.0%) to 67.248% when pulling **b7d050c2550776884ef1d3e6ae2f3543ea02b29d on CloudWise-Lukemiao:openapi-rest** into **0b2d0af48e00b6b72c5e4625e3a9613ac1c6204d on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] CloudWise-Lukemiao commented on a change in pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
CloudWise-Lukemiao commented on a change in pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#discussion_r684953569



##########
File path: openapi/java-rest/src/main/java/org/apache/iotdb/openapi/gen/handler/factories/V1ApiServiceFactory.java
##########
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// this file is generated by openapi generator
+
+package org.apache.iotdb.openapi.gen.handler.factories;
+
+import org.apache.iotdb.openapi.gen.handler.V1ApiService;
+import org.apache.iotdb.openapi.gen.handler.impl.V1ApiServiceImpl;
+
+@javax.annotation.Generated(

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/41928644/badge)](https://coveralls.io/builds/41928644)
   
   Coverage increased (+0.04%) to 68.262% when pulling **d140df882d68a9b4cdf9d2246fbe98d2da0fdcac on CloudWise-Lukemiao:openapi-rest** into **0b2d0af48e00b6b72c5e4625e3a9613ac1c6204d on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] CloudWise-Lukemiao commented on a change in pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
CloudWise-Lukemiao commented on a change in pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#discussion_r683289880



##########
File path: server/src/main/java/org/apache/iotdb/openapi/gen/handler/impl/V1ApiServiceImpl.java
##########
@@ -0,0 +1,1025 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// this file is generated by openapi generator
+
+package org.apache.iotdb.openapi.gen.handler.impl;
+
+import org.apache.iotdb.db.auth.AuthException;
+import org.apache.iotdb.db.auth.AuthorityChecker;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.Planner;
+import org.apache.iotdb.db.qp.executor.PlanExecutor;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.GroupByTimePlan;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowChildPathsPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowPlan.ShowContentType;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.executor.QueryRouter;
+import org.apache.iotdb.openapi.gen.handler.NotFoundException;
+import org.apache.iotdb.openapi.gen.handler.V1ApiService;
+import org.apache.iotdb.openapi.gen.model.GroupByFillPlan;
+import org.apache.iotdb.openapi.gen.model.ReadData;
+import org.apache.iotdb.openapi.gen.model.WriteData;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+
+import com.google.common.base.Joiner;
+import com.google.gson.Gson;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ProtocolStringList;
+import org.apache.commons.lang3.StringUtils;
+import org.xerial.snappy.Snappy;
+import prometheus.Remote;
+import prometheus.Remote.Query;
+import prometheus.Remote.ReadRequest;
+import prometheus.Remote.ReadResponse;
+import prometheus.Types;
+import prometheus.Types.Label;
+import prometheus.Types.LabelMatcher;
+import prometheus.Types.Sample;
+import prometheus.Types.TimeSeries;
+import prometheus.Types.TimeSeries.Builder;
+
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@javax.annotation.Generated(
+    value = "org.openapitools.codegen.languages.JavaJerseyServerCodegen",
+    date = "2021-01-13T21:45:03.765+08:00[Asia/Shanghai]")
+public class V1ApiServiceImpl extends V1ApiService {

Review comment:
       The method name is the code automatically generated from the configuration file  iotdb-rest.yaml 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/41953314/badge)](https://coveralls.io/builds/41953314)
   
   Coverage increased (+0.02%) to 68.246% when pulling **564e4ffcb4f7f421cfcc9f414fd4eafa98cebd9e on CloudWise-Lukemiao:openapi-rest** into **0b2d0af48e00b6b72c5e4625e3a9613ac1c6204d on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] CloudWise-Lukemiao commented on a change in pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
CloudWise-Lukemiao commented on a change in pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#discussion_r683289880



##########
File path: server/src/main/java/org/apache/iotdb/openapi/gen/handler/impl/V1ApiServiceImpl.java
##########
@@ -0,0 +1,1025 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// this file is generated by openapi generator
+
+package org.apache.iotdb.openapi.gen.handler.impl;
+
+import org.apache.iotdb.db.auth.AuthException;
+import org.apache.iotdb.db.auth.AuthorityChecker;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.Planner;
+import org.apache.iotdb.db.qp.executor.PlanExecutor;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.GroupByTimePlan;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowChildPathsPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowPlan.ShowContentType;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.executor.QueryRouter;
+import org.apache.iotdb.openapi.gen.handler.NotFoundException;
+import org.apache.iotdb.openapi.gen.handler.V1ApiService;
+import org.apache.iotdb.openapi.gen.model.GroupByFillPlan;
+import org.apache.iotdb.openapi.gen.model.ReadData;
+import org.apache.iotdb.openapi.gen.model.WriteData;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+
+import com.google.common.base.Joiner;
+import com.google.gson.Gson;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ProtocolStringList;
+import org.apache.commons.lang3.StringUtils;
+import org.xerial.snappy.Snappy;
+import prometheus.Remote;
+import prometheus.Remote.Query;
+import prometheus.Remote.ReadRequest;
+import prometheus.Remote.ReadResponse;
+import prometheus.Types;
+import prometheus.Types.Label;
+import prometheus.Types.LabelMatcher;
+import prometheus.Types.Sample;
+import prometheus.Types.TimeSeries;
+import prometheus.Types.TimeSeries.Builder;
+
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@javax.annotation.Generated(
+    value = "org.openapitools.codegen.languages.JavaJerseyServerCodegen",
+    date = "2021-01-13T21:45:03.765+08:00[Asia/Shanghai]")
+public class V1ApiServiceImpl extends V1ApiService {

Review comment:
       The method name is the code automatically generated from the configuration file  iotdb-rest.yaml 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/42104045/badge)](https://coveralls.io/builds/42104045)
   
   Coverage decreased (-1.02%) to 67.202% when pulling **7bd5d21f2f913803a6731dbd1559a32589c8652e on CloudWise-Lukemiao:openapi-rest** into **0b2d0af48e00b6b72c5e4625e3a9613ac1c6204d on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls commented on pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
coveralls commented on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/41715587/badge)](https://coveralls.io/builds/41715587)
   
   Coverage decreased (-0.005%) to 68.218% when pulling **a4dd6fa820731cd4eefa8a38b97de44c19dfe497 on CloudWise-Lukemiao:openapi-rest** into **0b2d0af48e00b6b72c5e4625e3a9613ac1c6204d on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/42059413/badge)](https://coveralls.io/builds/42059413)
   
   Coverage decreased (-0.9%) to 67.273% when pulling **b7d050c2550776884ef1d3e6ae2f3543ea02b29d on CloudWise-Lukemiao:openapi-rest** into **0b2d0af48e00b6b72c5e4625e3a9613ac1c6204d on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/42058027/badge)](https://coveralls.io/builds/42058027)
   
   Coverage decreased (-1.0%) to 67.249% when pulling **b7d050c2550776884ef1d3e6ae2f3543ea02b29d on CloudWise-Lukemiao:openapi-rest** into **0b2d0af48e00b6b72c5e4625e3a9613ac1c6204d on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/41949952/badge)](https://coveralls.io/builds/41949952)
   
   Coverage increased (+0.07%) to 68.29% when pulling **f57dc9aba4ce4c4ad55077a931c3c7177e84f9f5 on CloudWise-Lukemiao:openapi-rest** into **0b2d0af48e00b6b72c5e4625e3a9613ac1c6204d on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] CloudWise-Lukemiao commented on a change in pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
CloudWise-Lukemiao commented on a change in pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#discussion_r684264900



##########
File path: openapi/java-rest/pom.xml
##########
@@ -0,0 +1,597 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>openapi</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>java-rest</artifactId>
+    <!--<packaging>war</packaging>-->
+    <dependencies>
+        <dependency>
+            <groupId>org.xerial.snappy</groupId>
+            <artifactId>snappy-java</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+            <version>9.4.35.v20201120</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <profiles>
+        <profile>
+            <id>unix</id>
+            <activation>
+                <os>
+                    <family>unix</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>linux-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>
+                    <artifactId>jackson-jaxrs-json-provider</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <!-- Base64 encoding that works in both JVM and Android -->
+                <dependency>
+                    <groupId>com.brsanthu</groupId>
+                    <artifactId>migbase64</artifactId>
+                    <version>2.2</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                    <version>3.5.1</version>
+                </dependency>
+                <!-- Bean Validation API support -->
+                <!--             <dependency>
+                    <groupId>javax.validation</groupId>
+                    <artifactId>validation-api</artifactId>
+                    <version>${beanvalidation-version}</version>
+                    <scope>provided</scope>
+                </dependency>-->
+            </dependencies>
+        </profile>
+        <profile>
+            <id>mac</id>
+            <activation>
+                <os>
+                    <family>mac</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>osx-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>
+                    <artifactId>jackson-jaxrs-json-provider</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <!-- Base64 encoding that works in both JVM and Android -->
+                <dependency>
+                    <groupId>com.brsanthu</groupId>
+                    <artifactId>migbase64</artifactId>
+                    <version>2.2</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                    <version>3.5.1</version>
+                </dependency>
+                <!-- Bean Validation API support -->

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/41937365/badge)](https://coveralls.io/builds/41937365)
   
   Coverage increased (+0.06%) to 68.282% when pulling **f050e20a6081a1e86ccdcdcc1d83f6606dee9b32 on CloudWise-Lukemiao:openapi-rest** into **0b2d0af48e00b6b72c5e4625e3a9613ac1c6204d on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] CloudWise-Lukemiao commented on a change in pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
CloudWise-Lukemiao commented on a change in pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#discussion_r684263593



##########
File path: openapi/java-rest/src/main/java/org/apache/iotdb/openapi/gen/handler/impl/ProtobufMessageBodyWriter.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.openapi.gen.handler.impl;
+
+import com.google.protobuf.Message;
+
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyWriter;
+import javax.ws.rs.ext.Provider;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+import java.util.Map;
+import java.util.WeakHashMap;
+
+@Provider
+@Produces("application/x-protobuf")
+public class ProtobufMessageBodyWriter implements MessageBodyWriter<Message> {
+
+  @Override
+  public boolean isWriteable(
+      Class<?> aClass, Type type, Annotation[] annotations, MediaType mediaType) {
+    return Message.class.isAssignableFrom(aClass);
+    // return false;

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] CloudWise-Lukemiao commented on a change in pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
CloudWise-Lukemiao commented on a change in pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#discussion_r684263870



##########
File path: openapi/java-rest/src/main/java/org/apache/iotdb/openapi/gen/handler/impl/V1ApiServiceImpl.java
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// this file is generated by openapi generator
+
+package org.apache.iotdb.openapi.gen.handler.impl;
+
+import org.apache.iotdb.openapi.gen.handler.*;
+import org.apache.iotdb.openapi.gen.handler.NotFoundException;
+import org.apache.iotdb.openapi.gen.model.*;
+import org.apache.iotdb.openapi.gen.model.GroupByFillPlan;
+
+import javax.validation.constraints.*;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+
+import java.util.List;
+
+@javax.annotation.Generated(
+    value = "org.openapitools.codegen.languages.JavaJerseyServerCodegen",
+    date = "2021-01-13T21:45:03.765+08:00[Asia/Shanghai]")
+public class V1ApiServiceImpl extends V1ApiService {
+  @Override
+  public Response postV1GrafanaData(
+      GroupByFillPlan groupByFillPlan, SecurityContext securityContext) throws NotFoundException {
+    // do some magic!
+    return Response.ok().entity(new ApiResponseMessage(ApiResponseMessage.OK, "magic!")).build();
+  }
+
+  @Override
+  public Response postV1GrafanaDataSimplejson(
+      GroupByFillPlan groupByFillPlan, SecurityContext securityContext) throws NotFoundException {
+    // do some magic!

Review comment:
       fixed

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
##########
@@ -729,6 +729,100 @@
 
   private String adminPassword = "root";
 
+  /** if the startOpenApi is true, we will start OpenApi */
+  private boolean startOpenApi = true;
+
+  /** set the OpenApi reset port. */
+  private int OpenApiPort = 18080;
+
+  /** set the OpenApi storegroup number. */
+  private int sgCount = 5;
+
+  /** enable the OpenApi ssl. */
+  private boolean enable_https = false;
+  /** openapi ssl key Store Path */

Review comment:
       fixed

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
##########
@@ -729,6 +729,100 @@
 
   private String adminPassword = "root";
 
+  /** if the startOpenApi is true, we will start OpenApi */
+  private boolean startOpenApi = true;

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] qiaojialin commented on a change in pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#discussion_r683102792



##########
File path: openapi/java-rest/pom.xml
##########
@@ -0,0 +1,597 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>openapi</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>java-rest</artifactId>
+    <!--<packaging>war</packaging>-->
+    <dependencies>
+        <dependency>
+            <groupId>org.xerial.snappy</groupId>
+            <artifactId>snappy-java</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+            <version>9.4.35.v20201120</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <profiles>
+        <profile>
+            <id>unix</id>
+            <activation>
+                <os>
+                    <family>unix</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>linux-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>
+                    <artifactId>jackson-jaxrs-json-provider</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <!-- Base64 encoding that works in both JVM and Android -->
+                <dependency>
+                    <groupId>com.brsanthu</groupId>
+                    <artifactId>migbase64</artifactId>
+                    <version>2.2</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                    <version>3.5.1</version>
+                </dependency>
+                <!-- Bean Validation API support -->
+                <!--             <dependency>
+                    <groupId>javax.validation</groupId>
+                    <artifactId>validation-api</artifactId>
+                    <version>${beanvalidation-version}</version>
+                    <scope>provided</scope>
+                </dependency>-->
+            </dependencies>
+        </profile>
+        <profile>
+            <id>mac</id>
+            <activation>
+                <os>
+                    <family>mac</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>osx-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>

Review comment:
       please check if the license is forbidden by apache.
   http://www.apache.org/legal/resolved.html#category-x

##########
File path: openapi/java-rest/src/main/java/org/apache/iotdb/openapi/gen/handler/factories/V1ApiServiceFactory.java
##########
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// this file is generated by openapi generator
+
+package org.apache.iotdb.openapi.gen.handler.factories;
+
+import org.apache.iotdb.openapi.gen.handler.V1ApiService;
+import org.apache.iotdb.openapi.gen.handler.impl.V1ApiServiceImpl;
+
+@javax.annotation.Generated(

Review comment:
       is this needed?

##########
File path: openapi/java-rest/pom.xml
##########
@@ -0,0 +1,597 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>openapi</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>java-rest</artifactId>
+    <!--<packaging>war</packaging>-->
+    <dependencies>
+        <dependency>
+            <groupId>org.xerial.snappy</groupId>
+            <artifactId>snappy-java</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+            <version>9.4.35.v20201120</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <profiles>
+        <profile>
+            <id>unix</id>
+            <activation>
+                <os>
+                    <family>unix</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>linux-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>
+                    <artifactId>jackson-jaxrs-json-provider</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <!-- Base64 encoding that works in both JVM and Android -->
+                <dependency>
+                    <groupId>com.brsanthu</groupId>
+                    <artifactId>migbase64</artifactId>
+                    <version>2.2</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                    <version>3.5.1</version>
+                </dependency>
+                <!-- Bean Validation API support -->
+                <!--             <dependency>
+                    <groupId>javax.validation</groupId>
+                    <artifactId>validation-api</artifactId>
+                    <version>${beanvalidation-version}</version>
+                    <scope>provided</scope>
+                </dependency>-->
+            </dependencies>
+        </profile>
+        <profile>
+            <id>mac</id>
+            <activation>
+                <os>
+                    <family>mac</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>osx-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>

Review comment:
       check license

##########
File path: server/src/main/java/org/apache/iotdb/openapi/gen/handler/impl/V1ApiServiceImpl.java
##########
@@ -0,0 +1,1025 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// this file is generated by openapi generator
+
+package org.apache.iotdb.openapi.gen.handler.impl;
+
+import org.apache.iotdb.db.auth.AuthException;
+import org.apache.iotdb.db.auth.AuthorityChecker;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.Planner;
+import org.apache.iotdb.db.qp.executor.PlanExecutor;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.GroupByTimePlan;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowChildPathsPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowPlan.ShowContentType;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.executor.QueryRouter;
+import org.apache.iotdb.openapi.gen.handler.NotFoundException;
+import org.apache.iotdb.openapi.gen.handler.V1ApiService;
+import org.apache.iotdb.openapi.gen.model.GroupByFillPlan;
+import org.apache.iotdb.openapi.gen.model.ReadData;
+import org.apache.iotdb.openapi.gen.model.WriteData;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+
+import com.google.common.base.Joiner;
+import com.google.gson.Gson;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ProtocolStringList;
+import org.apache.commons.lang3.StringUtils;
+import org.xerial.snappy.Snappy;
+import prometheus.Remote;
+import prometheus.Remote.Query;
+import prometheus.Remote.ReadRequest;
+import prometheus.Remote.ReadResponse;
+import prometheus.Types;
+import prometheus.Types.Label;
+import prometheus.Types.LabelMatcher;
+import prometheus.Types.Sample;
+import prometheus.Types.TimeSeries;
+import prometheus.Types.TimeSeries.Builder;
+
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@javax.annotation.Generated(
+    value = "org.openapitools.codegen.languages.JavaJerseyServerCodegen",
+    date = "2021-01-13T21:45:03.765+08:00[Asia/Shanghai]")
+public class V1ApiServiceImpl extends V1ApiService {

Review comment:
       why  V1?

##########
File path: openapi/java-rest/src/main/java/org/apache/iotdb/openapi/gen/handler/impl/ProtobufMessageBodyWriter.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.openapi.gen.handler.impl;
+
+import com.google.protobuf.Message;
+
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyWriter;
+import javax.ws.rs.ext.Provider;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+import java.util.Map;
+import java.util.WeakHashMap;
+
+@Provider
+@Produces("application/x-protobuf")
+public class ProtobufMessageBodyWriter implements MessageBodyWriter<Message> {
+
+  @Override
+  public boolean isWriteable(
+      Class<?> aClass, Type type, Annotation[] annotations, MediaType mediaType) {
+    return Message.class.isAssignableFrom(aClass);
+    // return false;

Review comment:
       remove 

##########
File path: openapi/java-rest/src/main/java/org/apache/iotdb/openapi/gen/handler/impl/ProtobufMessageBodyWriter.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.openapi.gen.handler.impl;
+
+import com.google.protobuf.Message;
+
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyWriter;
+import javax.ws.rs.ext.Provider;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+import java.util.Map;
+import java.util.WeakHashMap;
+
+@Provider
+@Produces("application/x-protobuf")
+public class ProtobufMessageBodyWriter implements MessageBodyWriter<Message> {
+
+  @Override
+  public boolean isWriteable(
+      Class<?> aClass, Type type, Annotation[] annotations, MediaType mediaType) {
+    return Message.class.isAssignableFrom(aClass);
+    // return false;
+  }
+
+  private Map<Object, byte[]> buffer = new WeakHashMap<Object, byte[]>();
+
+  @Override
+  public long getSize(
+      Message message, Class<?> aClass, Type type, Annotation[] annotations, MediaType mediaType) {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    try {
+      message.writeTo(baos);
+    } catch (IOException e) {
+      return -1;
+    }
+    byte[] bytes = baos.toByteArray();
+    buffer.put(message, bytes);
+    return bytes.length;
+    // return 0;

Review comment:
       remove

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
##########
@@ -729,6 +729,100 @@
 
   private String adminPassword = "root";
 
+  /** if the startOpenApi is true, we will start OpenApi */
+  private boolean startOpenApi = true;

Review comment:
       the default could be false, be consistent with iotdb-engine.properties

##########
File path: openapi/java-rest/pom.xml
##########
@@ -0,0 +1,597 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>openapi</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>java-rest</artifactId>
+    <!--<packaging>war</packaging>-->
+    <dependencies>
+        <dependency>
+            <groupId>org.xerial.snappy</groupId>
+            <artifactId>snappy-java</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+            <version>9.4.35.v20201120</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <profiles>
+        <profile>
+            <id>unix</id>
+            <activation>
+                <os>
+                    <family>unix</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>linux-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>
+                    <artifactId>jackson-jaxrs-json-provider</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <!-- Base64 encoding that works in both JVM and Android -->
+                <dependency>
+                    <groupId>com.brsanthu</groupId>
+                    <artifactId>migbase64</artifactId>
+                    <version>2.2</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                    <version>3.5.1</version>
+                </dependency>
+                <!-- Bean Validation API support -->
+                <!--             <dependency>
+                    <groupId>javax.validation</groupId>
+                    <artifactId>validation-api</artifactId>
+                    <version>${beanvalidation-version}</version>
+                    <scope>provided</scope>
+                </dependency>-->
+            </dependencies>
+        </profile>
+        <profile>
+            <id>mac</id>
+            <activation>
+                <os>
+                    <family>mac</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>osx-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>
+                    <artifactId>jackson-jaxrs-json-provider</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <!-- Base64 encoding that works in both JVM and Android -->
+                <dependency>
+                    <groupId>com.brsanthu</groupId>
+                    <artifactId>migbase64</artifactId>
+                    <version>2.2</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                    <version>3.5.1</version>
+                </dependency>
+                <!-- Bean Validation API support -->

Review comment:
       remove unused dependency

##########
File path: openapi/java-rest/src/main/java/org/apache/iotdb/openapi/gen/handler/impl/V1ApiServiceImpl.java
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// this file is generated by openapi generator
+
+package org.apache.iotdb.openapi.gen.handler.impl;
+
+import org.apache.iotdb.openapi.gen.handler.*;
+import org.apache.iotdb.openapi.gen.handler.NotFoundException;
+import org.apache.iotdb.openapi.gen.model.*;
+import org.apache.iotdb.openapi.gen.model.GroupByFillPlan;
+
+import javax.validation.constraints.*;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+
+import java.util.List;
+
+@javax.annotation.Generated(
+    value = "org.openapitools.codegen.languages.JavaJerseyServerCodegen",
+    date = "2021-01-13T21:45:03.765+08:00[Asia/Shanghai]")
+public class V1ApiServiceImpl extends V1ApiService {
+  @Override
+  public Response postV1GrafanaData(
+      GroupByFillPlan groupByFillPlan, SecurityContext securityContext) throws NotFoundException {
+    // do some magic!
+    return Response.ok().entity(new ApiResponseMessage(ApiResponseMessage.OK, "magic!")).build();
+  }
+
+  @Override
+  public Response postV1GrafanaDataSimplejson(
+      GroupByFillPlan groupByFillPlan, SecurityContext securityContext) throws NotFoundException {
+    // do some magic!

Review comment:
       remove

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
##########
@@ -729,6 +729,100 @@
 
   private String adminPassword = "root";
 
+  /** if the startOpenApi is true, we will start OpenApi */
+  private boolean startOpenApi = true;
+
+  /** set the OpenApi reset port. */
+  private int OpenApiPort = 18080;
+
+  /** set the OpenApi storegroup number. */
+  private int sgCount = 5;
+
+  /** enable the OpenApi ssl. */
+  private boolean enable_https = false;
+  /** openapi ssl key Store Path */

Review comment:
       add an empty line




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/42058478/badge)](https://coveralls.io/builds/42058478)
   
   Coverage decreased (-1.0%) to 67.263% when pulling **b7d050c2550776884ef1d3e6ae2f3543ea02b29d on CloudWise-Lukemiao:openapi-rest** into **0b2d0af48e00b6b72c5e4625e3a9613ac1c6204d on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/42058878/badge)](https://coveralls.io/builds/42058878)
   
   Coverage decreased (-1.0%) to 67.248% when pulling **b7d050c2550776884ef1d3e6ae2f3543ea02b29d on CloudWise-Lukemiao:openapi-rest** into **0b2d0af48e00b6b72c5e4625e3a9613ac1c6204d on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] CloudWise-Lukemiao commented on a change in pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
CloudWise-Lukemiao commented on a change in pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#discussion_r684264726



##########
File path: openapi/java-rest/pom.xml
##########
@@ -0,0 +1,597 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>openapi</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>java-rest</artifactId>
+    <!--<packaging>war</packaging>-->
+    <dependencies>
+        <dependency>
+            <groupId>org.xerial.snappy</groupId>
+            <artifactId>snappy-java</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+            <version>9.4.35.v20201120</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <profiles>
+        <profile>
+            <id>unix</id>
+            <activation>
+                <os>
+                    <family>unix</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>linux-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>
+                    <artifactId>jackson-jaxrs-json-provider</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <!-- Base64 encoding that works in both JVM and Android -->
+                <dependency>
+                    <groupId>com.brsanthu</groupId>
+                    <artifactId>migbase64</artifactId>
+                    <version>2.2</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                    <version>3.5.1</version>
+                </dependency>
+                <!-- Bean Validation API support -->
+                <!--             <dependency>
+                    <groupId>javax.validation</groupId>
+                    <artifactId>validation-api</artifactId>
+                    <version>${beanvalidation-version}</version>
+                    <scope>provided</scope>
+                </dependency>-->
+            </dependencies>
+        </profile>
+        <profile>
+            <id>mac</id>
+            <activation>
+                <os>
+                    <family>mac</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>osx-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>

Review comment:
       apache license

##########
File path: openapi/java-rest/pom.xml
##########
@@ -0,0 +1,597 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>openapi</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>java-rest</artifactId>
+    <!--<packaging>war</packaging>-->
+    <dependencies>
+        <dependency>
+            <groupId>org.xerial.snappy</groupId>
+            <artifactId>snappy-java</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+            <version>9.4.35.v20201120</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <profiles>
+        <profile>
+            <id>unix</id>
+            <activation>
+                <os>
+                    <family>unix</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>linux-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>
+                    <artifactId>swagger-jersey2-jaxrs</artifactId>
+                    <scope>compile</scope>
+                    <version>${swagger-core-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-core</artifactId>
+                    <version>${logback-version}</version>
+                    <scope>compile</scope>
+                </dependency>
+                <dependency>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                    <version>${junit-version}</version>
+                    <scope>test</scope>
+                </dependency>
+                <dependency>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                    <version>${servlet-api-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.datatype</groupId>
+                    <artifactId>jackson-datatype-jsr310</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.fasterxml.jackson.jaxrs</groupId>
+                    <artifactId>jackson-jaxrs-json-provider</artifactId>
+                    <version>${jackson-version}</version>
+                </dependency>
+                <!-- Base64 encoding that works in both JVM and Android -->
+                <dependency>
+                    <groupId>com.brsanthu</groupId>
+                    <artifactId>migbase64</artifactId>
+                    <version>2.2</version>
+                </dependency>
+                <dependency>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                    <version>3.5.1</version>
+                </dependency>
+                <!-- Bean Validation API support -->
+                <!--             <dependency>
+                    <groupId>javax.validation</groupId>
+                    <artifactId>validation-api</artifactId>
+                    <version>${beanvalidation-version}</version>
+                    <scope>provided</scope>
+                </dependency>-->
+            </dependencies>
+        </profile>
+        <profile>
+            <id>mac</id>
+            <activation>
+                <os>
+                    <family>mac</family>
+                </os>
+            </activation>
+            <properties>
+                <os.detected.classifier>osx-x86_64</os.detected.classifier>
+                <swagger-core-version>1.5.18</swagger-core-version>
+                <beanvalidation-version>1.1.0.Final</beanvalidation-version>
+                <jetty-version>${jetty.version}</jetty-version>
+                <jersey2-version>2.22.2</jersey2-version>
+                <jackson-version>${jackson.version}</jackson-version>
+                <junit-version>${junit.version}</junit-version>
+                <logback-version>${logback.version}</logback-version>
+                <servlet-api-version>2.5</servlet-api-version>
+                <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.xolstice.maven.plugins</groupId>
+                        <artifactId>protobuf-maven-plugin</artifactId>
+                        <version>0.5.1</version>
+                        <extensions>true</extensions>
+                        <configuration>
+                            <protoSourceRoot>${project.basedir}/src/main/proto</protoSourceRoot>
+                            <protocArtifact>com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier}</protocArtifact>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>compile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-war-plugin</artifactId>
+                        <version>3.1.0</version>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-failsafe-plugin</artifactId>
+                        <version>2.6</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>integration-test</goal>
+                                    <goal>verify</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.eclipse.jetty</groupId>
+                        <artifactId>jetty-maven-plugin</artifactId>
+                        <version>${jetty-version}</version>
+                        <configuration>
+                            <webApp>
+                                <contextPath>/</contextPath>
+                            </webApp>
+                            <webAppSourceDirectory>target/${project.artifactId}-${project.version}</webAppSourceDirectory>
+                            <stopPort>8079</stopPort>
+                            <stopKey>stopit</stopKey>
+                            <stopWait>10</stopWait>
+                            <httpConnector>
+                                <port>5667</port>
+                                <idleTimeout>60000</idleTimeout>
+                            </httpConnector>
+                        </configuration>
+                        <dependencies>
+                            <dependency>
+                                <groupId>javax.validation</groupId>
+                                <artifactId>validation-api</artifactId>
+                                <version>${beanvalidation-version}</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>start-jetty</id>
+                                <phase>pre-integration-test</phase>
+                                <goals>
+                                    <goal>start</goal>
+                                </goals>
+                                <configuration>
+                                    <scanIntervalSeconds>0</scanIntervalSeconds>
+                                    <daemon>true</daemon>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>stop-jetty</id>
+                                <phase>post-integration-test</phase>
+                                <goals>
+                                    <goal>stop</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.9.1</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>${project.basedir}/../target/generated-sources/java/src/gen/java</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+            <dependencies>
+                <dependency>
+                    <groupId>io.swagger</groupId>

Review comment:
       apache license




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/42057541/badge)](https://coveralls.io/builds/42057541)
   
   Coverage decreased (-1.0%) to 67.259% when pulling **b7d050c2550776884ef1d3e6ae2f3543ea02b29d on CloudWise-Lukemiao:openapi-rest** into **0b2d0af48e00b6b72c5e4625e3a9613ac1c6204d on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] CloudWise-Lukemiao commented on a change in pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
CloudWise-Lukemiao commented on a change in pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#discussion_r686580472



##########
File path: server/src/test/java/org/apache/iotdb/openapi/apache/iotdb/OpenApiServiceTest.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.openapi.apache.iotdb;
+
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.openapi.gen.handler.NotFoundException;
+import org.apache.iotdb.openapi.gen.handler.filter.BasicSecurityContext;
+import org.apache.iotdb.openapi.gen.handler.impl.RestApiServiceImpl;
+import org.apache.iotdb.openapi.gen.handler.model.User;
+import org.apache.iotdb.openapi.gen.model.GroupByFillPlan;
+import org.apache.iotdb.openapi.gen.model.ReadData;
+import org.apache.iotdb.openapi.gen.model.WriteData;
+
+import com.google.gson.Gson;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.ws.rs.core.Response;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class OpenApiServiceTest {
+  GroupByFillPlan groupByFillPlan = new GroupByFillPlan();
+  User user = new User();
+  BasicSecurityContext basicSecurityContext = new BasicSecurityContext(user, false);
+  RestApiServiceImpl v1ApiService = new RestApiServiceImpl();
+
+  @Before
+  public void setUp() {
+    EnvironmentUtils.envSetUp();
+    user.setUsername("root");
+    user.setPassword("root");
+    BigDecimal stime = BigDecimal.valueOf(1);
+    BigDecimal etime = BigDecimal.valueOf(10);
+    List<String> path = new ArrayList<String>();
+    path.add("root");
+    path.add("sg");
+    path.add("aa");
+    groupByFillPlan.setStime(stime);
+    groupByFillPlan.setEtime(etime);
+    groupByFillPlan.setPaths(path);
+    // groupByFillPlan.set("1ms");

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/42058691/badge)](https://coveralls.io/builds/42058691)
   
   Coverage decreased (-0.9%) to 67.273% when pulling **b7d050c2550776884ef1d3e6ae2f3543ea02b29d on CloudWise-Lukemiao:openapi-rest** into **0b2d0af48e00b6b72c5e4625e3a9613ac1c6204d on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #3644: iotdb add openapi

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3644:
URL: https://github.com/apache/iotdb/pull/3644#issuecomment-888252635


   
   [![Coverage Status](https://coveralls.io/builds/42058460/badge)](https://coveralls.io/builds/42058460)
   
   Coverage decreased (-0.9%) to 67.281% when pulling **b7d050c2550776884ef1d3e6ae2f3543ea02b29d on CloudWise-Lukemiao:openapi-rest** into **0b2d0af48e00b6b72c5e4625e3a9613ac1c6204d on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org