szehon-ho commented on code in PR #7653:
URL: https://github.com/apache/iceberg/pull/7653#discussion_r1266249371


##########
spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteOrphanFilesSparkAction.java:
##########
@@ -296,4 +334,160 @@ private static FlatMapFunction<Iterator<String>, String> 
listDirsRecursively(
       return files.iterator();
     };
   }
+
+  @VisibleForTesting
+  static List<String> findOrphanFiles(
+      SparkSession spark,
+      Dataset<Row> actualFileDF,
+      Dataset<Row> validFileDF,
+      Map<String, String> equalSchemes,
+      Map<String, String> equalAuthorities,
+      PrefixMismatchMode prefixMismatchMode) {
+    Dataset<FileMetadata> actualFileMetadataDS =
+        actualFileDF.mapPartitions(
+            toFileMetadata(equalSchemes, equalAuthorities), 
Encoders.bean(FileMetadata.class));
+    Dataset<FileMetadata> validFileMetadataDS =
+        validFileDF.mapPartitions(
+            toFileMetadata(equalSchemes, equalAuthorities), 
Encoders.bean(FileMetadata.class));
+
+    SetAccumulator<Pair<String, String>> conflicts = new SetAccumulator<>();
+    spark.sparkContext().register(conflicts);
+
+    Column joinCond = 
actualFileMetadataDS.col("path").equalTo(validFileMetadataDS.col("path"));
+
+    List<String> orphanFiles =
+        actualFileMetadataDS
+            .joinWith(validFileMetadataDS, joinCond, "leftouter")
+            .mapPartitions(findOrphanFiles(prefixMismatchMode, conflicts), 
Encoders.STRING())
+            .collectAsList();
+
+    if (prefixMismatchMode == PrefixMismatchMode.ERROR && 
!conflicts.value().isEmpty()) {
+      throw new ValidationException(
+          "Unable to determine whether certain files are orphan. "
+              + "Metadata references files that match listed/provided files 
except for authority/scheme. "
+              + "Please, inspect the conflicting authorities/schemes and 
provide which of them are equal "
+              + "by further configuring the action via equalSchemes() and 
equalAuthorities() methods. "
+              + "Set the prefix mismatch mode to 'NONE' to ignore remaining 
locations with conflicting "
+              + "authorities/schemes or to 'DELETE' iff you are ABSOLUTELY 
confident that remaining conflicting "
+              + "authorities/schemes are different. It will be impossible to 
recover deleted files. "
+              + "Conflicting authorities/schemes: %s.",
+          conflicts.value());
+    }
+
+    return orphanFiles;
+  }
+
+  private static MapPartitionsFunction<Tuple2<FileMetadata, FileMetadata>, 
String> findOrphanFiles(
+      PrefixMismatchMode mode, SetAccumulator<Pair<String, String>> conflicts) 
{
+    return rows -> {
+      Iterator<String> transformed =
+          Iterators.transform(
+              rows,
+              row -> {
+                FileMetadata actual = row._1;
+                FileMetadata valid = row._2;
+
+                if (valid == null) {
+                  return actual.location;
+                }
+
+                boolean schemeMatch =
+                    Strings.isNullOrEmpty(valid.scheme)
+                        || valid.scheme.equalsIgnoreCase(actual.scheme);
+                boolean authorityMatch =
+                    Strings.isNullOrEmpty(valid.authority)
+                        || valid.authority.equalsIgnoreCase(actual.authority);
+
+                if ((!schemeMatch || !authorityMatch) && mode == 
PrefixMismatchMode.DELETE) {
+                  return actual.location;
+                } else {
+                  if (!schemeMatch) {
+                    conflicts.add(Pair.of(valid.scheme, actual.scheme));
+                  }
+                  if (!authorityMatch) {
+                    conflicts.add(Pair.of(valid.authority, actual.authority));
+                  }
+                }
+
+                return null;
+              });
+      return Iterators.filter(transformed, Objects::nonNull);
+    };
+  }
+
+  private static Map<String, String> flattenMap(Map<String, String> map) {
+    Map<String, String> flattenedMap = Maps.newHashMap();
+    if (map != null) {
+      for (String key : map.keySet()) {
+        String value = map.get(key);
+        for (String splitKey : COMMA.split(key)) {
+          flattenedMap.put(splitKey.trim(), value.trim());
+        }
+      }
+    }
+    return flattenedMap;
+  }
+
+  private static MapPartitionsFunction<Row, FileMetadata> toFileMetadata(
+      Map<String, String> equalSchemesMap, Map<String, String> 
equalAuthoritiesMap) {
+    return rows ->
+        Iterators.transform(

Review Comment:
   Same.



##########
spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteOrphanFilesSparkAction.java:
##########
@@ -296,4 +334,160 @@ private static FlatMapFunction<Iterator<String>, String> 
listDirsRecursively(
       return files.iterator();
     };
   }
+
+  @VisibleForTesting
+  static List<String> findOrphanFiles(
+      SparkSession spark,
+      Dataset<Row> actualFileDF,
+      Dataset<Row> validFileDF,
+      Map<String, String> equalSchemes,
+      Map<String, String> equalAuthorities,
+      PrefixMismatchMode prefixMismatchMode) {
+    Dataset<FileMetadata> actualFileMetadataDS =
+        actualFileDF.mapPartitions(
+            toFileMetadata(equalSchemes, equalAuthorities), 
Encoders.bean(FileMetadata.class));
+    Dataset<FileMetadata> validFileMetadataDS =
+        validFileDF.mapPartitions(
+            toFileMetadata(equalSchemes, equalAuthorities), 
Encoders.bean(FileMetadata.class));
+
+    SetAccumulator<Pair<String, String>> conflicts = new SetAccumulator<>();
+    spark.sparkContext().register(conflicts);
+
+    Column joinCond = 
actualFileMetadataDS.col("path").equalTo(validFileMetadataDS.col("path"));
+
+    List<String> orphanFiles =
+        actualFileMetadataDS
+            .joinWith(validFileMetadataDS, joinCond, "leftouter")
+            .mapPartitions(findOrphanFiles(prefixMismatchMode, conflicts), 
Encoders.STRING())
+            .collectAsList();
+
+    if (prefixMismatchMode == PrefixMismatchMode.ERROR && 
!conflicts.value().isEmpty()) {
+      throw new ValidationException(
+          "Unable to determine whether certain files are orphan. "
+              + "Metadata references files that match listed/provided files 
except for authority/scheme. "
+              + "Please, inspect the conflicting authorities/schemes and 
provide which of them are equal "
+              + "by further configuring the action via equalSchemes() and 
equalAuthorities() methods. "
+              + "Set the prefix mismatch mode to 'NONE' to ignore remaining 
locations with conflicting "
+              + "authorities/schemes or to 'DELETE' iff you are ABSOLUTELY 
confident that remaining conflicting "
+              + "authorities/schemes are different. It will be impossible to 
recover deleted files. "
+              + "Conflicting authorities/schemes: %s.",
+          conflicts.value());
+    }
+
+    return orphanFiles;
+  }
+
+  private static MapPartitionsFunction<Tuple2<FileMetadata, FileMetadata>, 
String> findOrphanFiles(
+      PrefixMismatchMode mode, SetAccumulator<Pair<String, String>> conflicts) 
{
+    return rows -> {
+      Iterator<String> transformed =
+          Iterators.transform(
+              rows,

Review Comment:
   Nit: could we quickly fix the formatting?  It should be on same line like in 
Spark 3.2?
   
   `Iterator<String> transformed = Iterators.transform(rows, row -> {`



##########
spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteOrphanFilesSparkAction.java:
##########
@@ -296,4 +334,160 @@ private static FlatMapFunction<Iterator<String>, String> 
listDirsRecursively(
       return files.iterator();
     };
   }
+
+  @VisibleForTesting
+  static List<String> findOrphanFiles(
+      SparkSession spark,
+      Dataset<Row> actualFileDF,
+      Dataset<Row> validFileDF,
+      Map<String, String> equalSchemes,
+      Map<String, String> equalAuthorities,
+      PrefixMismatchMode prefixMismatchMode) {
+    Dataset<FileMetadata> actualFileMetadataDS =
+        actualFileDF.mapPartitions(
+            toFileMetadata(equalSchemes, equalAuthorities), 
Encoders.bean(FileMetadata.class));
+    Dataset<FileMetadata> validFileMetadataDS =
+        validFileDF.mapPartitions(
+            toFileMetadata(equalSchemes, equalAuthorities), 
Encoders.bean(FileMetadata.class));
+
+    SetAccumulator<Pair<String, String>> conflicts = new SetAccumulator<>();
+    spark.sparkContext().register(conflicts);
+
+    Column joinCond = 
actualFileMetadataDS.col("path").equalTo(validFileMetadataDS.col("path"));
+
+    List<String> orphanFiles =
+        actualFileMetadataDS
+            .joinWith(validFileMetadataDS, joinCond, "leftouter")
+            .mapPartitions(findOrphanFiles(prefixMismatchMode, conflicts), 
Encoders.STRING())
+            .collectAsList();
+
+    if (prefixMismatchMode == PrefixMismatchMode.ERROR && 
!conflicts.value().isEmpty()) {
+      throw new ValidationException(
+          "Unable to determine whether certain files are orphan. "
+              + "Metadata references files that match listed/provided files 
except for authority/scheme. "
+              + "Please, inspect the conflicting authorities/schemes and 
provide which of them are equal "
+              + "by further configuring the action via equalSchemes() and 
equalAuthorities() methods. "
+              + "Set the prefix mismatch mode to 'NONE' to ignore remaining 
locations with conflicting "
+              + "authorities/schemes or to 'DELETE' iff you are ABSOLUTELY 
confident that remaining conflicting "
+              + "authorities/schemes are different. It will be impossible to 
recover deleted files. "
+              + "Conflicting authorities/schemes: %s.",
+          conflicts.value());
+    }
+
+    return orphanFiles;
+  }
+
+  private static MapPartitionsFunction<Tuple2<FileMetadata, FileMetadata>, 
String> findOrphanFiles(
+      PrefixMismatchMode mode, SetAccumulator<Pair<String, String>> conflicts) 
{
+    return rows -> {
+      Iterator<String> transformed =
+          Iterators.transform(
+              rows,
+              row -> {
+                FileMetadata actual = row._1;
+                FileMetadata valid = row._2;
+
+                if (valid == null) {
+                  return actual.location;
+                }
+
+                boolean schemeMatch =
+                    Strings.isNullOrEmpty(valid.scheme)
+                        || valid.scheme.equalsIgnoreCase(actual.scheme);
+                boolean authorityMatch =
+                    Strings.isNullOrEmpty(valid.authority)
+                        || valid.authority.equalsIgnoreCase(actual.authority);
+
+                if ((!schemeMatch || !authorityMatch) && mode == 
PrefixMismatchMode.DELETE) {
+                  return actual.location;
+                } else {
+                  if (!schemeMatch) {
+                    conflicts.add(Pair.of(valid.scheme, actual.scheme));
+                  }
+                  if (!authorityMatch) {
+                    conflicts.add(Pair.of(valid.authority, actual.authority));
+                  }
+                }
+
+                return null;
+              });
+      return Iterators.filter(transformed, Objects::nonNull);
+    };
+  }
+
+  private static Map<String, String> flattenMap(Map<String, String> map) {

Review Comment:
   Should this method be in the same place, after the first findOrphanFiles(), 
for easier cherry-pick of further changes?



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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to