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 2020/08/29 09:50:10 UTC

[GitHub] [hudi] bvaradar commented on a change in pull request #2048: [HUDI-1072][WIP] Introduce REPLACE top level action

bvaradar commented on a change in pull request #2048:
URL: https://github.com/apache/hudi/pull/2048#discussion_r479626501



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
##########
@@ -173,29 +180,59 @@ protected void refreshTimeline(HoodieTimeline visibleActiveTimeline) {
     List<HoodieFileGroup> fileGroups = new ArrayList<>();
     fileIdSet.forEach(pair -> {
       String fileId = pair.getValue();
-      HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), fileId, timeline);
-      if (baseFiles.containsKey(pair)) {
-        baseFiles.get(pair).forEach(group::addBaseFile);
-      }
-      if (logFiles.containsKey(pair)) {
-        logFiles.get(pair).forEach(group::addLogFile);
-      }
+      String partitionPath = pair.getKey();
+      if (isExcludeFileGroup(partitionPath, fileId)) {

Review comment:
       As discussed, lets retain all the file-groups but perform filtering in the get APIs. THis would avoid correctness issues in filtering and also makes handling incremental file system view easier.

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java
##########
@@ -89,6 +89,10 @@ public FileSliceHandler(Configuration conf, FileSystemViewManager viewManager) t
         .collect(Collectors.toList());
   }
 
+  public List<String> getExcludeFileGroups(String basePath, String partitionPath) {

Review comment:
       Rename to getReplacedFileGroups

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java
##########
@@ -355,6 +357,18 @@ public RemoteHoodieTableFileSystemView(String server, int port, HoodieTableMetaC
     }
   }
 
+  @Override
+  public Stream<String> getAllExcludeFileGroups(final String partitionPath) {

Review comment:
       getAllExcludeFileGroups -> getReplacedFileGroups ?

##########
File path: hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java
##########
@@ -104,4 +104,8 @@ protected SyncableFileSystemView getFileSystemViewWithUnCommittedSlices(HoodieTa
   protected HoodieTableType getTableType() {
     return HoodieTableType.COPY_ON_WRITE;
   }
+
+  protected boolean areTimeTravelTestsEnabled() {

Review comment:
       why is this needed ?

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceStat.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.common.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Statistics about a single Hoodie replace operation.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class HoodieReplaceStat extends HoodieWriteStat {
+
+  // records from the 'getFileId()' can be written to multiple new file groups. This list tracks all new fileIds
+  private List<String> newFileIds;

Review comment:
       Why are we not tracking dropped fileIds ?

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/FileSystemViewHandler.java
##########
@@ -284,6 +284,13 @@ private void registerFileSlicesAPI() {
       writeValueAsString(ctx, dtos);
     }, true));
 
+    app.get(RemoteHoodieTableFileSystemView.ALL_EXCLUDE_FILEGROUPS_FOR_PARTITION_URL, new ViewHandler(ctx -> {

Review comment:
       ALL_EXCLUDE_FILEGROUPS_FOR_PARTITION_URL -> REPLACED_FILEGROUPS_FOR_PARTITION_URL
   
   Lets use a single consistent name "replaced" instead of exclude everywhere.

##########
File path: hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java
##########
@@ -102,7 +102,7 @@ public void commit(WriterCommitMessage[] messages) {
             .flatMap(m -> m.getWriteStatuses().stream().map(m2 -> m2.getStat())).collect(Collectors.toList());
 
     try {
-      writeClient.commitStats(instantTime, writeStatList, Option.empty());
+      writeClient.commitStats(instantTime, writeStatList, Option.empty(), HoodieTimeline.COMMIT_ACTION); //TODO get action type from HoodieWriterCommitMessage

Review comment:
       Are you planning to address this TODO as part of this PR ?




----------------------------------------------------------------
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