vinothchandar commented on a change in pull request #2485: URL: https://github.com/apache/hudi/pull/2485#discussion_r571478862
########## File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieSourceOffset.scala ########## @@ -0,0 +1,68 @@ +/* + * 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 com.fasterxml.jackson.annotation.JsonInclude.Include Review comment: this is very simple right, can we just hand format the json without the jackson dependency? just a thought. leave it to you ########## File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieSourceOffset.scala ########## @@ -0,0 +1,68 @@ +/* + * 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 com.fasterxml.jackson.annotation.JsonInclude.Include +import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import com.fasterxml.jackson.module.scala.experimental.ScalaObjectMapper +import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset} + +case class HoodieSourceOffset(commitTime: String) extends Offset { + + override def json(): String = { + HoodieSourceOffset.toJson(this) + } + + override def equals(obj: Any): Boolean = { + obj match { + case HoodieSourceOffset(otherCommitTime) => + otherCommitTime == commitTime + case _=> false + } + } + + override def hashCode(): Int = { + commitTime.hashCode + } +} + + +object HoodieSourceOffset { + val mapper = new ObjectMapper with ScalaObjectMapper + mapper.setSerializationInclusion(Include.NON_ABSENT) + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + mapper.registerModule(DefaultScalaModule) + + def toJson(offset: HoodieSourceOffset): String = { + mapper.writeValueAsString(offset) + } + + def fromJson(json: String): HoodieSourceOffset = { + mapper.readValue[HoodieSourceOffset](json) + } + + def apply(offset: Offset): HoodieSourceOffset = { + offset match { + case SerializedOffset(json) => fromJson(json) + case o: HoodieSourceOffset => o + } + } + + val INIT_OFFSET = HoodieSourceOffset("000") Review comment: should/can we reuse HoodieTimeline#INIT_INSTANT_TS ? ########## File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.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, HoodieSparkUtils, 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.HoodieStreamSource.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} + +/** + * The Struct Stream Source for Hudi to consume the data by streaming job. + * @param sqlContext + * @param metadataPath + * @param schemaOption + * @param parameters + */ +class HoodieStreamSource( + 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( Review comment: nice. ########## File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.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, HoodieSparkUtils, 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.HoodieStreamSource.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} + +/** + * The Struct Stream Source for Hudi to consume the data by streaming job. + * @param sqlContext + * @param metadataPath + * @param schemaOption + * @param parameters + */ +class HoodieStreamSource( + 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 Review comment: could nt this be done only lazily in the else block? i.e remove this line? ########## File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkRowEncoder.java ########## @@ -23,6 +23,8 @@ import java.io.Serializable; -public interface SparkRowDeserializer extends Serializable { +public interface SparkRowEncoder extends Serializable { Review comment: may be `SparkRowSerDe` is an apt name? ########## File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.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, HoodieSparkUtils, 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.HoodieStreamSource.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} + +/** + * The Struct Stream Source for Hudi to consume the data by streaming job. + * @param sqlContext + * @param metadataPath + * @param schemaOption + * @param parameters + */ +class HoodieStreamSource( + 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 { Review comment: some comments on this code would be helpful ########## File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.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, HoodieSparkUtils, 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.HoodieStreamSource.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} + +/** + * The Struct Stream Source for Hudi to consume the data by streaming job. + * @param sqlContext + * @param metadataPath + * @param schemaOption + * @param parameters + */ +class HoodieStreamSource( + 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] = { Review comment: just a rant. `Source#getOffset()` is such a bad name. its actually the latest offset. :( ########## File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.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, HoodieSparkUtils, 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.HoodieStreamSource.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} + +/** + * The Struct Stream Source for Hudi to consume the data by streaming job. + * @param sqlContext + * @param metadataPath + * @param schemaOption + * @param parameters + */ +class HoodieStreamSource( + 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)) Review comment: FileIOUtils etc have a similar method ########## File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieSourceOffset.scala ########## @@ -0,0 +1,69 @@ +/* + * 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 Review comment: yeah I had same initial thoughts. but spark does not expose some key things like this. So we have resort to these tricks. Other projects do it too AFAIK ########## File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.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, HoodieSparkUtils, 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.HoodieStreamSource.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} + +/** + * The Struct Stream Source for Hudi to consume the data by streaming job. + * @param sqlContext + * @param metadataPath + * @param schemaOption + * @param parameters + */ +class HoodieStreamSource( + 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) Review comment: this does serialize well actually ########## File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.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 Review comment: Please use the Hudi version of IOUtils. We need the same checkstyle rules applied for scala apache-commons is an illegal import in java 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. For queries about this service, please contact Infrastructure at: [email protected]
