[GitHub] [hudi] vinothchandar commented on a change in pull request #1702: [HUDI-426] Bootstrap datasource integration

2020-08-06 Thread GitBox


vinothchandar commented on a change in pull request #1702:
URL: https://github.com/apache/hudi/pull/1702#discussion_r48582



##
File path: hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala
##
@@ -92,36 +102,69 @@ class IncrementalRelation(val sqlContext: SQLContext,
   override def schema: StructType = latestSchema
 
   override def buildScan(): RDD[Row] = {
-val fileIdToFullPath = mutable.HashMap[String, String]()
+val regularFileIdToFullPath = mutable.HashMap[String, String]()
+var metaBootstrapFileIdToFullPath = mutable.HashMap[String, String]()
+
 for (commit <- commitsToReturn) {
   val metadata: HoodieCommitMetadata = 
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit)
 .get, classOf[HoodieCommitMetadata])
-  fileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).toMap
+
+  if (HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS == commit.getTimestamp) 
{

Review comment:
   Hmmm. that seems fair. although I would have probably done this after an 
ask came up. Since you have it already, let's just keep 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




[GitHub] [hudi] vinothchandar commented on a change in pull request #1702: [HUDI-426] Bootstrap datasource integration

2020-08-06 Thread GitBox


vinothchandar commented on a change in pull request #1702:
URL: https://github.com/apache/hudi/pull/1702#discussion_r4



##
File path: hudi-spark/src/main/scala/org/apache/hudi/HudiBootstrapRDD.scala
##
@@ -0,0 +1,131 @@
+/*
+ * 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.hudi
+
+import org.apache.spark.{Partition, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+class HudiBootstrapRDD(@transient spark: SparkSession,

Review comment:
   No. there are no other files like this. This is a large topic - configs, 
.hoodie , ..code .. Like to do everything in a more streamlined fashion. 





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




[GitHub] [hudi] vinothchandar commented on a change in pull request #1702: [HUDI-426] Bootstrap datasource integration

2020-08-06 Thread GitBox


vinothchandar commented on a change in pull request #1702:
URL: https://github.com/apache/hudi/pull/1702#discussion_r47402



##
File path: hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##
@@ -56,29 +58,56 @@ class DefaultSource extends RelationProvider
 val parameters = Map(QUERY_TYPE_OPT_KEY -> DEFAULT_QUERY_TYPE_OPT_VAL) ++ 
translateViewTypesToQueryTypes(optParams)
 
 val path = parameters.get("path")
-if (path.isEmpty) {
-  throw new HoodieException("'path' must be specified.")
-}
 
 if (parameters(QUERY_TYPE_OPT_KEY).equals(QUERY_TYPE_SNAPSHOT_OPT_VAL)) {
-  // this is just effectively RO view only, where `path` can contain a mix 
of
-  // non-hoodie/hoodie path files. set the path filter up
-  sqlContext.sparkContext.hadoopConfiguration.setClass(
-"mapreduce.input.pathFilter.class",
-classOf[HoodieROTablePathFilter],
-classOf[org.apache.hadoop.fs.PathFilter])
-
-  log.info("Constructing hoodie (as parquet) data source with options :" + 
parameters)
-  log.warn("Snapshot view not supported yet via data source, for 
MERGE_ON_READ tables. " +
-"Please query the Hive table registered using Spark SQL.")
-  // simply return as a regular parquet relation
-  DataSource.apply(
-sparkSession = sqlContext.sparkSession,
-userSpecifiedSchema = Option(schema),
-className = "parquet",
-options = parameters)
-.resolveRelation()
+  val readPathsStr = 
parameters.get(DataSourceReadOptions.READ_PATHS_OPT_KEY)
+  if (path.isEmpty && readPathsStr.isEmpty) {
+throw new HoodieException(s"'path' or '$READ_PATHS_OPT_KEY' or both 
must be specified.")
+  }
+
+  val readPaths = readPathsStr.map(p => 
p.split(",").toSeq).getOrElse(Seq())
+  val allPaths = path.map(p => Seq(p)).getOrElse(Seq()) ++ readPaths
+
+  val fs = FSUtils.getFs(allPaths.head, 
sqlContext.sparkContext.hadoopConfiguration)
+  val globPaths = HudiSparkUtils.checkAndGlobPathIfNecessary(allPaths, fs)
+
+  val tablePath = DataSourceUtils.getTablePath(fs, globPaths.toArray)
+  log.info("Obtained hudi table path: " + tablePath)
+
+  val metaClient = new HoodieTableMetaClient(fs.getConf, tablePath)
+  val bootstrapIndex = BootstrapIndex.getBootstrapIndex(metaClient)
+
+  val isBootstrappedTable = bootstrapIndex.useIndex()

Review comment:
   lets file a follow up on that? I feel the TableFileSystemView can 
already provide you that metadata where the file slices dont have an external 
base file.





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




[GitHub] [hudi] vinothchandar commented on a change in pull request #1702: [HUDI-426] Bootstrap datasource integration

2020-08-06 Thread GitBox


vinothchandar commented on a change in pull request #1702:
URL: https://github.com/apache/hudi/pull/1702#discussion_r466223251



##
File path: hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##
@@ -56,29 +58,56 @@ class DefaultSource extends RelationProvider
 val parameters = Map(QUERY_TYPE_OPT_KEY -> DEFAULT_QUERY_TYPE_OPT_VAL) ++ 
translateViewTypesToQueryTypes(optParams)
 
 val path = parameters.get("path")
-if (path.isEmpty) {
-  throw new HoodieException("'path' must be specified.")
-}
 
 if (parameters(QUERY_TYPE_OPT_KEY).equals(QUERY_TYPE_SNAPSHOT_OPT_VAL)) {
-  // this is just effectively RO view only, where `path` can contain a mix 
of
-  // non-hoodie/hoodie path files. set the path filter up
-  sqlContext.sparkContext.hadoopConfiguration.setClass(
-"mapreduce.input.pathFilter.class",
-classOf[HoodieROTablePathFilter],
-classOf[org.apache.hadoop.fs.PathFilter])
-
-  log.info("Constructing hoodie (as parquet) data source with options :" + 
parameters)
-  log.warn("Snapshot view not supported yet via data source, for 
MERGE_ON_READ tables. " +
-"Please query the Hive table registered using Spark SQL.")
-  // simply return as a regular parquet relation
-  DataSource.apply(
-sparkSession = sqlContext.sparkSession,
-userSpecifiedSchema = Option(schema),
-className = "parquet",
-options = parameters)
-.resolveRelation()
+  val readPathsStr = 
parameters.get(DataSourceReadOptions.READ_PATHS_OPT_KEY)
+  if (path.isEmpty && readPathsStr.isEmpty) {
+throw new HoodieException(s"'path' or '$READ_PATHS_OPT_KEY' or both 
must be specified.")
+  }
+
+  val readPaths = readPathsStr.map(p => 
p.split(",").toSeq).getOrElse(Seq())
+  val allPaths = path.map(p => Seq(p)).getOrElse(Seq()) ++ readPaths
+
+  val fs = FSUtils.getFs(allPaths.head, 
sqlContext.sparkContext.hadoopConfiguration)
+  val globPaths = HudiSparkUtils.checkAndGlobPathIfNecessary(allPaths, fs)
+
+  val tablePath = DataSourceUtils.getTablePath(fs, globPaths.toArray)
+  log.info("Obtained hudi table path: " + tablePath)
+
+  val metaClient = new HoodieTableMetaClient(fs.getConf, tablePath)
+  val bootstrapIndex = BootstrapIndex.getBootstrapIndex(metaClient)
+
+  val isBootstrappedTable = bootstrapIndex.useIndex()

Review comment:
   we can just check `metaClient.getTableConfig()` for the bootstrap base 
path?

##
File path: hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala
##
@@ -92,36 +102,69 @@ class IncrementalRelation(val sqlContext: SQLContext,
   override def schema: StructType = latestSchema
 
   override def buildScan(): RDD[Row] = {
-val fileIdToFullPath = mutable.HashMap[String, String]()
+val regularFileIdToFullPath = mutable.HashMap[String, String]()
+var metaBootstrapFileIdToFullPath = mutable.HashMap[String, String]()
+
 for (commit <- commitsToReturn) {
   val metadata: HoodieCommitMetadata = 
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit)
 .get, classOf[HoodieCommitMetadata])
-  fileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).toMap
+
+  if (HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS == commit.getTimestamp) 
{

Review comment:
   
   bootstrap is one time operation done during migration. not sure if the user 
wants to incrementally fetch that existing data as a single batch. Not sure 
what it means to incrementally query from bootstrap instant ts?  can we just 
avoid this whole thing by only supporting incremental query on commit 
timestamps after bootstrap? 

##
File path: hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##
@@ -54,29 +58,54 @@ class DefaultSource extends RelationProvider
 val parameters = Map(QUERY_TYPE_OPT_KEY -> DEFAULT_QUERY_TYPE_OPT_VAL) ++ 
translateViewTypesToQueryTypes(optParams)
 
 val path = parameters.get("path")
-if (path.isEmpty) {
-  throw new HoodieException("'path' must be specified.")
-}
 
 if (parameters(QUERY_TYPE_OPT_KEY).equals(QUERY_TYPE_SNAPSHOT_OPT_VAL)) {
-  // this is just effectively RO view only, where `path` can contain a mix 
of
-  // non-hoodie/hoodie path files. set the path filter up
-  sqlContext.sparkContext.hadoopConfiguration.setClass(
-"mapreduce.input.pathFilter.class",
-classOf[HoodieROTablePathFilter],
-classOf[org.apache.hadoop.fs.PathFilter])
-
-  log.info("Constructing hoodie (as parquet) data source with options :" + 
parameters)
-  log.warn("Snapshot view not supported yet via data source, for 
MERGE_ON_READ tables. " +
-"Please query the Hive table registered using Spark SQL.")
-  // simply return as a regular parquet relation
-  DataSource.apply(
-sparkSession = sqlContext.sparkSession,
-userSpecifiedSchema = Option(sc

[GitHub] [hudi] vinothchandar commented on a change in pull request #1702: [HUDI-426] Bootstrap datasource integration

2020-08-04 Thread GitBox


vinothchandar commented on a change in pull request #1702:
URL: https://github.com/apache/hudi/pull/1702#discussion_r465353210



##
File path: hudi-client/pom.xml
##
@@ -101,6 +101,11 @@
   org.apache.spark
   spark-sql_${scala.binary.version}
 
+
+  org.apache.spark
+  spark-avro_${scala.binary.version}

Review comment:
   >That is why I had to introduce spark-avro in hudi-client. If you agree 
with the above suggestion, and do not want spark-avro to be added to 
hudi-client then I would suggest moving this class to hudi-spark.
   
   If that class is support to work with Spark Datasource only, yes, lets move 
it to hudi-spark 





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