You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2021/01/07 17:54:22 UTC

[GitHub] [hudi] vinothchandar commented on a change in pull request #1938: [HUDI-920] Support Incremental query for MOR table

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



##########
File path: hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java
##########
@@ -470,4 +471,45 @@ private static HoodieBaseFile refreshFileStatus(Configuration conf, HoodieBaseFi
     }
   }
 
+  /**
+   * List affected file status based on given commits.

Review comment:
       a little bit more clearer doc? does this method obtain all the file status that were affected by the list of commits to check?

##########
File path: hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java
##########
@@ -470,4 +471,45 @@ private static HoodieBaseFile refreshFileStatus(Configuration conf, HoodieBaseFi
     }
   }
 
+  /**
+   * List affected file status based on given commits.
+   * @param basePath
+   * @param commitsToCheck
+   * @param timeline
+   * @return HashMap<partitionPath, HashMap<fileName, FileStatus>>
+   * @throws IOException
+   */
+  public static HashMap<String, HashMap<String, FileStatus>> listStatusForAffectedPartitions(
+      Path basePath, List<HoodieInstant> commitsToCheck, HoodieTimeline timeline) throws IOException {
+    // Extract files touched by these commits.
+    // TODO This might need to be done in parallel like listStatus parallelism ?

Review comment:
       Can we redo this such that it can use the metadata table for obtaining the listing? You can see how this is done in HoodieParquetInputFormat. 

##########
File path: hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java
##########
@@ -470,4 +471,45 @@ private static HoodieBaseFile refreshFileStatus(Configuration conf, HoodieBaseFi
     }
   }
 
+  /**
+   * List affected file status based on given commits.
+   * @param basePath
+   * @param commitsToCheck
+   * @param timeline
+   * @return HashMap<partitionPath, HashMap<fileName, FileStatus>>
+   * @throws IOException
+   */
+  public static HashMap<String, HashMap<String, FileStatus>> listStatusForAffectedPartitions(
+      Path basePath, List<HoodieInstant> commitsToCheck, HoodieTimeline timeline) throws IOException {
+    // Extract files touched by these commits.
+    // TODO This might need to be done in parallel like listStatus parallelism ?
+    HashMap<String, HashMap<String, FileStatus>> partitionToFileStatusesMap = new HashMap<>();
+    for (HoodieInstant commit: commitsToCheck) {
+      HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get(),
+          HoodieCommitMetadata.class);
+      for (Map.Entry<String, List<HoodieWriteStat>> entry: commitMetadata.getPartitionToWriteStats().entrySet()) {
+        if (!partitionToFileStatusesMap.containsKey(entry.getKey())) {
+          partitionToFileStatusesMap.put(entry.getKey(), new HashMap<>());
+        }
+        for (HoodieWriteStat stat : entry.getValue()) {
+          String relativeFilePath = stat.getPath();
+          Path fullPath = relativeFilePath != null ? FSUtils.getPartitionPath(basePath, relativeFilePath) : null;
+          if (fullPath != null) {
+            if (partitionToFileStatusesMap.get(entry.getKey()).containsKey(fullPath.getName())) {
+              // If filesystem support Append. Update the FileStatus of log file if being appended.

Review comment:
       you can just pick the latest such file now. it will have the latest log size using `getFileSizeInBytes()`. No need to do the addition here. 

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.hadoop.fs.{FileStatus, FileSystem, GlobPattern, Path}
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.model.HoodieRecord
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView
+import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.listStatusForAffectedPartitions
+import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes
+import org.apache.log4j.LogManager
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
+import org.apache.spark.sql.sources._
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{Row, SQLContext}
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ListBuffer
+
+/**
+  * Experimental.
+  * Relation, that implements the Hoodie incremental view for Merge On Read table.
+  *
+  */
+class MergeOnReadIncrementalRelation(val sqlContext: SQLContext,

Review comment:
       can we subclass `IncrementalRelation` and reuse some of the code esp lines 54-66 etc? worth it?

##########
File path: hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java
##########
@@ -470,4 +471,45 @@ private static HoodieBaseFile refreshFileStatus(Configuration conf, HoodieBaseFi
     }
   }
 
+  /**
+   * List affected file status based on given commits.
+   * @param basePath
+   * @param commitsToCheck
+   * @param timeline
+   * @return HashMap<partitionPath, HashMap<fileName, FileStatus>>
+   * @throws IOException
+   */
+  public static HashMap<String, HashMap<String, FileStatus>> listStatusForAffectedPartitions(

Review comment:
       can we just have this in `hudi-spark` for now. thats the only module that needs to call this. 

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.hadoop.fs.{FileStatus, FileSystem, GlobPattern, Path}
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.model.HoodieRecord
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView
+import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.listStatusForAffectedPartitions
+import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes
+import org.apache.log4j.LogManager
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
+import org.apache.spark.sql.sources._
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{Row, SQLContext}
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ListBuffer
+
+/**
+  * Experimental.
+  * Relation, that implements the Hoodie incremental view for Merge On Read table.
+  *
+  */
+class MergeOnReadIncrementalRelation(val sqlContext: SQLContext,
+                                     val optParams: Map[String, String],
+                                     val userSchema: StructType,
+                                     val metaClient: HoodieTableMetaClient)
+  extends BaseRelation with PrunedFilteredScan {
+
+  private val log = LogManager.getLogger(classOf[MergeOnReadIncrementalRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+  private val jobConf = new JobConf(conf)
+  private val fs = FSUtils.getFs(metaClient.getBasePath, conf)
+  private val commitTimeline = metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants()
+  if (commitTimeline.empty()) {
+    throw new HoodieException("No instants to incrementally pull")
+  }
+  if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY)) {
+    throw new HoodieException(s"Specify the begin instant time to pull from using " +
+      s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY}")
+  }
+
+  private val lastInstant = commitTimeline.lastInstant().get()
+  private val mergeType = optParams.getOrElse(
+    DataSourceReadOptions.REALTIME_MERGE_OPT_KEY,
+    DataSourceReadOptions.DEFAULT_REALTIME_MERGE_OPT_VAL)
+
+  private val commitsTimelineToReturn = commitTimeline.findInstantsInRange(
+    optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY),
+    optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, lastInstant.getTimestamp))
+  log.debug(s"${commitsTimelineToReturn.getInstants.iterator().toList.map(f => f.toString).mkString(",")}")
+  private val commitsToReturn = commitsTimelineToReturn.getInstants.iterator().toList
+  private val schemaUtil = new TableSchemaResolver(metaClient)
+  private val tableAvroSchema = schemaUtil.getTableAvroSchema
+  private val tableStructSchema = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema)
+  private val maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(jobConf)
+  private val fileIndex = buildFileIndex()
+
+  override def schema: StructType = tableStructSchema
+
+  override def needConversion: Boolean = false
+
+  override def unhandledFilters(filters: Array[Filter]): Array[Filter] = {
+    val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD)
+    val largerThanFilter = GreaterThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp)
+    val lessThanFilter = LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp)
+    filters :+isNotNullFilter :+ largerThanFilter :+ lessThanFilter
+  }
+
+  override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = {
+    log.debug(s"buildScan requiredColumns = ${requiredColumns.mkString(",")}")
+    log.debug(s"buildScan filters = ${filters.mkString(",")}")
+    // config to ensure the push down filter for parquet will be applied.
+    sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.filterPushdown", "true")
+    sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.recordLevelFilter.enabled", "true")
+    sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.enableVectorizedReader", "false")
+    val pushDownFilter = {

Review comment:
       this is the secret sauce to do the filtering at the record level, I guess




----------------------------------------------------------------
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:
users@infra.apache.org