garyli1019 commented on a change in pull request #2485:
URL: https://github.com/apache/hudi/pull/2485#discussion_r564535063



##########
File path: 
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##########
@@ -181,4 +184,35 @@ class DefaultSource extends RelationProvider
         .resolveRelation()
     }
   }
+
+  override def sourceSchema(sqlContext: SQLContext,
+                            schema: Option[StructType],
+                            providerName: String,
+                            parameters: Map[String, String]): (String, 
StructType) = {
+    val path = parameters.get("path")
+    if (path.isEmpty || path.get == null) {
+      throw new HoodieException(s"'path'  must be specified.")
+    }
+    val metaClient = new HoodieTableMetaClient(
+      sqlContext.sparkSession.sessionState.newHadoopConf(), path.get)
+    val schemaResolver = new TableSchemaResolver(metaClient)
+    val sqlSchema =
+      try {
+        val avroSchema = schemaResolver.getTableAvroSchema
+        
SchemaConverters.toSqlType(avroSchema).dataType.asInstanceOf[StructType]

Review comment:
       I think we should use the hudi internal converter from 
`AvroConversionUtils`, to be consistent with others.

##########
File path: 
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieSource.scala
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.spark.sql.hudi.streaming
+
+import java.io.{BufferedWriter, InputStream, InputStreamReader, OutputStream, 
OutputStreamWriter}
+import java.nio.charset.StandardCharsets
+import java.util.Date
+
+import org.apache.commons.io.IOUtils
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.{DataSourceReadOptions, IncrementalRelation, 
MergeOnReadIncrementalRelation}
+import org.apache.hudi.common.model.HoodieTableType
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
+import org.apache.hudi.common.table.{HoodieTableMetaClient, 
TableSchemaResolver}
+import org.apache.hudi.common.util.TablePathUtils
+import org.apache.spark.sql.hudi.streaming.HoodieSource.VERSION
+import org.apache.spark.sql.hudi.streaming.HoodieSourceOffset.INIT_OFFSET
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.avro.SchemaConverters
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, Offset, 
Source}
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{DataFrame, SQLContext}
+
+/**
+  * Struct Stream Source for Hudi.
+  * @param sqlContext
+  * @param metadataPath
+  * @param schemaOption
+  * @param parameters
+  */
+class HoodieSource(

Review comment:
       More specific name? like `StructuredStreamingHoodieSource`?

##########
File path: 
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieSource.scala
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.spark.sql.hudi.streaming
+
+import java.io.{BufferedWriter, InputStream, InputStreamReader, OutputStream, 
OutputStreamWriter}
+import java.nio.charset.StandardCharsets
+import java.util.Date
+
+import org.apache.commons.io.IOUtils
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.{DataSourceReadOptions, IncrementalRelation, 
MergeOnReadIncrementalRelation}
+import org.apache.hudi.common.model.HoodieTableType
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
+import org.apache.hudi.common.table.{HoodieTableMetaClient, 
TableSchemaResolver}
+import org.apache.hudi.common.util.TablePathUtils
+import org.apache.spark.sql.hudi.streaming.HoodieSource.VERSION
+import org.apache.spark.sql.hudi.streaming.HoodieSourceOffset.INIT_OFFSET
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.avro.SchemaConverters
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, Offset, 
Source}
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{DataFrame, SQLContext}
+
+/**
+  * Struct Stream Source for Hudi.
+  * @param sqlContext
+  * @param metadataPath
+  * @param schemaOption
+  * @param parameters
+  */
+class HoodieSource(
+    sqlContext: SQLContext,
+    metadataPath: String,
+    schemaOption: Option[StructType],
+    parameters: Map[String, String])
+  extends Source with Logging with Serializable {
+
+  @transient private val hadoopConf = 
sqlContext.sparkSession.sessionState.newHadoopConf()
+  private lazy val tablePath: Path = {
+    val path = new Path(parameters.getOrElse("path", "Missing 'path' option"))
+    val fs = path.getFileSystem(hadoopConf)
+    TablePathUtils.getTablePath(fs, path).get()
+  }
+  @transient private lazy val metaClient = new 
HoodieTableMetaClient(hadoopConf, tablePath.toString)
+  private lazy val tableType = metaClient.getTableType
+
+  @transient private var lastOffset: HoodieSourceOffset = _
+  @transient private lazy val initialPartitionOffsets = {
+    val metadataLog =
+      new HDFSMetadataLog[HoodieSourceOffset](sqlContext.sparkSession, 
metadataPath) {
+        override def serialize(metadata: HoodieSourceOffset, out: 
OutputStream): Unit = {
+          val writer = new BufferedWriter(new OutputStreamWriter(out, 
StandardCharsets.UTF_8))
+          writer.write("v" + VERSION + "\n")
+          writer.write(metadata.json)
+          writer.flush()
+        }
+
+        override def deserialize(in: InputStream): HoodieSourceOffset = {
+          val content = IOUtils.toString(new InputStreamReader(in, 
StandardCharsets.UTF_8))
+
+          val firstLineEnd = content.indexOf("\n")
+          if (firstLineEnd > 0) {
+            val version = getVersion(content.substring(0, firstLineEnd))
+            if (version > VERSION) {
+              throw new IllegalStateException(s"UnSupportVersion: max support 
version is: $VERSION" +
+                s" current version is: $version")
+            }
+            HoodieSourceOffset.fromJson(content.substring(firstLineEnd + 1))
+          } else {
+            throw new IllegalStateException(s"Bad metadata format, failed to 
find the version line.")
+          }
+        }
+      }
+    metadataLog.get(0).getOrElse {
+      metadataLog.add(0, INIT_OFFSET)
+      INIT_OFFSET
+    }
+  }
+
+  private def getVersion(versionLine: String): Int = {
+    if (versionLine.startsWith("v")) {
+      versionLine.substring(1).toInt
+    } else {
+      throw new IllegalStateException(s"Illegal version line: $versionLine " +
+        s"in the streaming metadata path")
+    }
+  }
+
+  override def schema: StructType = {
+    schemaOption.getOrElse {
+      val schemaUtil = new TableSchemaResolver(metaClient)
+      SchemaConverters.toSqlType(schemaUtil.getTableAvroSchema)
+        .dataType.asInstanceOf[StructType]
+    }
+  }
+
+  override def getOffset: Option[Offset] = {
+    initialPartitionOffsets
+
+    metaClient.reloadActiveTimeline()
+    val activeInstants = 
metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants
+    if (!activeInstants.empty()) {
+      val currentLatestCommitTime = 
activeInstants.lastInstant().get().getTimestamp
+      if (lastOffset == null || currentLatestCommitTime > 
lastOffset.commitTime) {
+        lastOffset = HoodieSourceOffset(currentLatestCommitTime)
+      }
+    } else { // if there are no active commits, use the init offset
+      lastOffset = initialPartitionOffsets
+    }
+    Some(lastOffset)
+  }
+
+  override def getBatch(start: Option[Offset], end: Offset): DataFrame = {
+    initialPartitionOffsets
+
+    val startOffset = start.map(HoodieSourceOffset(_))
+      .getOrElse(initialPartitionOffsets)
+    val endOffset = HoodieSourceOffset(end)
+
+    if (startOffset == endOffset) {
+      sqlContext.internalCreateDataFrame(
+        sqlContext.sparkContext.emptyRDD[InternalRow].setName("empty"), 
schema, isStreaming = true)
+    } else {
+      // Consume the data between (startCommitTime, endCommitTime]
+      val incParams = parameters ++ Map(
+        DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY -> 
startCommitTime(startOffset),
+        DataSourceReadOptions.END_INSTANTTIME_OPT_KEY -> endOffset.commitTime
+      )
+
+      val rdd = tableType match {
+        case HoodieTableType.COPY_ON_WRITE =>
+          val encoder = RowEncoder(schema)
+          new IncrementalRelation(sqlContext, incParams, schema, metaClient)
+            .buildScan()
+            .map(encoder.toRow)
+        case HoodieTableType.MERGE_ON_READ =>
+          val requiredColumns = schema.fields.map(_.name)
+          new MergeOnReadIncrementalRelation(sqlContext, incParams, schema, 
metaClient)
+            .buildScan(requiredColumns, Array.empty[Filter])
+            .asInstanceOf[RDD[InternalRow]]
+        case _ => throw new IllegalArgumentException(s"UnSupport tableType: 
$tableType")
+      }
+      sqlContext.internalCreateDataFrame(rdd, schema, isStreaming = true)
+    }
+  }
+
+  private def startCommitTime(startOffset: HoodieSourceOffset): String = {
+    startOffset match {
+      case INIT_OFFSET => startOffset.commitTime
+      case HoodieSourceOffset(commitTime) =>
+        val time = 
HoodieActiveTimeline.COMMIT_FORMATTER.parse(commitTime).getTime
+        // As we consume the data between (start, end], start is not included,
+        // so we +1s to the start commit time here.
+        HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date(time + 1000))
+      case _=> throw new IllegalStateException("UnKnow offset type.")
+    }
+  }
+
+  override def stop(): Unit = {
+
+  }
+}
+
+object HoodieSource {
+  val VERSION = 1

Review comment:
       Initial version?

##########
File path: 
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieSource.scala
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.spark.sql.hudi.streaming
+
+import java.io.{BufferedWriter, InputStream, InputStreamReader, OutputStream, 
OutputStreamWriter}
+import java.nio.charset.StandardCharsets
+import java.util.Date
+
+import org.apache.commons.io.IOUtils
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.{DataSourceReadOptions, IncrementalRelation, 
MergeOnReadIncrementalRelation}
+import org.apache.hudi.common.model.HoodieTableType
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
+import org.apache.hudi.common.table.{HoodieTableMetaClient, 
TableSchemaResolver}
+import org.apache.hudi.common.util.TablePathUtils
+import org.apache.spark.sql.hudi.streaming.HoodieSource.VERSION
+import org.apache.spark.sql.hudi.streaming.HoodieSourceOffset.INIT_OFFSET
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.avro.SchemaConverters
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, Offset, 
Source}
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{DataFrame, SQLContext}
+
+/**
+  * Struct Stream Source for Hudi.

Review comment:
       We could add more docs. This sounds like using structured streaming to 
sink to Hudi. 




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


Reply via email to