[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #381: use jdbc to connect iotdb in spark

2019-09-11 Thread GitBox
qiaojialin commented on a change in pull request #381: use jdbc to connect 
iotdb in spark
URL: https://github.com/apache/incubator-iotdb/pull/381#discussion_r323096786
 
 

 ##
 File path: 
spark-iotdb-connector/src/main/scala/org/apache/iotdb/tsfile/IoTDBRDD.scala
 ##
 @@ -0,0 +1,133 @@
+/**
+  * 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.tsfile
+
+import org.apache.spark.{Partition, SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.sources._
+import java.sql.{Connection, DriverManager, ResultSet, Statement}
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.types._
+
+
+//IoTDB data partition
+case class IoTDBPartition(where: String, id: Int, start: java.lang.Long, end: 
java.lang.Long) extends Partition {
+  override def index: Int = id
+}
+
+object IoTDBRDD {
+
+  private def pruneSchema(schema: StructType, columns: Array[String]): 
StructType = {
+val fieldMap = Map(schema.fields.map(x => x.name -> x): _*)
+new StructType(columns.map(name => fieldMap(name)))
+  }
+
+}
+
+class IoTDBRDD private[iotdb](
+   sc: SparkContext,
+   options: IoTDBOptions,
+   schema: StructType,
+   requiredColumns: Array[String],
+   filters: Array[Filter],
+   partitions: Array[Partition])
+  extends RDD[Row](sc, Nil) {
+
+  override def compute(split: Partition, context: TaskContext): Iterator[Row] 
= new Iterator[Row] {
+var finished = false
+var gotNext = false
+var nextValue: Row = _
+val inputMetrics = context.taskMetrics().inputMetrics
+
+val part = split.asInstanceOf[IoTDBPartition]
+
+var taskInfo: String = _
+Option(TaskContext.get()).foreach { taskContext => {
+  taskContext.addTaskCompletionListener { _ => conn.close() }
+  taskInfo = "task Id: " + taskContext.taskAttemptId() + " partition Id: " 
+ taskContext.partitionId()
+}
+}
+
+Class.forName("org.apache.iotdb.jdbc.IoTDBDriver")
+val conn: Connection = DriverManager.getConnection(options.url, 
options.user, options.password)
+val stmt: Statement = conn.createStatement()
+
+var sql = options.sql
+// for different partition
+if (part.where != null) {
+  val sqlPart = options.sql.split(SQLConstant.WHERE)
+  sql = sqlPart(0) + " " + SQLConstant.WHERE + " (" + part.where + ") "
+  if (sqlPart.length == 2) {
+sql += "and (" + sqlPart(1) + ")"
+  }
+}
+//
+var rs: ResultSet = stmt.executeQuery(sql)
+val prunedSchema = IoTDBRDD.pruneSchema(schema, requiredColumns)
+private val rowBuffer = Array.fill[Any](prunedSchema.length)(null)
+
+def getNext: Row = {
+  if (rs.next()) {
+val fields = new scala.collection.mutable.HashMap[String, String]()
 
 Review comment:
   is this map needed? could we only put result one by one?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #381: use jdbc to connect iotdb in spark

2019-09-11 Thread GitBox
qiaojialin commented on a change in pull request #381: use jdbc to connect 
iotdb in spark
URL: https://github.com/apache/incubator-iotdb/pull/381#discussion_r323094106
 
 

 ##
 File path: 
spark-iotdb-connector/src/main/scala/org/apache/iotdb/tsfile/IoTDBOptions.scala
 ##
 @@ -0,0 +1,49 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing,
+  * software distributed under the License is distributed on an
+  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  * KIND, either express or implied.  See the License for the
+  * specific language governing permissions and limitations
+  * under the License.
+  */
+package org.apache.iotdb.tsfile
+
+/**
+  * Created by qjl on 16-11-4.
+  */
+class IoTDBOptions(
+@transient private val parameters: Map[String, String])
+  extends Serializable {
+
+  val url = parameters.getOrElse("url", sys.error("Option 'url' not 
specified"))
+
+  val user = parameters.getOrElse("user", "root")
+
+  val password = parameters.getOrElse("password", "root")
+
+  val sql = parameters.getOrElse("sql", sys.error("Option 'sql' not 
specified"))
+
+  // deprecated:
 
 Review comment:
   why deprecated? since this is the first version, if they are not used, just 
remove them. Or remove this comment


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #381: use jdbc to connect iotdb in spark

2019-09-11 Thread GitBox
qiaojialin commented on a change in pull request #381: use jdbc to connect 
iotdb in spark
URL: https://github.com/apache/incubator-iotdb/pull/381#discussion_r323092670
 
 

 ##
 File path: spark-iotdb-connector/pom.xml
 ##
 @@ -0,0 +1,144 @@
+
+
+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";>
+4.0.0
+
+org.apache.iotdb
+iotdb-parent
+0.9.0-SNAPSHOT
+../pom.xml
+
+spark-iotdb-connector
+0.9.0-SNAPSHOT
+jar
+
+UTF-8
+2.6.5
+1.8
+
+
+
+
+
+
+org.json
+json
+20170516
+
+
+junit
+junit
+${junit.version}
+test
+
+
+org.apache.iotdb
+tsfile
+0.9.0-SNAPSHOT
+
 
 Review comment:
   iotdb-jdbc already contains tsfile, you can remove this


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #381: use jdbc to connect iotdb in spark

2019-09-11 Thread GitBox
qiaojialin commented on a change in pull request #381: use jdbc to connect 
iotdb in spark
URL: https://github.com/apache/incubator-iotdb/pull/381#discussion_r323094566
 
 

 ##
 File path: 
spark-iotdb-connector/src/main/scala/org/apache/iotdb/tsfile/IoTDBRelation.scala
 ##
 @@ -0,0 +1,120 @@
+/**
+  * 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.tsfile
+
+import org.apache.spark.Partition
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext, SparkSession}
+import org.apache.spark.sql.sources.{BaseRelation, PrunedFilteredScan}
+import org.apache.spark.sql.types.StructType
+import org.slf4j.LoggerFactory
+import org.apache.spark.sql.sources.Filter
+
+import scala.collection.mutable.ArrayBuffer
+
+/**
+  * Created by qjl on 16-8-25.
 
 Review comment:
   remove


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #381: use jdbc to connect iotdb in spark

2019-09-11 Thread GitBox
qiaojialin commented on a change in pull request #381: use jdbc to connect 
iotdb in spark
URL: https://github.com/apache/incubator-iotdb/pull/381#discussion_r323092868
 
 

 ##
 File path: 
spark-iotdb-connector/src/main/java/org/apache/iotdb/tsfile/SQLConstant.java
 ##
 @@ -0,0 +1,31 @@
+/**
+ * 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.tsfile;
+
+/**
+ * this class contains several constants used in SQL.
+ */
+public class SQLConstant {
+
+  public static final String NEED_NOT_TO_PRINT_TIMESTAMP = "AGGREGATION";
+  public static final String RESERVED_TIME = "time";
+  public static final String TIMESTAMP_STR = "Time"; // new added
 
 Review comment:
   remove the // new added


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #381: use jdbc to connect iotdb in spark

2019-09-11 Thread GitBox
qiaojialin commented on a change in pull request #381: use jdbc to connect 
iotdb in spark
URL: https://github.com/apache/incubator-iotdb/pull/381#discussion_r323093005
 
 

 ##
 File path: 
spark-iotdb-connector/src/main/scala/org/apache/iotdb/tsfile/Converter.scala
 ##
 @@ -0,0 +1,90 @@
+/**
+  * 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.tsfile
+
+import java.sql._
+
+import org.apache.spark.sql.types._
+import org.slf4j.LoggerFactory
+import java.sql.Statement
+
+import scala.collection.mutable.ListBuffer
+
+/**
+  * Created by qjl on 16-11-3.
 
 Review comment:
   remove this


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #381: use jdbc to connect iotdb in spark

2019-09-11 Thread GitBox
qiaojialin commented on a change in pull request #381: use jdbc to connect 
iotdb in spark
URL: https://github.com/apache/incubator-iotdb/pull/381#discussion_r323093562
 
 

 ##
 File path: 
spark-iotdb-connector/src/main/scala/org/apache/iotdb/tsfile/IoTDBOptions.scala
 ##
 @@ -0,0 +1,49 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing,
+  * software distributed under the License is distributed on an
+  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  * KIND, either express or implied.  See the License for the
+  * specific language governing permissions and limitations
+  * under the License.
+  */
+package org.apache.iotdb.tsfile
+
+/**
+  * Created by qjl on 16-11-4.
 
 Review comment:
   remove


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #381: use jdbc to connect iotdb in spark

2019-09-11 Thread GitBox
qiaojialin commented on a change in pull request #381: use jdbc to connect 
iotdb in spark
URL: https://github.com/apache/incubator-iotdb/pull/381#discussion_r323094513
 
 

 ##
 File path: 
spark-iotdb-connector/src/main/scala/org/apache/iotdb/tsfile/package.scala
 ##
 @@ -0,0 +1,39 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing,
+  * software distributed under the License is distributed on an
+  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  * KIND, either express or implied.  See the License for the
+  * specific language governing permissions and limitations
+  * under the License.
+  */
+package org.apache.iotdb
+
+/**
+  * Created by qjl on 16-9-5.
 
 Review comment:
   remove


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #381: use jdbc to connect iotdb in spark

2019-09-11 Thread GitBox
qiaojialin commented on a change in pull request #381: use jdbc to connect 
iotdb in spark
URL: https://github.com/apache/incubator-iotdb/pull/381#discussion_r323092401
 
 

 ##
 File path: spark-iotdb-connector/Readme.md
 ##
 @@ -0,0 +1,68 @@
+
+## version
+
+The versions required for Spark and Java are as follow:
+
+| Spark Version | Scala Version | Java Version | TsFile |
+| - | - |  | |
+| `2.4.3`| `2.11`| `1.8`| `0.9.0-SNAPSHOT`|
+
+
+## install
+mvn clean scala:compile compile install
+
+
+## maven dependency
+
+```
+
+  org.apache.iotdb
+  spark-iotdb-connector
+  0.9.0-SNAPSHOT
+
+```
+
+
+## spark-shell user guide
+
+```
+spark-shell --jars 
spark-iotdb-connector-0.9.0-SNAPSHOT.jar,tsfile-0.9.0-SNAPSHOT-jar-with-dependencies.jar,iotdb-jdbc-0.9.0-SNAPSHOT-jar-with-dependencies.jar
+
+val df = 
spark.read.format("org.apache.iotdb.tsfile").option("url","jdbc:iotdb://127.0.0.1:6667/").option("sql","select
 * from root").load
 
 Review comment:
   how to specify the table format?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #381: use jdbc to connect iotdb in spark

2019-09-11 Thread GitBox
qiaojialin commented on a change in pull request #381: use jdbc to connect 
iotdb in spark
URL: https://github.com/apache/incubator-iotdb/pull/381#discussion_r323093492
 
 

 ##
 File path: 
spark-iotdb-connector/src/main/scala/org/apache/iotdb/tsfile/DefaultSource.scala
 ##
 @@ -0,0 +1,42 @@
+/**
+  * 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.tsfile
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister, 
RelationProvider}
+import org.slf4j.LoggerFactory
+
+private[iotdb] class DefaultSource extends RelationProvider with 
DataSourceRegister {
+  private final val logger = LoggerFactory.getLogger(classOf[DefaultSource])
+
+  override def shortName(): String = "tsfile"
+
+  override def createRelation(
+   sqlContext: SQLContext,
+   parameters: Map[String, String]): BaseRelation 
= {
+
+val iotdbOptions = new IoTDBOptions(parameters)
+
+if (iotdbOptions.url == null || iotdbOptions.sql == null) {
+  sys.error("TSFile node or sql not specified")
 
 Review comment:
   ```suggestion
 sys.error("IoTDB url or sql not specified")
   ```


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #381: use jdbc to connect iotdb in spark

2019-09-11 Thread GitBox
qiaojialin commented on a change in pull request #381: use jdbc to connect 
iotdb in spark
URL: https://github.com/apache/incubator-iotdb/pull/381#discussion_r323091718
 
 

 ##
 File path: docs/Documentation/UserGuide/9-Tools-spark-iotdb.md
 ##
 @@ -0,0 +1,68 @@
+
+## version
+
+The versions required for Spark and Java are as follow:
+
+| Spark Version | Scala Version | Java Version | TsFile |
+| - | - |  | |
+| `2.4.3`| `2.11`| `1.8`| `0.9.0-SNAPSHOT`|
+
+
+## install
+mvn clean scala:compile compile install
+
+
+## maven dependency
+
+```
+
+  org.apache.iotdb
+  spark-iotdb-connector
+  0.9.0-SNAPSHOT
+
+```
+
+
+## spark-shell user guide
+
+```
+spark-shell --jars 
spark-iotdb-connector-0.9.0-SNAPSHOT.jar,tsfile-0.9.0-SNAPSHOT-jar-with-dependencies.jar,iotdb-jdbc-0.9.0-SNAPSHOT-jar-with-dependencies.jar
 
 Review comment:
   【iotdb-jdbc-0.9.0-SNAPSHOT-jar-with-dependencies.jar】 contains 
【tsfile-0.9.0-SNAPSHOT-jar-with-dependencies.jar】


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services