[
https://issues.apache.org/jira/browse/HUDI-2107?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17375546#comment-17375546
]
ASF GitHub Bot commented on HUDI-2107:
--------------------------------------
pengzhiwei2018 commented on a change in pull request #3193:
URL: https://github.com/apache/hudi/pull/3193#discussion_r664402382
##########
File path:
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java
##########
@@ -161,29 +162,21 @@
.map(instant ->
fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath,
instant.getTimestamp()))
.orElse(Stream.empty());
- // subgroup splits again by file id & match with log files.
- Map<String, List<HoodieBaseFile>> groupedInputSplits =
partitionsToParquetSplits.get(partitionPath).stream()
- .collect(Collectors.groupingBy(file ->
FSUtils.getFileId(file.getFileStatus().getPath().getName())));
latestFileSlices.forEach(fileSlice -> {
- List<HoodieBaseFile> dataFileSplits =
groupedInputSplits.get(fileSlice.getFileId());
- dataFileSplits.forEach(split -> {
- try {
- List<String> logFilePaths =
fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
+ List<String> logFilePaths =
fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
.map(logFile ->
logFile.getPath().toString()).collect(Collectors.toList());
- resultMap.put(split, logFilePaths);
- } catch (Exception e) {
- throw new HoodieException("Error creating hoodie real time split
", e);
- }
- });
+ baseAndLogsList.add(Pair.of(fileSlice.getBaseFile(), logFilePaths));
});
} catch (Exception e) {
throw new HoodieException("Error obtaining data file/log file
grouping: " + partitionPath, e);
}
});
- return resultMap;
+ return baseAndLogsList;
}
+
+
Review comment:
done
##########
File path:
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala
##########
@@ -137,11 +136,14 @@ class MergeOnReadSnapshotRelation(val sqlContext:
SQLContext,
}
def buildFileIndex(filters: Array[Filter]): List[HoodieMergeOnReadFileSplit]
= {
-
- val fileStatuses = if (globPaths.isDefined) {
+ // Get all partition paths
+ val partitionPaths = if (globPaths.isDefined) {
// Load files from the global paths if it has defined to be compatible
with the original mode
val inMemoryFileIndex =
HoodieSparkUtils.createInMemoryFileIndex(sqlContext.sparkSession, globPaths.get)
- inMemoryFileIndex.allFiles()
+ val fsView = new HoodieTableFileSystemView(metaClient,
+ metaClient.getActiveTimeline.getCommitsTimeline
+ .filterCompletedInstants, inMemoryFileIndex.allFiles().toArray)
+
fsView.getLatestBaseFiles.iterator().asScala.toList.map(_.getFileStatus.getPath.getParent)
Review comment:
Yes, currently I have not support reading-log only table for the
`globPaths` query which users must specify some `*` in the query path and the
partition prune is also not work.
In the long way, we should recommend the non-globalPath query which use
the `HoodieFileIndex` to list files and partitions. User do not need to specify
the `*` in the query path and can also have better performance from the
partition prune and meta table list.
##########
File path:
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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
+
+import org.apache.hudi.testutils.DataSourceTestUtils
+
+class TestMergeIntoTable2 extends TestHoodieSqlBase {
Review comment:
As the origin `TestMergeIntoTable` is too long, I add another test class
for merge-into. All subsequent cases about merge-into will be placed on
TestMergeIntoTable2.
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
> Support Read Log Only MOR Table For Spark
> -----------------------------------------
>
> Key: HUDI-2107
> URL: https://issues.apache.org/jira/browse/HUDI-2107
> Project: Apache Hudi
> Issue Type: Bug
> Components: Spark Integration
> Reporter: pengzhiwei
> Assignee: pengzhiwei
> Priority: Major
> Labels: pull-request-available
> Fix For: 0.9.0
>
>
> Currently we cannot support read log-only mor table(which is generated by
> index like InMemeoryIndex, HbaseIndex and FlinkIndex which support indexing
> log file) for spark.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)