aokolnychyi commented on code in PR #5495:
URL: https://github.com/apache/iceberg/pull/5495#discussion_r948459162
##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java:
##########
@@ -426,65 +405,159 @@ private static Map<String, String>
flattenMap(Map<String, String> map) {
if (map != null) {
for (String key : map.keySet()) {
String value = map.get(key);
- for (String splitKey : COMMA.split(key)) {
+ for (String splitKey : COMMA_SPLITTER.split(key)) {
flattenedMap.put(splitKey.trim(), value.trim());
}
}
}
return flattenedMap;
}
- 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 class ListDirsRecursively implements
FlatMapFunction<Iterator<String>, String> {
+
+ private final Broadcast<SerializableConfiguration> hadoopConf;
+ private final long olderThanTimestamp;
+ private final PathFilter pathFilter;
+
+ ListDirsRecursively(
+ Broadcast<SerializableConfiguration> hadoopConf,
+ long olderThanTimestamp,
+ PathFilter pathFilter) {
+
+ this.hadoopConf = hadoopConf;
+ this.olderThanTimestamp = olderThanTimestamp;
+ this.pathFilter = pathFilter;
+ }
+
+ @Override
+ public Iterator<String> call(Iterator<String> dirs) throws Exception {
+ List<String> subDirs = Lists.newArrayList();
+ List<String> files = Lists.newArrayList();
+
+ Predicate<FileStatus> predicate = file -> file.getModificationTime() <
olderThanTimestamp;
+
+ dirs.forEachRemaining(
+ dir -> {
+ listDirRecursively(
+ dir,
+ predicate,
+ hadoopConf.value().value(),
+ MAX_EXECUTOR_LISTING_DEPTH,
+ MAX_EXECUTOR_LISTING_DIRECT_SUB_DIRS,
+ subDirs,
+ pathFilter,
+ files);
+ });
+
+ if (!subDirs.isEmpty()) {
+ throw new RuntimeException(
+ "Could not list sub directories, reached maximum depth: " +
MAX_EXECUTOR_LISTING_DEPTH);
+ }
+
+ return files.iterator();
+ }
}
- private static MapPartitionsFunction<Row, FileMetadata> toFileMetadata(
- Map<String, String> equalSchemesMap, Map<String, String>
equalAuthoritiesMap) {
- return rows ->
- Iterators.transform(
- rows,
- row -> {
- String location = row.getString(0);
- URI uri = new Path(location).toUri();
- String scheme = equalSchemesMap.getOrDefault(uri.getScheme(),
uri.getScheme());
- String authority =
- equalAuthoritiesMap.getOrDefault(uri.getAuthority(),
uri.getAuthority());
- return new FileMetadata(scheme, authority, uri.getPath(),
location);
- });
+ private static class FindOrphanFiles
+ implements MapPartitionsFunction<Tuple2<FileURI, FileURI>, String> {
+
+ private final PrefixMismatchMode mode;
+ private final SetAccumulator<Pair<String, String>> conflicts;
+
+ FindOrphanFiles(PrefixMismatchMode mode, SetAccumulator<Pair<String,
String>> conflicts) {
+ this.mode = mode;
+ this.conflicts = conflicts;
+ }
+
+ @Override
+ public Iterator<String> call(Iterator<Tuple2<FileURI, FileURI>> rows)
throws Exception {
+ Iterator<String> orphanFiles = Iterators.transform(rows,
this::toOrphanFile);
+ return Iterators.filter(orphanFiles, Objects::nonNull);
+ }
+
+ private String toOrphanFile(Tuple2<FileURI, FileURI> row) {
+ FileURI actual = row._1;
+ FileURI valid = row._2;
+
+ if (valid == null) {
+ return actual.valueAsString;
+ }
+
+ boolean schemeMatch = uriComponentMatch(valid.scheme, actual.scheme);
+ boolean authorityMatch = uriComponentMatch(valid.authority,
actual.authority);
+
+ if ((!schemeMatch || !authorityMatch) && mode ==
PrefixMismatchMode.DELETE) {
+ return actual.valueAsString;
+ } else {
+ if (!schemeMatch) {
+ conflicts.add(Pair.of(valid.scheme, actual.scheme));
+ }
+
+ if (!authorityMatch) {
+ conflicts.add(Pair.of(valid.authority, actual.authority));
+ }
+
+ return null;
+ }
+ }
+
+ private boolean uriComponentMatch(String valid, String actual) {
+ return Strings.isNullOrEmpty(valid) || valid.equalsIgnoreCase(actual);
+ }
+ }
+
+ @VisibleForTesting
+ static class StringToFileURI extends ToFileURI<String> {
Review Comment:
Actually, I had to revert this change as tests started to fail. Spark is
picky with closures and the closure wasn't serializable.
--
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]