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/10/07 04:18:47 UTC

[GitHub] [iotdb] jun0315 opened a new pull request #4081: [IOTDB-1797]Compatibility of Apache IoTDB with InfluxDB——Frame construction

jun0315 opened a new pull request #4081:
URL: https://github.com/apache/iotdb/pull/4081


   complete InfluxDB protocol frame code


-- 
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 commented on a change in pull request #4081: [IOTDB-1797]Compatibility of Apache IoTDB with InfluxDB——Frame construction

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



##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String url, String userName, String password) {
+    try {
+      URI uri = new URI(url);
+      new IoTDBInfluxDB(uri.getHost(), uri.getPort(), userName, password);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }

Review comment:
       ```suggestion
       URI uri = new URI(url);
       this(uri.getHost(), uri.getPort(), userName, password);
   ```

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */

Review comment:
       Comments here are meaningless...

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String url, String userName, String password) {
+    try {
+      URI uri = new URI(url);
+      new IoTDBInfluxDB(uri.getHost(), uri.getPort(), userName, password);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  /**
+   * constructor function
+   *
+   * @param host host
+   * @param rpcPort port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String host, int rpcPort, String userName, String password) {
+    session = new Session(host, rpcPort, userName, password);
+    try {
+      session.open(false);
+    } catch (IoTDBConnectionException e) {
+      throw new IllegalArgumentException(e.getMessage());

Review comment:
       IoTDBConnectionException and IllegalArgumentException are not the same. Handling IoTDBConnectionException in this way is not elegant. 

##########
File path: influxdb-protocol/src/test/java/org/apache/iotdb/influxdb/integration/IoTDBInfluxDBIT.java
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.influxdb.integration;
+
+import org.apache.iotdb.influxdb.IoTDBInfluxDBFactory;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.InfluxDB;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.testcontainers.containers.GenericContainer;
+
+public class IoTDBInfluxDBIT {
+  private String host;
+  private Integer port;
+  private String username;
+  private String password;
+
+  @Rule
+  public GenericContainer IotDB =
+      new GenericContainer("apache/iotdb:latest").withExposedPorts(6667);
+
+  @Before
+  public void setUp() {
+    host = IotDB.getContainerIpAddress();
+    port = IotDB.getMappedPort(6667);
+    username = "root";
+    password = "root";
+  }
+
+  @Test
+  public void testConnect1() {
+    InfluxDB influxDB =
+        IoTDBInfluxDBFactory.connect("https://" + host + ":" + port, username, password);
+    influxDB.close();

Review comment:
       ```suggestion
       IoTDBInfluxDBFactory.connect("https://" + host + ":" + port, username, password).close();
   ```

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String url, String userName, String password) {
+    try {
+      URI uri = new URI(url);
+      new IoTDBInfluxDB(uri.getHost(), uri.getPort(), userName, password);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  /**
+   * constructor function
+   *
+   * @param host host
+   * @param rpcPort port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String host, int rpcPort, String userName, String password) {
+    session = new Session(host, rpcPort, userName, password);
+    try {
+      session.open(false);
+    } catch (IoTDBConnectionException e) {
+      throw new IllegalArgumentException(e.getMessage());
+    }
+    session.setFetchSize(10000);
+  }
+
+  /**
+   * constructor function
+   *
+   * @param s session
+   */
+  public IoTDBInfluxDB(Session s) {
+    session = s;
+  }
+
+  /**
+   * constructor function
+   *
+   * @param builder session builder
+   */
+  public IoTDBInfluxDB(Session.Builder builder) {
+    session = builder.build();
+    session.setFetchSize(10000);
+  }
+
+  /**
+   * write function compatible with influxdb
+   *
+   * @param point Data structure for inserting data
+   */

Review comment:
       These comments are unnecessary because the method in the interface `InfluxDB` already has detailed descriptions. 

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String url, String userName, String password) {
+    try {
+      URI uri = new URI(url);
+      new IoTDBInfluxDB(uri.getHost(), uri.getPort(), userName, password);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  /**
+   * constructor function
+   *
+   * @param host host
+   * @param rpcPort port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String host, int rpcPort, String userName, String password) {
+    session = new Session(host, rpcPort, userName, password);
+    try {
+      session.open(false);
+    } catch (IoTDBConnectionException e) {
+      throw new IllegalArgumentException(e.getMessage());
+    }
+    session.setFetchSize(10000);
+  }
+
+  /**
+   * constructor function
+   *
+   * @param s session
+   */
+  public IoTDBInfluxDB(Session s) {
+    session = s;
+  }
+
+  /**
+   * constructor function
+   *
+   * @param builder session builder
+   */
+  public IoTDBInfluxDB(Session.Builder builder) {
+    session = builder.build();
+    session.setFetchSize(10000);
+  }
+
+  /**
+   * write function compatible with influxdb
+   *
+   * @param point Data structure for inserting data
+   */
+  @Override
+  public void write(final Point point) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  /**
+   * query function compatible with influxdb
+   *
+   * @param query query parameters of influxdb, including databasename and SQL statement
+   * @return returns the query result of influxdb
+   */
+  @Override
+  public QueryResult query(final Query query) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  /**
+   * create database,write to iotdb
+   *
+   * @param name database name
+   */
+  @Override
+  public void createDatabase(final String name) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  /**
+   * delete database
+   *
+   * @param name database name
+   */
+  @Override
+  public void deleteDatabase(final String name) {
+
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  /**
+   * set database,and get the list and order of all tags corresponding to the database
+   *
+   * @param database database name
+   */
+  @Override
+  public InfluxDB setDatabase(final String database) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(final String records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(final List<String> records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(final String database, final String retentionPolicy, final Point point) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(final int udpPort, final Point point) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(final BatchPoints batchPoints) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void writeWithRetry(final BatchPoints batchPoints) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(
+      final String database,
+      final String retentionPolicy,
+      final ConsistencyLevel consistency,
+      final String records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(
+      final String database,
+      final String retentionPolicy,
+      final ConsistencyLevel consistency,
+      final TimeUnit precision,
+      final String records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(
+      final String database,
+      final String retentionPolicy,
+      final ConsistencyLevel consistency,
+      final List<String> records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(
+      final String database,
+      final String retentionPolicy,
+      final ConsistencyLevel consistency,
+      final TimeUnit precision,
+      final List<String> records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(final int udpPort, final String records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(final int udpPort, final List<String> records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void query(
+      final Query query,
+      final Consumer<QueryResult> onSuccess,
+      final Consumer<Throwable> onFailure) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void query(Query query, int chunkSize, Consumer<QueryResult> onNext) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void query(Query query, int chunkSize, BiConsumer<Cancellable, QueryResult> onNext) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void query(Query query, int chunkSize, Consumer<QueryResult> onNext, Runnable onComplete) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void query(
+      Query query,
+      int chunkSize,
+      BiConsumer<Cancellable, QueryResult> onNext,
+      Runnable onComplete) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void query(
+      Query query,
+      int chunkSize,
+      BiConsumer<Cancellable, QueryResult> onNext,
+      Runnable onComplete,
+      Consumer<Throwable> onFailure) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public QueryResult query(Query query, TimeUnit timeUnit) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public List<String> describeDatabases() {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean databaseExists(final String name) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void flush() {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void close() {
+    try {
+      session.close();
+    } catch (IoTDBConnectionException e) {
+      throw new IllegalArgumentException(e.getMessage());

Review comment:
       IoTDBConnectionException and IllegalArgumentException are not the same. Handling IoTDBConnectionException in this way is not elegant.
   
   

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/example/InfluxDBExample.java
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.influxdb.example;
+
+import org.apache.iotdb.influxdb.IoTDBInfluxDBFactory;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.StatementExecutionException;
+
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class InfluxDBExample {
+  private static InfluxDB influxDB;
+
+  public static void main(String[] args) throws Exception {
+    // init
+    influxDB = IoTDBInfluxDBFactory.connect("http://127.0.0.1:6667", "root", "root");
+    // create database
+    influxDB.createDatabase("database");
+    // set database
+    influxDB.setDatabase("database");
+
+    insertData();
+    queryData();
+  }
+
+  // insert data
+  public static void insertData() throws IoTDBConnectionException, StatementExecutionException {
+
+    // insert the build parameter to construct the influxdb
+    Point.Builder builder = Point.measurement("student");
+    Map<String, String> tags = new HashMap<>();
+    Map<String, Object> fields = new HashMap<>();
+    tags.put("name", "xie");
+    tags.put("sex", "m");
+    fields.put("score", 87);
+    fields.put("tel", "110");
+    fields.put("country", "china");
+    builder.tag(tags);
+    builder.fields(fields);
+    builder.time(System.currentTimeMillis(), TimeUnit.MILLISECONDS);
+    Point point = builder.build();
+    // after the build construction is completed, start writing
+    influxDB.write(point);
+
+    builder = Point.measurement("student");
+    tags = new HashMap<>();
+    fields = new HashMap<>();
+    tags.put("name", "xie");
+    tags.put("sex", "m");
+    tags.put("province", "anhui");
+    fields.put("score", 99);
+    fields.put("country", "china");
+    builder.tag(tags);
+    builder.fields(fields);
+    builder.time(System.currentTimeMillis(), TimeUnit.MILLISECONDS);
+    point = builder.build();
+    influxDB.write(point);
+  }
+
+  // query data
+  private static void queryData() throws Exception {
+
+    Query query;
+    QueryResult result;
+
+    //     the selector query is parallel to the field value

Review comment:
       ```suggestion
       // the selector query is parallel to the field value
   ```

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String url, String userName, String password) {
+    try {
+      URI uri = new URI(url);
+      new IoTDBInfluxDB(uri.getHost(), uri.getPort(), userName, password);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  /**
+   * constructor function
+   *
+   * @param host host
+   * @param rpcPort port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String host, int rpcPort, String userName, String password) {
+    session = new Session(host, rpcPort, userName, password);
+    try {
+      session.open(false);
+    } catch (IoTDBConnectionException e) {
+      throw new IllegalArgumentException(e.getMessage());
+    }
+    session.setFetchSize(10000);

Review comment:
       This is not necessary, we already have a default fetch size.




-- 
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 #4081: [IOTDB-1797]Compatibility of Apache IoTDB with InfluxDB——Frame construction

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


   
   [![Coverage Status](https://coveralls.io/builds/43331441/badge)](https://coveralls.io/builds/43331441)
   
   Coverage decreased (-0.004%) to 67.777% when pulling **f34cc5e50fc200c3f12e07f54b0883d3686ac842 on jun0315:influxdb-frame** into **cbbdc6caf51660e5817a4e9c854831d820315b72 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 #4081: [IOTDB-1797]Compatibility of Apache IoTDB with InfluxDB——Frame construction

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


   
   [![Coverage Status](https://coveralls.io/builds/43331432/badge)](https://coveralls.io/builds/43331432)
   
   Coverage increased (+0.006%) to 67.787% when pulling **f34cc5e50fc200c3f12e07f54b0883d3686ac842 on jun0315:influxdb-frame** into **cbbdc6caf51660e5817a4e9c854831d820315b72 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 #4081: [IOTDB-1797]Compatibility of Apache IoTDB with InfluxDB——Frame construction

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






-- 
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 merged pull request #4081: [IOTDB-1797][IOTDB-1799] Compatibility of Apache IoTDB with InfluxDB - Intergration Framework

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


   


-- 
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] jun0315 commented on a change in pull request #4081: [IOTDB-1797] Compatibility of Apache IoTDB with InfluxDB - Intergration Framework

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



##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String url, String userName, String password) {
+    try {
+      URI uri = new URI(url);
+      new IoTDBInfluxDB(uri.getHost(), uri.getPort(), userName, password);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }

Review comment:
       Thanks for your review~
   
   If this() is called, it may only be called in this way
     
   ```java
   
   public IoTDBInfluxDB(String url, String userName, String password) throws URISyntaxException {
         this(new URI(url).getHost(), new URI(url).getPort(), userName, password);
     }
   ```
   Because this () can only be called on the first line, there can be no try/catch and URI generation.
   I think throwing mistakes may not be a good choice?




-- 
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 #4081: [IOTDB-1797]Compatibility of Apache IoTDB with InfluxDB——Frame construction

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


   
   [![Coverage Status](https://coveralls.io/builds/43331620/badge)](https://coveralls.io/builds/43331620)
   
   Coverage increased (+0.004%) to 67.784% when pulling **f34cc5e50fc200c3f12e07f54b0883d3686ac842 on jun0315:influxdb-frame** into **cbbdc6caf51660e5817a4e9c854831d820315b72 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 #4081: [IOTDB-1797]Compatibility of Apache IoTDB with InfluxDB——Frame construction

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


   
   [![Coverage Status](https://coveralls.io/builds/43331439/badge)](https://coveralls.io/builds/43331439)
   
   Coverage decreased (-0.01%) to 67.769% when pulling **f34cc5e50fc200c3f12e07f54b0883d3686ac842 on jun0315:influxdb-frame** into **cbbdc6caf51660e5817a4e9c854831d820315b72 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] SteveYurongSu commented on a change in pull request #4081: [IOTDB-1797]Compatibility of Apache IoTDB with InfluxDB——Frame construction

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



##########
File path: influxdb-protocol/pom.xml
##########
@@ -0,0 +1,127 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>iotdb-parent</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <artifactId>influxdb-protocol</artifactId>
+    <name>InfluxDB Protocol</name>
+    <description>compatible with the protocol of influxdb.</description>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-session</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.influxdb</groupId>
+            <artifactId>influxdb-java</artifactId>
+            <version>2.21</version>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>RELEASE</version>

Review comment:
       Use the same version as the parent pom.xml?

##########
File path: influxdb-protocol/pom.xml
##########
@@ -0,0 +1,127 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>iotdb-parent</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <artifactId>influxdb-protocol</artifactId>
+    <name>InfluxDB Protocol</name>
+    <description>compatible with the protocol of influxdb.</description>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-session</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.influxdb</groupId>
+            <artifactId>influxdb-java</artifactId>
+            <version>2.21</version>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>RELEASE</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.antlr</groupId>
+            <artifactId>antlr4-runtime</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>testcontainers</artifactId>
+            <version>LATEST</version>

Review comment:
       Use the same version as the parent pom.xml?
   
   

##########
File path: influxdb-protocol/pom.xml
##########
@@ -0,0 +1,127 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>iotdb-parent</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <artifactId>influxdb-protocol</artifactId>
+    <name>InfluxDB Protocol</name>
+    <description>compatible with the protocol of influxdb.</description>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-session</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.influxdb</groupId>
+            <artifactId>influxdb-java</artifactId>
+            <version>2.21</version>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>RELEASE</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.antlr</groupId>
+            <artifactId>antlr4-runtime</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>testcontainers</artifactId>
+            <version>LATEST</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.antlr</groupId>
+                <artifactId>antlr4-maven-plugin</artifactId>
+                <version>${antlr4.version}</version>
+                <executions>
+                    <execution>
+                        <configuration>
+                            <listener>false</listener>
+                            <visitor>true</visitor>
+                        </configuration>
+                        <goals>
+                            <goal>antlr4</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>3.2.0</version>

Review comment:
       Use the same version as the parent pom.xml?
   
   




-- 
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] jun0315 commented on a change in pull request #4081: [IOTDB-1797] Compatibility of Apache IoTDB with InfluxDB - Intergration Framework

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



##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String url, String userName, String password) {
+    try {
+      URI uri = new URI(url);
+      new IoTDBInfluxDB(uri.getHost(), uri.getPort(), userName, password);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }

Review comment:
       Thanks for your review~
   
   If this() is called, it may only be called in this way
     
   ```java
   
   public IoTDBInfluxDB(String url, String userName, String password) throws URISyntaxException {
         this(new URI(url).getHost(), new URI(url).getPort(), userName, password);
     }
   ```
   Because this () can only be called on the first line, there can be no try/catch and URI generation.
   I think throwing exception may not be a good choice?




-- 
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 commented on a change in pull request #4081: [IOTDB-1797]Compatibility of Apache IoTDB with InfluxDB——Frame construction

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



##########
File path: influxdb-protocol/pom.xml
##########
@@ -0,0 +1,127 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>iotdb-parent</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <artifactId>influxdb-protocol</artifactId>
+    <name>InfluxDB Protocol</name>
+    <description>compatible with the protocol of influxdb.</description>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-session</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.influxdb</groupId>
+            <artifactId>influxdb-java</artifactId>
+            <version>2.21</version>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>RELEASE</version>

Review comment:
       Use the same version as the parent pom.xml?

##########
File path: influxdb-protocol/pom.xml
##########
@@ -0,0 +1,127 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>iotdb-parent</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <artifactId>influxdb-protocol</artifactId>
+    <name>InfluxDB Protocol</name>
+    <description>compatible with the protocol of influxdb.</description>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-session</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.influxdb</groupId>
+            <artifactId>influxdb-java</artifactId>
+            <version>2.21</version>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>RELEASE</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.antlr</groupId>
+            <artifactId>antlr4-runtime</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>testcontainers</artifactId>
+            <version>LATEST</version>

Review comment:
       Use the same version as the parent pom.xml?
   
   

##########
File path: influxdb-protocol/pom.xml
##########
@@ -0,0 +1,127 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>iotdb-parent</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <artifactId>influxdb-protocol</artifactId>
+    <name>InfluxDB Protocol</name>
+    <description>compatible with the protocol of influxdb.</description>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-session</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.influxdb</groupId>
+            <artifactId>influxdb-java</artifactId>
+            <version>2.21</version>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>RELEASE</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.antlr</groupId>
+            <artifactId>antlr4-runtime</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>testcontainers</artifactId>
+            <version>LATEST</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.antlr</groupId>
+                <artifactId>antlr4-maven-plugin</artifactId>
+                <version>${antlr4.version}</version>
+                <executions>
+                    <execution>
+                        <configuration>
+                            <listener>false</listener>
+                            <visitor>true</visitor>
+                        </configuration>
+                        <goals>
+                            <goal>antlr4</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>3.2.0</version>

Review comment:
       Use the same version as the parent pom.xml?
   
   

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String url, String userName, String password) {
+    try {
+      URI uri = new URI(url);
+      new IoTDBInfluxDB(uri.getHost(), uri.getPort(), userName, password);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }

Review comment:
       ```suggestion
       URI uri = new URI(url);
       this(uri.getHost(), uri.getPort(), userName, password);
   ```

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */

Review comment:
       Comments here are meaningless...

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String url, String userName, String password) {
+    try {
+      URI uri = new URI(url);
+      new IoTDBInfluxDB(uri.getHost(), uri.getPort(), userName, password);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  /**
+   * constructor function
+   *
+   * @param host host
+   * @param rpcPort port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String host, int rpcPort, String userName, String password) {
+    session = new Session(host, rpcPort, userName, password);
+    try {
+      session.open(false);
+    } catch (IoTDBConnectionException e) {
+      throw new IllegalArgumentException(e.getMessage());

Review comment:
       IoTDBConnectionException and IllegalArgumentException are not the same. Handling IoTDBConnectionException in this way is not elegant. 

##########
File path: influxdb-protocol/src/test/java/org/apache/iotdb/influxdb/integration/IoTDBInfluxDBIT.java
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.influxdb.integration;
+
+import org.apache.iotdb.influxdb.IoTDBInfluxDBFactory;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.InfluxDB;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.testcontainers.containers.GenericContainer;
+
+public class IoTDBInfluxDBIT {
+  private String host;
+  private Integer port;
+  private String username;
+  private String password;
+
+  @Rule
+  public GenericContainer IotDB =
+      new GenericContainer("apache/iotdb:latest").withExposedPorts(6667);
+
+  @Before
+  public void setUp() {
+    host = IotDB.getContainerIpAddress();
+    port = IotDB.getMappedPort(6667);
+    username = "root";
+    password = "root";
+  }
+
+  @Test
+  public void testConnect1() {
+    InfluxDB influxDB =
+        IoTDBInfluxDBFactory.connect("https://" + host + ":" + port, username, password);
+    influxDB.close();

Review comment:
       ```suggestion
       IoTDBInfluxDBFactory.connect("https://" + host + ":" + port, username, password).close();
   ```

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String url, String userName, String password) {
+    try {
+      URI uri = new URI(url);
+      new IoTDBInfluxDB(uri.getHost(), uri.getPort(), userName, password);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  /**
+   * constructor function
+   *
+   * @param host host
+   * @param rpcPort port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String host, int rpcPort, String userName, String password) {
+    session = new Session(host, rpcPort, userName, password);
+    try {
+      session.open(false);
+    } catch (IoTDBConnectionException e) {
+      throw new IllegalArgumentException(e.getMessage());
+    }
+    session.setFetchSize(10000);
+  }
+
+  /**
+   * constructor function
+   *
+   * @param s session
+   */
+  public IoTDBInfluxDB(Session s) {
+    session = s;
+  }
+
+  /**
+   * constructor function
+   *
+   * @param builder session builder
+   */
+  public IoTDBInfluxDB(Session.Builder builder) {
+    session = builder.build();
+    session.setFetchSize(10000);
+  }
+
+  /**
+   * write function compatible with influxdb
+   *
+   * @param point Data structure for inserting data
+   */

Review comment:
       These comments are unnecessary because the method in the interface `InfluxDB` already has detailed descriptions. 

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String url, String userName, String password) {
+    try {
+      URI uri = new URI(url);
+      new IoTDBInfluxDB(uri.getHost(), uri.getPort(), userName, password);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  /**
+   * constructor function
+   *
+   * @param host host
+   * @param rpcPort port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String host, int rpcPort, String userName, String password) {
+    session = new Session(host, rpcPort, userName, password);
+    try {
+      session.open(false);
+    } catch (IoTDBConnectionException e) {
+      throw new IllegalArgumentException(e.getMessage());
+    }
+    session.setFetchSize(10000);
+  }
+
+  /**
+   * constructor function
+   *
+   * @param s session
+   */
+  public IoTDBInfluxDB(Session s) {
+    session = s;
+  }
+
+  /**
+   * constructor function
+   *
+   * @param builder session builder
+   */
+  public IoTDBInfluxDB(Session.Builder builder) {
+    session = builder.build();
+    session.setFetchSize(10000);
+  }
+
+  /**
+   * write function compatible with influxdb
+   *
+   * @param point Data structure for inserting data
+   */
+  @Override
+  public void write(final Point point) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  /**
+   * query function compatible with influxdb
+   *
+   * @param query query parameters of influxdb, including databasename and SQL statement
+   * @return returns the query result of influxdb
+   */
+  @Override
+  public QueryResult query(final Query query) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  /**
+   * create database,write to iotdb
+   *
+   * @param name database name
+   */
+  @Override
+  public void createDatabase(final String name) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  /**
+   * delete database
+   *
+   * @param name database name
+   */
+  @Override
+  public void deleteDatabase(final String name) {
+
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  /**
+   * set database,and get the list and order of all tags corresponding to the database
+   *
+   * @param database database name
+   */
+  @Override
+  public InfluxDB setDatabase(final String database) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(final String records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(final List<String> records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(final String database, final String retentionPolicy, final Point point) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(final int udpPort, final Point point) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(final BatchPoints batchPoints) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void writeWithRetry(final BatchPoints batchPoints) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(
+      final String database,
+      final String retentionPolicy,
+      final ConsistencyLevel consistency,
+      final String records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(
+      final String database,
+      final String retentionPolicy,
+      final ConsistencyLevel consistency,
+      final TimeUnit precision,
+      final String records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(
+      final String database,
+      final String retentionPolicy,
+      final ConsistencyLevel consistency,
+      final List<String> records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(
+      final String database,
+      final String retentionPolicy,
+      final ConsistencyLevel consistency,
+      final TimeUnit precision,
+      final List<String> records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(final int udpPort, final String records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void write(final int udpPort, final List<String> records) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void query(
+      final Query query,
+      final Consumer<QueryResult> onSuccess,
+      final Consumer<Throwable> onFailure) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void query(Query query, int chunkSize, Consumer<QueryResult> onNext) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void query(Query query, int chunkSize, BiConsumer<Cancellable, QueryResult> onNext) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void query(Query query, int chunkSize, Consumer<QueryResult> onNext, Runnable onComplete) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void query(
+      Query query,
+      int chunkSize,
+      BiConsumer<Cancellable, QueryResult> onNext,
+      Runnable onComplete) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void query(
+      Query query,
+      int chunkSize,
+      BiConsumer<Cancellable, QueryResult> onNext,
+      Runnable onComplete,
+      Consumer<Throwable> onFailure) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public QueryResult query(Query query, TimeUnit timeUnit) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public List<String> describeDatabases() {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public boolean databaseExists(final String name) {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void flush() {
+    throw new UnsupportedOperationException(METHOD_NOT_SUPPORTED);
+  }
+
+  @Override
+  public void close() {
+    try {
+      session.close();
+    } catch (IoTDBConnectionException e) {
+      throw new IllegalArgumentException(e.getMessage());

Review comment:
       IoTDBConnectionException and IllegalArgumentException are not the same. Handling IoTDBConnectionException in this way is not elegant.
   
   

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/example/InfluxDBExample.java
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.influxdb.example;
+
+import org.apache.iotdb.influxdb.IoTDBInfluxDBFactory;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.StatementExecutionException;
+
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class InfluxDBExample {
+  private static InfluxDB influxDB;
+
+  public static void main(String[] args) throws Exception {
+    // init
+    influxDB = IoTDBInfluxDBFactory.connect("http://127.0.0.1:6667", "root", "root");
+    // create database
+    influxDB.createDatabase("database");
+    // set database
+    influxDB.setDatabase("database");
+
+    insertData();
+    queryData();
+  }
+
+  // insert data
+  public static void insertData() throws IoTDBConnectionException, StatementExecutionException {
+
+    // insert the build parameter to construct the influxdb
+    Point.Builder builder = Point.measurement("student");
+    Map<String, String> tags = new HashMap<>();
+    Map<String, Object> fields = new HashMap<>();
+    tags.put("name", "xie");
+    tags.put("sex", "m");
+    fields.put("score", 87);
+    fields.put("tel", "110");
+    fields.put("country", "china");
+    builder.tag(tags);
+    builder.fields(fields);
+    builder.time(System.currentTimeMillis(), TimeUnit.MILLISECONDS);
+    Point point = builder.build();
+    // after the build construction is completed, start writing
+    influxDB.write(point);
+
+    builder = Point.measurement("student");
+    tags = new HashMap<>();
+    fields = new HashMap<>();
+    tags.put("name", "xie");
+    tags.put("sex", "m");
+    tags.put("province", "anhui");
+    fields.put("score", 99);
+    fields.put("country", "china");
+    builder.tag(tags);
+    builder.fields(fields);
+    builder.time(System.currentTimeMillis(), TimeUnit.MILLISECONDS);
+    point = builder.build();
+    influxDB.write(point);
+  }
+
+  // query data
+  private static void queryData() throws Exception {
+
+    Query query;
+    QueryResult result;
+
+    //     the selector query is parallel to the field value

Review comment:
       ```suggestion
       // the selector query is parallel to the field value
   ```

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String url, String userName, String password) {
+    try {
+      URI uri = new URI(url);
+      new IoTDBInfluxDB(uri.getHost(), uri.getPort(), userName, password);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  /**
+   * constructor function
+   *
+   * @param host host
+   * @param rpcPort port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String host, int rpcPort, String userName, String password) {
+    session = new Session(host, rpcPort, userName, password);
+    try {
+      session.open(false);
+    } catch (IoTDBConnectionException e) {
+      throw new IllegalArgumentException(e.getMessage());
+    }
+    session.setFetchSize(10000);

Review comment:
       This is not necessary, we already have a default fetch size.




-- 
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 #4081: [IOTDB-1797]Compatibility of Apache IoTDB with InfluxDB——Frame construction

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


   
   [![Coverage Status](https://coveralls.io/builds/43331432/badge)](https://coveralls.io/builds/43331432)
   
   Coverage increased (+0.006%) to 67.787% when pulling **f34cc5e50fc200c3f12e07f54b0883d3686ac842 on jun0315:influxdb-frame** into **cbbdc6caf51660e5817a4e9c854831d820315b72 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 #4081: [IOTDB-1797]Compatibility of Apache IoTDB with InfluxDB——Frame construction

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


   
   [![Coverage Status](https://coveralls.io/builds/43331781/badge)](https://coveralls.io/builds/43331781)
   
   Coverage increased (+0.005%) to 67.785% when pulling **f34cc5e50fc200c3f12e07f54b0883d3686ac842 on jun0315:influxdb-frame** into **cbbdc6caf51660e5817a4e9c854831d820315b72 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] jun0315 commented on a change in pull request #4081: [IOTDB-1797] Compatibility of Apache IoTDB with InfluxDB - Intergration Framework

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



##########
File path: influxdb-protocol/pom.xml
##########
@@ -0,0 +1,127 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>iotdb-parent</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <artifactId>influxdb-protocol</artifactId>
+    <name>InfluxDB Protocol</name>
+    <description>compatible with the protocol of influxdb.</description>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-session</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.influxdb</groupId>
+            <artifactId>influxdb-java</artifactId>
+            <version>2.21</version>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>RELEASE</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.antlr</groupId>
+            <artifactId>antlr4-runtime</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>testcontainers</artifactId>
+            <version>LATEST</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.antlr</groupId>
+                <artifactId>antlr4-maven-plugin</artifactId>
+                <version>${antlr4.version}</version>
+                <executions>
+                    <execution>
+                        <configuration>
+                            <listener>false</listener>
+                            <visitor>true</visitor>
+                        </configuration>
+                        <goals>
+                            <goal>antlr4</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>3.2.0</version>

Review comment:
       This is same with the pom of iotdb-antlr~
   
   

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String url, String userName, String password) {
+    try {
+      URI uri = new URI(url);
+      new IoTDBInfluxDB(uri.getHost(), uri.getPort(), userName, password);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }

Review comment:
       Thanks for your review~
   
   If this() is called, it may only be called in this way
     
   ```java
   
   public IoTDBInfluxDB(String url, String userName, String password) throws URISyntaxException {
         this(new URI(url).getHost(), new URI(url).getPort(), userName, password);
     }
   ```
   Because this () can only be called on the first line, there can be no try/catch and URI generation.
   I think throwing mistakes may not be a good choice?

##########
File path: influxdb-protocol/src/main/java/org/apache/iotdb/influxdb/IoTDBInfluxDB.java
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.influxdb;
+
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.session.Session;
+
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Pong;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+public class IoTDBInfluxDB implements InfluxDB {
+
+  static final String METHOD_NOT_SUPPORTED = "Method not supported";
+
+  private static Session session;
+
+  /**
+   * constructor function
+   *
+   * @param url contain host and port
+   * @param userName username
+   * @param password user password
+   */
+  public IoTDBInfluxDB(String url, String userName, String password) {
+    try {
+      URI uri = new URI(url);
+      new IoTDBInfluxDB(uri.getHost(), uri.getPort(), userName, password);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }

Review comment:
       Thanks for your review~
   
   If this() is called, it may only be called in this way
     
   ```java
   
   public IoTDBInfluxDB(String url, String userName, String password) throws URISyntaxException {
         this(new URI(url).getHost(), new URI(url).getPort(), userName, password);
     }
   ```
   Because this () can only be called on the first line, there can be no try/catch and URI generation.
   I think throwing exception may not be a good choice?




-- 
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] jun0315 commented on a change in pull request #4081: [IOTDB-1797] Compatibility of Apache IoTDB with InfluxDB - Intergration Framework

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



##########
File path: influxdb-protocol/pom.xml
##########
@@ -0,0 +1,127 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>iotdb-parent</artifactId>
+        <version>0.13.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <artifactId>influxdb-protocol</artifactId>
+    <name>InfluxDB Protocol</name>
+    <description>compatible with the protocol of influxdb.</description>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-session</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>iotdb-thrift</artifactId>
+            <version>0.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.influxdb</groupId>
+            <artifactId>influxdb-java</artifactId>
+            <version>2.21</version>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>RELEASE</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.antlr</groupId>
+            <artifactId>antlr4-runtime</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>testcontainers</artifactId>
+            <version>LATEST</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.antlr</groupId>
+                <artifactId>antlr4-maven-plugin</artifactId>
+                <version>${antlr4.version}</version>
+                <executions>
+                    <execution>
+                        <configuration>
+                            <listener>false</listener>
+                            <visitor>true</visitor>
+                        </configuration>
+                        <goals>
+                            <goal>antlr4</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>3.2.0</version>

Review comment:
       This is same with the pom of iotdb-antlr~
   
   




-- 
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