[GitHub] [hudi] vinothchandar commented on a change in pull request #1702: [HUDI-426] Bootstrap datasource integration
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
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
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
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
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