You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/05/07 18:13:44 UTC

[GitHub] [iceberg] karuppayya opened a new pull request #2564: ICEBERG-2559: Add utility methods

karuppayya opened a new pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564


   This is a precursor for PR https://github.com/apache/iceberg/pull/2415
   Check comment https://github.com/apache/iceberg/pull/2415#discussion_r627834645 for more details


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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629590394



##########
File path: core/src/test/java/org/apache/iceberg/util/TestMetadataLocationUtils.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.iceberg.util;
+
+import java.io.File;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.MetadataLocationUtils;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public class TestMetadataLocationUtils {
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withRecordCount(1)
+      .build();
+  private static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    File tableDir = temp.newFolder();
+    String tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+
+  @Test
+  public void testManifestListPaths() {
+    table.newAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    List<String> manifestListPaths = MetadataLocationUtils.manifestListPaths(table);
+    Assert.assertEquals(manifestListPaths.size(), 2);
+  }
+
+  @Test
+  public void testMiscMetadataFiles() {
+    table.updateProperties()
+        .set(TableProperties.METADATA_PREVIOUS_VERSIONS_MAX, "1")
+        .commit();
+
+    table.newAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    List<String> miscMetadataFilePaths = MetadataLocationUtils
+        .miscMetadataFiles(((HasTableOperations) table).operations(), true);
+    Assert.assertEquals(miscMetadataFilePaths.size(), 5);
+
+    miscMetadataFilePaths = MetadataLocationUtils
+        .miscMetadataFiles(((HasTableOperations) table).operations(), false);
+    Assert.assertEquals(miscMetadataFilePaths.size(), 3);
+  }
+
+  @Test
+  public void testMiscMetadataFilesWithMissingFiles() {
+    table.updateProperties()
+        .set(TableProperties.METADATA_PREVIOUS_VERSIONS_MAX, "2")
+        .commit();
+
+    table.newAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    TableOperations operations = ((HasTableOperations) table).operations();
+    String location = operations.current().metadataFileLocation();
+    table.newAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    // delete v3.metadata.json making v2.metadata.json and v1.metadata.json inaccessible
+    table.io().deleteFile(location);

Review comment:
       Can we add a test case that metadata.json files aren't reachable? For example,  we can delete both v2 and v3 here, so v1 is not reachable.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r630685453



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtils.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtils.class);
+
+  private ReachableFileUtils() {
+  }
+
+  /**
+   * Returns the location of version.text file
+   * @param table table whose version.text path needs to be retrieved
+   * @return the path to version.text

Review comment:
       nit: `return the location of the version hint file`

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtils.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtils.class);
+
+  private ReachableFileUtils() {
+  }
+
+  /**
+   * Returns the location of version.text file
+   * @param table table whose version.text path needs to be retrieved
+   * @return the path to version.text
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns the metadata.json files associated with {@code table}
+   * @param table table to get the metadata json files from
+   * @param recursive
+   * <p>When true, recursively retrieves all the reachable metadata.json files.
+   * <p>when false, gets the all the metadata.json files only from the current metadata.
+   * @return a list of paths to metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = new HashSet<>();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean isRecursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
+      metaFiles.addAll(previousMetadataFiles);
+      // Find the first existent metadata json file and recurse
+      if (isRecursive) {
+        for (String metadataFileLocation : previousMetadataFiles) {
+          try {
+            TableMetadata newMetadata = TableMetadataParser.read(io, metadataFileLocation);
+            metadataFileLocations(newMetadata, metaFiles, io, isRecursive);
+            break;
+          } catch (Exception e) {
+            LOG.error("Failed to load {}", metadataFileLocation, e);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Returns all the path locations of all Manifest Lists for a given table

Review comment:
       nit: `Returns locations of manifest lists in a table.`
   
   Let's add an empty line before params too.

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtils.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtils {

Review comment:
       nit: Iceberg mostly calls utility classes as `xxxUtil`, not `xxxUtils`.

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtils.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtils.class);
+
+  private ReachableFileUtils() {
+  }
+
+  /**
+   * Returns the location of version.text file

Review comment:
       nit: `... of the version hint file.`
   
   Let's also add an empty line before the params for readability.

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtils.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtils.class);
+
+  private ReachableFileUtils() {
+  }
+
+  /**
+   * Returns the location of version.text file
+   * @param table table whose version.text path needs to be retrieved
+   * @return the path to version.text
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns the metadata.json files associated with {@code table}
+   * @param table table to get the metadata json files from
+   * @param recursive
+   * <p>When true, recursively retrieves all the reachable metadata.json files.
+   * <p>when false, gets the all the metadata.json files only from the current metadata.
+   * @return a list of paths to metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = new HashSet<>();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean isRecursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =

Review comment:
       This creates a temp list that gets added to the set immediately. Let's not add the overhead.

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtils.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtils.class);
+
+  private ReachableFileUtils() {
+  }
+
+  /**
+   * Returns the location of version.text file
+   * @param table table whose version.text path needs to be retrieved
+   * @return the path to version.text
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns the metadata.json files associated with {@code table}
+   * @param table table to get the metadata json files from
+   * @param recursive
+   * <p>When true, recursively retrieves all the reachable metadata.json files.
+   * <p>when false, gets the all the metadata.json files only from the current metadata.
+   * @return a list of paths to metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = new HashSet<>();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean isRecursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
+      metaFiles.addAll(previousMetadataFiles);
+      // Find the first existent metadata json file and recurse

Review comment:
       I think we better give up if the oldest one is not reachable. Trying metadata files one by one will send a request each time. If the lineage is broken, that's not our fault and it will not impact correctness.

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtils.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtils.class);
+
+  private ReachableFileUtils() {
+  }
+
+  /**
+   * Returns the location of version.text file
+   * @param table table whose version.text path needs to be retrieved
+   * @return the path to version.text
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns the metadata.json files associated with {@code table}
+   * @param table table to get the metadata json files from
+   * @param recursive
+   * <p>When true, recursively retrieves all the reachable metadata.json files.

Review comment:
       Can we move this description into the main Javadoc?

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtils.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtils.class);
+
+  private ReachableFileUtils() {
+  }
+
+  /**
+   * Returns the location of version.text file
+   * @param table table whose version.text path needs to be retrieved
+   * @return the path to version.text
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns the metadata.json files associated with {@code table}

Review comment:
       nit: `Returns locations of JSON metadata files in a table.`
   
   Let's also add an empty line before params.

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtils.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtils.class);
+
+  private ReachableFileUtils() {
+  }
+
+  /**
+   * Returns the location of version.text file
+   * @param table table whose version.text path needs to be retrieved
+   * @return the path to version.text
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns the metadata.json files associated with {@code table}
+   * @param table table to get the metadata json files from
+   * @param recursive
+   * <p>When true, recursively retrieves all the reachable metadata.json files.
+   * <p>when false, gets the all the metadata.json files only from the current metadata.
+   * @return a list of paths to metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = new HashSet<>();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,

Review comment:
       I think I still prefer something like [this](https://github.com/apache/iceberg/pull/2564#discussion_r629842136) without a separate method. 

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtils.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtils.class);
+
+  private ReachableFileUtils() {
+  }
+
+  /**
+   * Returns the location of version.text file
+   * @param table table whose version.text path needs to be retrieved
+   * @return the path to version.text
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns the metadata.json files associated with {@code table}
+   * @param table table to get the metadata json files from
+   * @param recursive
+   * <p>When true, recursively retrieves all the reachable metadata.json files.
+   * <p>when false, gets the all the metadata.json files only from the current metadata.
+   * @return a list of paths to metadata files

Review comment:
       nit: `locations of JSON metadata files`

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtils.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtils.class);
+
+  private ReachableFileUtils() {
+  }
+
+  /**
+   * Returns the location of version.text file
+   * @param table table whose version.text path needs to be retrieved
+   * @return the path to version.text
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns the metadata.json files associated with {@code table}
+   * @param table table to get the metadata json files from
+   * @param recursive
+   * <p>When true, recursively retrieves all the reachable metadata.json files.
+   * <p>when false, gets the all the metadata.json files only from the current metadata.
+   * @return a list of paths to metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = new HashSet<>();

Review comment:
       nit: `Sets.newHashSet()`




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629757920



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       This is required for https://github.com/apache/iceberg/pull/2415#discussion_r623479608, Yes listing directory would be ideal. Currently Icebereg doesn't support listing dirs and wanted to take that up as part of different 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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r633697226



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.iceberg;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version hint file
+   *
+   * @param table table for which version hint file's path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation(Util.VERSION_HINT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param table     Table to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                  When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metadataFileLocations,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      List<String> metadataLocations = metadataLogEntries.stream()
+          .map(TableMetadata.MetadataLogEntry::file)
+          .collect(Collectors.toList());
+      metadataFileLocations.addAll(metadataLocations);
+      if (recursive) {
+        TableMetadata previousMetadata = findFirstExistentPreviousMetadata(metadataLocations, io);
+        if (previousMetadata != null) {
+          metadataFileLocations(previousMetadata, metadataFileLocations, io, recursive);
+        }
+      }
+    }
+  }
+
+  private static TableMetadata findFirstExistentPreviousMetadata(List<String> metadataLocations, FileIO io) {
+    TableMetadata metadata = null;
+    for (String metadataLocation : metadataLocations) {
+      try {
+        metadata =  TableMetadataParser.read(io, metadataLocation);

Review comment:
       nit: Extra space before `TableMetadataParser`

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.iceberg;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version hint file
+   *
+   * @param table table for which version hint file's path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation(Util.VERSION_HINT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param table     Table to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                  When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metadataFileLocations,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();

Review comment:
       nit: let's import `MetadataLogEntry` directly to shorten the lines. 

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.iceberg;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version hint file
+   *
+   * @param table table for which version hint file's path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation(Util.VERSION_HINT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param table     Table to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                  When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metadataFileLocations,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      List<String> metadataLocations = metadataLogEntries.stream()

Review comment:
       nit: I think it would be best to avoid the temp list here as this method maybe called in a memory-sensitive env. I liked the previous implementation when we just iterated through `MetadataLogEntry`. You can also pass log entries to the method below.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java
##########
@@ -167,19 +133,21 @@ protected Table newStaticTable(TableMetadata metadata, FileIO io) {
   }
 
   protected Dataset<Row> buildManifestListDF(Table table) {
-    List<String> manifestLists = getManifestListPaths(table.snapshots());
+    List<String> manifestLists = ReachableFileUtil.manifestListLocations(table);
     return spark.createDataset(manifestLists, Encoders.STRING()).toDF("file_path");
   }
 
-  protected Dataset<Row> buildOtherMetadataFileDF(TableOperations ops) {
-    List<String> otherMetadataFiles = getOtherMetadataFilePaths(ops);
-    return spark.createDataset(otherMetadataFiles, Encoders.STRING()).toDF("file_path");
+  protected Dataset<Row> buildOtherMetadataFileDF(Table table) {
+    Set<String> otherMetadataFiles = ReachableFileUtil.metadataFileLocations(table, false);

Review comment:
       nit: I think it is a little confusing to call metadata file locations as otherMetadataFiles and then add version hint to it. Plus, the set returned from the utility may be immutable in the future. 
   
   What about this?
   
   ```
   List<String> otherMetadataFiles = Lists.newArrayList();
   otherMetadataFiles.addAll(ReachableFileUtil.metadataFileLocations(table, false));
   otherMetadataFiles.add(ReachableFileUtil.versionHintLocation(table));
   return spark.createDataset(otherMetadataFiles, Encoders.STRING()).toDF("file_path");
   ```




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r631289564



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java
##########
@@ -109,40 +110,6 @@ protected JobGroupInfo newJobGroupInfo(String groupId, String desc) {
     return new JobGroupInfo(groupId + "-" + JOB_COUNTER.incrementAndGet(), desc, false);
   }
 
-  /**
-   * Returns all the path locations of all Manifest Lists for a given list of snapshots
-   * @param snapshots snapshots
-   * @return the paths of the Manifest Lists
-   */
-  private List<String> getManifestListPaths(Iterable<Snapshot> snapshots) {

Review comment:
       Can we remove the same method in class `BaseAction`? This applies to `getOtherMetadataFilePaths` as well. 




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r631331589



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean isRecursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      for (TableMetadata.MetadataLogEntry metadataLogEntry : metadataLogEntries) {
+        metaFiles.add(metadataLogEntry.file());
+      }
+      if (isRecursive) {

Review comment:
       I was doing doing that initially. Based on https://github.com/apache/iceberg/pull/2564#discussion_r630695415, I am now checking the only the oldest file and bailing out if not existent.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#issuecomment-842693596


   Thanks, @karuppayya! Thanks for reviewing, @flyrain @kbendick @RussellSpitzer!


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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629742472



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       As suggested in https://github.com/apache/iceberg/pull/2564#discussion_r629586988,  once we remove the `break`, yes we will end up reading the same files multiple times.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r631294010



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean isRecursive) {

Review comment:
       A nit: let's be consistent of naming. isRecursive -> recursive.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629744013



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       Maybe we should have that be a "findFirst" then instead of the break? To make things a bit more straightforward?




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r632841336



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation(Util.VERSION_HINT_TXT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      for (TableMetadata.MetadataLogEntry metadataLogEntry : metadataLogEntries) {

Review comment:
       `MetadataLogEntry` is  a static inner class, we would not be able to do a static import of the class.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#issuecomment-834667332


   cc: @aokolnychyi @RussellSpitzer @flyrain @kbendick for review. Thanks


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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#issuecomment-837326826


   Sorry for the delay @karuppayya, let me take a look now.


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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629742472



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       As suggested in https://github.com/apache/iceberg/pull/2564#discussion_r629586988,  once we remove the `break`(which will help us find more files), yes we will end up reading the same files multiple times.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r630330950



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {

Review comment:
       `MetadataFileUtil`?




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r631292681



##########
File path: core/src/main/java/org/apache/iceberg/MetadataTableUtils.java
##########
@@ -23,6 +23,7 @@
 import org.apache.iceberg.exceptions.NoSuchTableException;
 
 public class MetadataTableUtils {

Review comment:
       Do we need this class?




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r633733352



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.iceberg;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version hint file
+   *
+   * @param table table for which version hint file's path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation(Util.VERSION_HINT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param table     Table to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                  When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metadataFileLocations,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();

Review comment:
       org.apache.iceberg.TableMetadata.MetadataLogEntry is a static inner class of TableMetadata, I think Java does allow static import of class(Only methods and field members).




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r632662150



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {

Review comment:
       I actually liked passing `Table` objects in the initial implementation as `TableOperations` is a very low-level API that is subject to change/be replaced. We are trying to avoid exposing it too much. I've refactored a few places in actions to work with `Table` and I think the only remaining part is getting all reachable files. It may be a good time to refactor the remaining parts in the actions that rely on `TableOperations`.

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file

Review comment:
       nit: It is not `version.txt`, so we should either say `of the version hint file` or `of the version-hint.text file`. 

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation(Util.VERSION_HINT_TXT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,

Review comment:
       nit: `metaFiles` -> `metadataFileLocations`

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation(Util.VERSION_HINT_TXT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      for (TableMetadata.MetadataLogEntry metadataLogEntry : metadataLogEntries) {
+        metaFiles.add(metadataLogEntry.file());
+      }
+      if (recursive) {
+        String metadataFileLocation = metadataLogEntries.get(0).file();
+        try {
+          TableMetadata newMetadata = TableMetadataParser.read(io, metadataFileLocation);

Review comment:
       nit: `newMetadata` -> `previousMetadata` or something?

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation(Util.VERSION_HINT_TXT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      for (TableMetadata.MetadataLogEntry metadataLogEntry : metadataLogEntries) {
+        metaFiles.add(metadataLogEntry.file());
+      }
+      if (recursive) {
+        String metadataFileLocation = metadataLogEntries.get(0).file();
+        try {
+          TableMetadata newMetadata = TableMetadataParser.read(io, metadataFileLocation);
+          metadataFileLocations(newMetadata, metaFiles, io, recursive);
+        } catch (Exception e) {
+          LOG.error("Failed to load {}", metadataFileLocation, e);
+        }
+      }
+    }
+  }
+
+  /**
+   * Returns locations of manifest lists in a table.
+   *
+   * @param table table for which manifestList needs to be fetched
+   * @return the paths of the Manifest Lists

Review comment:
       nit: `@return locations of manifest lists`

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved

Review comment:
       nit: same here

##########
File path: core/src/main/java/org/apache/iceberg/hadoop/Util.java
##########
@@ -38,6 +38,9 @@
 import org.slf4j.LoggerFactory;
 
 public class Util {
+
+  public static final String VERSION_HINT_TXT_FILENAME = "version-hint.text";

Review comment:
       +1 for creating a constant.
   
   I think we can drop `TXT` in the constant name, `VERSION_HINT_FILENAME` should be descriptive enough.

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation(Util.VERSION_HINT_TXT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      for (TableMetadata.MetadataLogEntry metadataLogEntry : metadataLogEntries) {
+        metaFiles.add(metadataLogEntry.file());
+      }
+      if (recursive) {
+        String metadataFileLocation = metadataLogEntries.get(0).file();

Review comment:
       Would it make sense to name it like `oldestMetadataFileLocation` or `previousMetadataFileLocation`?

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation(Util.VERSION_HINT_TXT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      for (TableMetadata.MetadataLogEntry metadataLogEntry : metadataLogEntries) {

Review comment:
       Can we add an import for `MetadataLogEntry` to shorten the lines?

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation(Util.VERSION_HINT_TXT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.

Review comment:
       nit: can we align both `when`?

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation(Util.VERSION_HINT_TXT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      for (TableMetadata.MetadataLogEntry metadataLogEntry : metadataLogEntries) {
+        metaFiles.add(metadataLogEntry.file());
+      }
+      if (recursive) {
+        String metadataFileLocation = metadataLogEntries.get(0).file();
+        try {
+          TableMetadata newMetadata = TableMetadataParser.read(io, metadataFileLocation);
+          metadataFileLocations(newMetadata, metaFiles, io, recursive);
+        } catch (Exception e) {
+          LOG.error("Failed to load {}", metadataFileLocation, e);
+        }
+      }
+    }
+  }
+
+  /**
+   * Returns locations of manifest lists in a table.
+   *
+   * @param table table for which manifestList needs to be fetched
+   * @return the paths of the Manifest Lists
+   */
+  public static List<String> manifestListLocations(Table table) {
+    Iterable<Snapshot> snapshots = table.snapshots();
+    List<String> manifestListLocations = new ArrayList<>();

Review comment:
       nit: `Lists.newArrayList()`

##########
File path: core/src/test/java/org/apache/iceberg/util/TestReachableFileUtil.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.iceberg.util;
+
+import java.io.File;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.ReachableFileUtil;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public class TestReachableFileUtil {
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withRecordCount(1)
+      .build();
+  private static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    File tableDir = temp.newFolder();
+    String tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+

Review comment:
       nit: extra line

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java
##########
@@ -109,40 +110,6 @@ protected JobGroupInfo newJobGroupInfo(String groupId, String desc) {
     return new JobGroupInfo(groupId + "-" + JOB_COUNTER.incrementAndGet(), desc, false);
   }
 
-  /**
-   * Returns all the path locations of all Manifest Lists for a given list of snapshots
-   * @param snapshots snapshots
-   * @return the paths of the Manifest Lists
-   */
-  private List<String> getManifestListPaths(Iterable<Snapshot> snapshots) {

Review comment:
       Yeah, we should clean up `BaseAction`. I am fine doing that in a follow-up PR too. @karuppayya, could you submit one when the current is in?




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r631342697



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtils.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtils.class);
+
+  private ReachableFileUtils() {
+  }
+
+  /**
+   * Returns the location of version.text file
+   * @param table table whose version.text path needs to be retrieved
+   * @return the path to version.text
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns the metadata.json files associated with {@code table}
+   * @param table table to get the metadata json files from
+   * @param recursive
+   * <p>When true, recursively retrieves all the reachable metadata.json files.
+   * <p>when false, gets the all the metadata.json files only from the current metadata.
+   * @return a list of paths to metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = new HashSet<>();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,

Review comment:
       The original method takes `TableOperations` as input but the recursion happens on `TableMetadata`. Hence I think a second method is required. Please Letting me know if i am missing something here




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r633733352



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.iceberg;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version hint file
+   *
+   * @param table table for which version hint file's path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation(Util.VERSION_HINT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param table     Table to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                  When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metadataFileLocations,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();

Review comment:
       org.apache.iceberg.TableMetadata.MetadataLogEntry is a static inner class of TableMetadata, I think Java does allow static import of class(Only methods and field members).




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r632680898



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean isRecursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      for (TableMetadata.MetadataLogEntry metadataLogEntry : metadataLogEntries) {
+        metaFiles.add(metadataLogEntry.file());
+      }
+      if (isRecursive) {

Review comment:
       I am a bit worried about the number of requests we can send that way. I feel like if the lineage broken then it is fine to give up. That being said, I am also fine if both of you think it is going to be helpful. I agree a separate method should simplify it a bit.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629746951



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       oh, wait, to get the first isn't correct here. Image you have v1 v2 v3 and v4, METADATA_PREVIOUS_VERSIONS_MAX is 2. You can delete v2 in v4. v2 is the first one, if you only search for the first one, you are going to miss things in v3.metadata.json, which points to v1 and v2.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629765099



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       1. In that sense, can we remove method getOtherMetadataFilePaths and getManifestListPaths in BaseAction?
   2. Not sure I understand, does removal of orphan files need all reachable metadata json files? Base on comment from #2415, maybe we could hold on the implementation recursive part?




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629841356



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       I think we can try to load previous metadata files but give up if we cannot do that. It should be okay to leave those files too. In case if Iceberg owns the location, we will drop the entire location after we clean up whatever we could reach.
   
   Listing may not be an option as the same location may contain files for multiple tables. In HiveCatalog, we have table UUID as part of the file name while we don't have that in HadoopCatalog for example. 
   
   If it is too complicated, we can drop the recursion too and I say we only clean up files currently referenced. Shouldn't be too bad.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r631337743



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean isRecursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      for (TableMetadata.MetadataLogEntry metadataLogEntry : metadataLogEntries) {
+        metaFiles.add(metadataLogEntry.file());
+      }
+      if (isRecursive) {

Review comment:
       Thanks @karuppayya. I think your initial way makes sense, for implementation, I'd suggest to wrap the logic into a small method like `findFirstExistedJson`. Let's hear thoughts from @aokolnychyi.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629726033



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       In the above case, we will first pick the latest version(ie file 3), add all the files in file 3.
   Pick the oldest exist file from the list of files in file 3, and recurse with that file.
   ```
         if (isRecursive) {
           for (String fileName : previousMetadataFiles) {
             // Find the first existent metadata json file and recurse
             if (io.newInputFile(fileName).exists()) {
               miscMetadataFiles(fileName, metaFiles, io, isRecursive);
               break;
             }
           }
         }
   ```
   We dont queue up all the files present in a particular file. The max number of files queued up will be 1




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629841356



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       I think we can try to load previous metadata files but give up if we cannot do that. It should be okay to leave those files too. In case if Iceberg owns the location, we will drop the entire location after that too. If not, we can leave them be.
   
   Listing may not be an option as the same location may contain files for multiple tables. In HiveCatalog, we have table UUID as part of the file name while we don't have that in HadoopCatalog for example. 
   
   If it is too complicated, we can drop the recursion too and I say we only clean up files currently referenced.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629755124



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       What is the use case of getting all metadata.json files? How about listing all *metadata.json files in the dir?




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629843002



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {
+    Set<String> miscMetadataLocation = new HashSet<>();
+    miscMetadataLocation.add(ops.metadataFileLocation("version-hint.text"));
+    String location = ops.current().metadataFileLocation();
+    miscMetadataLocation.add(location);
+    miscMetadataFiles(location, miscMetadataLocation, ops.io(), isRecursive);
+    return new ArrayList<>(miscMetadataLocation);
+  }
+
+  private static void miscMetadataFiles(String metadataFileLocation, Set<String> metaFiles,
+                                        FileIO io, boolean isRecursive) {
+    if (metadataFileLocation == null) {
+      return;
+    }
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
+      metaFiles.addAll(previousMetadataFiles);
+      if (isRecursive) {
+        for (String fileName : previousMetadataFiles) {
+          // Find the first existent metadata json file and recurse
+          if (io.newInputFile(fileName).exists()) {
+            miscMetadataFiles(fileName, metaFiles, io, isRecursive);
+            break;
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Returns all the path locations of all Manifest Lists for a given table
+   * @param table table for which manifestList needs to be fetched
+   * @return the paths of the Manifest Lists
+   */
+  public static List<String> manifestListPaths(Table table) {
+    Iterable<Snapshot> snapshots = table.snapshots();
+    List<String> manifestLists = new ArrayList<>();

Review comment:
       nit: `manifestLists` -> `manifestListLocations`




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629841356



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       I think we can try to load previous metadata files but give up if we cannot do that. It should be okay to leave those files too. In case if Iceberg owns the location, we will drop the entire location after we clean up whatever we could reach.
   
   Listing may not be an option as the same location may contain files for multiple tables. In HiveCatalog, we have table UUID as part of the file name while we don't have that in HadoopCatalog for example. In addition, there may be quite some metadata files and listing can be expensive.
   
   If it is too complicated, we can drop the recursion too and I say we only clean up files currently referenced. Shouldn't be too bad.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r633706243



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtils.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtils.class);
+
+  private ReachableFileUtils() {
+  }
+
+  /**
+   * Returns the location of version.text file
+   * @param table table whose version.text path needs to be retrieved
+   * @return the path to version.text
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns the metadata.json files associated with {@code table}
+   * @param table table to get the metadata json files from
+   * @param recursive
+   * <p>When true, recursively retrieves all the reachable metadata.json files.
+   * <p>when false, gets the all the metadata.json files only from the current metadata.
+   * @return a list of paths to metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = new HashSet<>();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,

Review comment:
       Resolving this.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629746951



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       oh, wait, to get the first isn't correct here. Image you have v1 v2 v3 and v4, METADATA_PREVIOUS_VERSIONS_MAX is 2. You can delete v2.metadata.json. v2 is the first one in v4, if you only search for the first one, you are going to miss things in v3.metadata.json, which points to v1 and v2.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629843619



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {

Review comment:
       I think it is also more accurate to call them `xxxLocation` and `xxxLocations`.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi merged pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi merged pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564


   


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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r633733352



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.iceberg;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version hint file
+   *
+   * @param table table for which version hint file's path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation(Util.VERSION_HINT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param table     Table to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                  When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metadataFileLocations,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();

Review comment:
       ~org.apache.iceberg.TableMetadata.MetadataLogEntry is a static inner class of TableMetadata, I think Java does allow static import of class(Only methods and field members).~ Thanks @aokolnychyi , I have added the import now.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r633706071



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       Resolving this.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629586988



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {
+    Set<String> miscMetadataLocation = new HashSet<>();
+    miscMetadataLocation.add(ops.metadataFileLocation("version-hint.text"));
+    String location = ops.current().metadataFileLocation();
+    miscMetadataLocation.add(location);
+    miscMetadataFiles(location, miscMetadataLocation, ops.io(), isRecursive);
+    return new ArrayList<>(miscMetadataLocation);
+  }
+
+  private static void miscMetadataFiles(String metadataFileLocation, Set<String> metaFiles,
+                                        FileIO io, boolean isRecursive) {
+    if (metadataFileLocation == null) {
+      return;
+    }
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
+      metaFiles.addAll(previousMetadataFiles);
+      if (isRecursive) {
+        for (String fileName : previousMetadataFiles) {
+          // Find the first existent metadata json file and recurse
+          if (io.newInputFile(fileName).exists()) {
+            miscMetadataFiles(fileName, metaFiles, io, isRecursive);
+            break;

Review comment:
       Why `break`? When I look at the test `testMiscMetadataFilesWithMissingFiles`, what if v3.metadata.json be the first one in the loop?  In that case, v1 isn't reachable since v3 has been deleted, and the loop will be terminated because of the `break`, so it won't open v2.metadata.json and check recursively.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629671612



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {
+    Set<String> miscMetadataLocation = new HashSet<>();
+    miscMetadataLocation.add(ops.metadataFileLocation("version-hint.text"));
+    String location = ops.current().metadataFileLocation();
+    miscMetadataLocation.add(location);
+    miscMetadataFiles(location, miscMetadataLocation, ops.io(), isRecursive);
+    return new ArrayList<>(miscMetadataLocation);
+  }
+
+  private static void miscMetadataFiles(String metadataFileLocation, Set<String> metaFiles,
+                                        FileIO io, boolean isRecursive) {
+    if (metadataFileLocation == null) {
+      return;
+    }
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
+      metaFiles.addAll(previousMetadataFiles);
+      if (isRecursive) {
+        for (String fileName : previousMetadataFiles) {
+          // Find the first existent metadata json file and recurse
+          if (io.newInputFile(fileName).exists()) {
+            miscMetadataFiles(fileName, metaFiles, io, isRecursive);
+            break;

Review comment:
       I'd skip the break too




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629842136



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       I think recursion makes it a bit hard to read and there is another problem. We will always open the metadata file even for getting the previous files from the current metadata. That was not the case before.
   
   We could have something like this:
   
   ```
     public static Set<String> metadataFileLocations(Table table, boolean recursive) {
       Set<String> metadataFileLocations = Sets.newHashSet();
   
       TableOperations ops = ((HasTableOperations) table).operations();
       TableMetadata metadata = ops.current();
   
       while (metadata != null) {
         metadataFileLocations.add(metadata.metadataFileLocation());
   
         List<MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
   
         for (MetadataLogEntry metadataLogEntry : metadataLogEntries) {
           metadataFileLocations.add(metadataLogEntry.file());
         }
   
         if (recursive && metadataLogEntries.size() > 0) {
           String oldestMetadataFileLocation = metadataLogEntries.get(0).file();
           try {
             metadata = TableMetadataParser.read(table.io(), oldestMetadataFileLocation);
           } catch (Exception e) {
             LOG.error("Failed to load {}", oldestMetadataFileLocation, e);
             break;
           }
         } else {
           break;
         }
       }
   
       return metadataFileLocations;
     }
   ```
   
   Just an idea.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629548324



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.

Review comment:
       Somewhat of a nit: This doc comment seems partially incorrect to me. I think this is only gathering entries in `metadata.previousFiles` when `isRecursive` is true, if I understand correctly.
   
   If my understanding is right, maybe consider `this includes "version-hint" files, and also includes entries from all metadata.previousFiles when {@param isRecursive} is true.`?

##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {
+    Set<String> miscMetadataLocation = new HashSet<>();
+    miscMetadataLocation.add(ops.metadataFileLocation("version-hint.text"));

Review comment:
       Unrelated to your PR, but I see this string hardcoded in a number of places and it's always felt to me like it should be a constant.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629839567



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {

Review comment:
       What about splitting this into two separate methods and accepting `Table`?
   
   ```
   public static String versionHintLocation(Table table) {
       TableOperations ops = ((HasTableOperations) table).operations();
       return ops.metadataFileLocation("version-hint.text");
   }
   
   public static Set<String> metadataFileLocations(Table table, boolean recursive) {
     ...
   }
   ```
   
   Then we can use return a `Set` with metadata locations directly. We can then construct whatever is needed by Spark or another query engine.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r632841535



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {

Review comment:
       I have refactored to use `Table` objects




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629842703



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {
+    Set<String> miscMetadataLocation = new HashSet<>();
+    miscMetadataLocation.add(ops.metadataFileLocation("version-hint.text"));
+    String location = ops.current().metadataFileLocation();
+    miscMetadataLocation.add(location);
+    miscMetadataFiles(location, miscMetadataLocation, ops.io(), isRecursive);
+    return new ArrayList<>(miscMetadataLocation);
+  }
+
+  private static void miscMetadataFiles(String metadataFileLocation, Set<String> metaFiles,
+                                        FileIO io, boolean isRecursive) {
+    if (metadataFileLocation == null) {
+      return;
+    }
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
+      metaFiles.addAll(previousMetadataFiles);
+      if (isRecursive) {
+        for (String fileName : previousMetadataFiles) {
+          // Find the first existent metadata json file and recurse
+          if (io.newInputFile(fileName).exists()) {
+            miscMetadataFiles(fileName, metaFiles, io, isRecursive);
+            break;
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Returns all the path locations of all Manifest Lists for a given table
+   * @param table table for which manifestList needs to be fetched
+   * @return the paths of the Manifest Lists
+   */
+  public static List<String> manifestListPaths(Table table) {
+    Iterable<Snapshot> snapshots = table.snapshots();
+    List<String> manifestLists = new ArrayList<>();

Review comment:
       nit: I have nothing against constructing `ArrayList`s directly but historically Iceberg uses `Lists.newArrayList()` so I think it makes sense to be consistent with the rest of the codebase.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629751306



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       We need to find the first existent file, like in my previous logic. 
   But there will be issues with that as well. There can be unreachable files like in this test case. https://github.com/apache/iceberg/pull/2564/files#diff-efb7c769faae3f19c11382d2dc4c53a69074265a87f7291f8430ad91410d48e2R141 
   Another case which might not work is updating `org.apache.iceberg.TableProperties#METADATA_PREVIOUS_VERSIONS_MAX` for a table, 
   ```
   say v101 has reference to 1-100
   Update org.apache.iceberg.TableProperties#METADATA_PREVIOUS_VERSIONS_MAX to 1
   v102 will have reference to v101
   v103 will have reference to v102(Let this be the latest version)
   ```
   If v 102 is removed(say as part of RemoveOrphanFiles), we would have lost all reference between 1 and 101.
   
   
   
   




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629586988



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {
+    Set<String> miscMetadataLocation = new HashSet<>();
+    miscMetadataLocation.add(ops.metadataFileLocation("version-hint.text"));
+    String location = ops.current().metadataFileLocation();
+    miscMetadataLocation.add(location);
+    miscMetadataFiles(location, miscMetadataLocation, ops.io(), isRecursive);
+    return new ArrayList<>(miscMetadataLocation);
+  }
+
+  private static void miscMetadataFiles(String metadataFileLocation, Set<String> metaFiles,
+                                        FileIO io, boolean isRecursive) {
+    if (metadataFileLocation == null) {
+      return;
+    }
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
+      metaFiles.addAll(previousMetadataFiles);
+      if (isRecursive) {
+        for (String fileName : previousMetadataFiles) {
+          // Find the first existent metadata json file and recurse
+          if (io.newInputFile(fileName).exists()) {
+            miscMetadataFiles(fileName, metaFiles, io, isRecursive);
+            break;

Review comment:
       Why `break`? When I look at the test `testMiscMetadataFilesWithMissingFiles`, what if v3.metadata.json be the first one in the loop?  In that case, v1 isn't reachable since v3 has been deleted, and the loop will be terminated because of the `break`.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r633705632



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean isRecursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      for (TableMetadata.MetadataLogEntry metadataLogEntry : metadataLogEntries) {
+        metaFiles.add(metadataLogEntry.file());
+      }
+      if (isRecursive) {

Review comment:
       Sounds good to me. Resolving this.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629751306



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       We need to find the first existent file, like in my previous logic. 
   But there will be issues with that as well. There can be unreachable files like in this test case. https://github.com/apache/iceberg/pull/2564/files#diff-efb7c769faae3f19c11382d2dc4c53a69074265a87f7291f8430ad91410d48e2R141 
   
   
   




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629604641



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {
+    Set<String> miscMetadataLocation = new HashSet<>();
+    miscMetadataLocation.add(ops.metadataFileLocation("version-hint.text"));
+    String location = ops.current().metadataFileLocation();
+    miscMetadataLocation.add(location);
+    miscMetadataFiles(location, miscMetadataLocation, ops.io(), isRecursive);
+    return new ArrayList<>(miscMetadataLocation);
+  }
+
+  private static void miscMetadataFiles(String metadataFileLocation, Set<String> metaFiles,
+                                        FileIO io, boolean isRecursive) {
+    if (metadataFileLocation == null) {
+      return;
+    }
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
+      metaFiles.addAll(previousMetadataFiles);
+      if (isRecursive) {
+        for (String fileName : previousMetadataFiles) {
+          // Find the first existent metadata json file and recurse
+          if (io.newInputFile(fileName).exists()) {
+            miscMetadataFiles(fileName, metaFiles, io, isRecursive);
+            break;
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Returns all the path locations of all Manifest Lists for a given list of snapshots

Review comment:
       I'd expect method parameter to be a list of snapshots. Is this method for a given table?




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629745508



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       Make sense to get the first item more explicitly. Another concern, is there a chance that order is broken? 




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629731744



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {
+    Set<String> miscMetadataLocation = new HashSet<>();
+    miscMetadataLocation.add(ops.metadataFileLocation("version-hint.text"));
+    String location = ops.current().metadataFileLocation();
+    miscMetadataLocation.add(location);
+    miscMetadataFiles(location, miscMetadataLocation, ops.io(), isRecursive);
+    return new ArrayList<>(miscMetadataLocation);
+  }
+
+  private static void miscMetadataFiles(String metadataFileLocation, Set<String> metaFiles,
+                                        FileIO io, boolean isRecursive) {
+    if (metadataFileLocation == null) {
+      return;
+    }
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
+      metaFiles.addAll(previousMetadataFiles);
+      if (isRecursive) {
+        for (String fileName : previousMetadataFiles) {
+          // Find the first existent metadata json file and recurse
+          if (io.newInputFile(fileName).exists()) {
+            miscMetadataFiles(fileName, metaFiles, io, isRecursive);
+            break;

Review comment:
       Looking at `org.apache.iceberg.TableMetadata#addPreviousFile(org.apache.iceberg.io.InputFile, long, java.util.Map<java.lang.String,java.lang.String>)`, it looks like the ordering of the previous files is maintained. The first accessible file will access to deepest file history. 
   But thinking again,  there can be cases where the `org.apache.iceberg.TableProperties#METADATA_PREVIOUS_VERSIONS_MAX`, could be updated and later files in the list might have access to more filenames than the current one. It make sense to remove the  `break`. The only catch here is we will be opening metadata files multiple times. 




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629838752



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {

Review comment:
       I am still thinking about the best name for this. I did suggest `MetadataLocationUtil` but I am no longer sure it is a good one. Since we will be dealing with files that are reachable, we can consider `ReachableFileUtil` or similar.
   
   Any ideas, @RussellSpitzer @karuppayya @flyrain @kbendick?




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629742510



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {
+    Set<String> miscMetadataLocation = new HashSet<>();
+    miscMetadataLocation.add(ops.metadataFileLocation("version-hint.text"));
+    String location = ops.current().metadataFileLocation();
+    miscMetadataLocation.add(location);
+    miscMetadataFiles(location, miscMetadataLocation, ops.io(), isRecursive);
+    return new ArrayList<>(miscMetadataLocation);
+  }
+
+  private static void miscMetadataFiles(String metadataFileLocation, Set<String> metaFiles,
+                                        FileIO io, boolean isRecursive) {
+    if (metadataFileLocation == null) {
+      return;
+    }
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
+      metaFiles.addAll(previousMetadataFiles);
+      if (isRecursive) {
+        for (String fileName : previousMetadataFiles) {
+          // Find the first existent metadata json file and recurse
+          if (io.newInputFile(fileName).exists()) {
+            miscMetadataFiles(fileName, metaFiles, io, isRecursive);
+            break;
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Returns all the path locations of all Manifest Lists for a given list of snapshots

Review comment:
       sorry, missed to fix the javadoc. Thanks!!




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629842296



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {
+    Set<String> miscMetadataLocation = new HashSet<>();
+    miscMetadataLocation.add(ops.metadataFileLocation("version-hint.text"));
+    String location = ops.current().metadataFileLocation();
+    miscMetadataLocation.add(location);
+    miscMetadataFiles(location, miscMetadataLocation, ops.io(), isRecursive);
+    return new ArrayList<>(miscMetadataLocation);
+  }
+
+  private static void miscMetadataFiles(String metadataFileLocation, Set<String> metaFiles,
+                                        FileIO io, boolean isRecursive) {
+    if (metadataFileLocation == null) {
+      return;
+    }
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
+      metaFiles.addAll(previousMetadataFiles);
+      if (isRecursive) {
+        for (String fileName : previousMetadataFiles) {
+          // Find the first existent metadata json file and recurse
+          if (io.newInputFile(fileName).exists()) {
+            miscMetadataFiles(fileName, metaFiles, io, isRecursive);
+            break;
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Returns all the path locations of all Manifest Lists for a given table
+   * @param table table for which manifestList needs to be fetched
+   * @return the paths of the Manifest Lists
+   */
+  public static List<String> manifestListPaths(Table table) {

Review comment:
       nit: `manifestListPaths` -> `manifestListLocations`




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r632681231



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean isRecursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      for (TableMetadata.MetadataLogEntry metadataLogEntry : metadataLogEntries) {
+        metaFiles.add(metadataLogEntry.file());
+      }
+      if (isRecursive) {

Review comment:
       Leave this up to you, @karuppayya @flyrain. 




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] karuppayya commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
karuppayya commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r632842020



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean isRecursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      for (TableMetadata.MetadataLogEntry metadataLogEntry : metadataLogEntries) {
+        metaFiles.add(metadataLogEntry.file());
+      }
+      if (isRecursive) {

Review comment:
       Add a additional method to find the first existent metadata and using it figure the metadata file locations.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r631307310



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean isRecursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      for (TableMetadata.MetadataLogEntry metadataLogEntry : metadataLogEntries) {
+        metaFiles.add(metadataLogEntry.file());
+      }
+      if (isRecursive) {

Review comment:
       Per off-line discussion, we are trying to find the first existing metadata.json file while recursive searching. Let's take the test as an example,  there are v1 v2 v3 and v4, METADATA_PREVIOUS_VERSIONS_MAX is 2. Inside the v4, you have v2 and v3, and v2 is the first item, if v2 is there, open v2 and continue searching, but if v2.metadata.json is deleted, we can open the second item(v3.metadata.json) and continue searching. It's simple logic-wise and perf is also good since we don't open the same file multiple times.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629679189



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       I think i missed the discussion on this, won't this recursion end up visiting the same files over and over?
   
   Ie we should be (by default) storing 100 entries in previous files
   Then every time we add a new entry it should be deleting the oldest one.
   So in our normal recursion we visit file 
   1 - 100 and queue up a recursion on all 100 of those
   On file 2 we then open up
   2 - 102 and queue up a recursion on all 100 of those
   on file 3
   3- 103 and queue up a recursion on 100 of those?
   
   So we end up opening up files 5 thousand or so times with the default history saving levels and we open up the same files up to 99 times right?
   




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r633697226



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.iceberg;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version hint file
+   *
+   * @param table table for which version hint file's path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation(Util.VERSION_HINT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param table     Table to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                  When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metadataFileLocations,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      List<String> metadataLocations = metadataLogEntries.stream()
+          .map(TableMetadata.MetadataLogEntry::file)
+          .collect(Collectors.toList());
+      metadataFileLocations.addAll(metadataLocations);
+      if (recursive) {
+        TableMetadata previousMetadata = findFirstExistentPreviousMetadata(metadataLocations, io);
+        if (previousMetadata != null) {
+          metadataFileLocations(previousMetadata, metadataFileLocations, io, recursive);
+        }
+      }
+    }
+  }
+
+  private static TableMetadata findFirstExistentPreviousMetadata(List<String> metadataLocations, FileIO io) {
+    TableMetadata metadata = null;
+    for (String metadataLocation : metadataLocations) {
+      try {
+        metadata =  TableMetadataParser.read(io, metadataLocation);

Review comment:
       nit: Extra space before `TableMetadataParser`

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.iceberg;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version hint file
+   *
+   * @param table table for which version hint file's path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation(Util.VERSION_HINT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param table     Table to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                  When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metadataFileLocations,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();

Review comment:
       nit: let's import `MetadataLogEntry` directly to shorten the lines. 

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.iceberg;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version hint file
+   *
+   * @param table table for which version hint file's path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation(Util.VERSION_HINT_FILENAME);
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param table     Table to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                  When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metadataFileLocations,
+                                            FileIO io, boolean recursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      List<String> metadataLocations = metadataLogEntries.stream()

Review comment:
       nit: I think it would be best to avoid the temp list here as this method maybe called in a memory-sensitive env. I liked the previous implementation when we just iterated through `MetadataLogEntry`. You can also pass log entries to the method below.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java
##########
@@ -167,19 +133,21 @@ protected Table newStaticTable(TableMetadata metadata, FileIO io) {
   }
 
   protected Dataset<Row> buildManifestListDF(Table table) {
-    List<String> manifestLists = getManifestListPaths(table.snapshots());
+    List<String> manifestLists = ReachableFileUtil.manifestListLocations(table);
     return spark.createDataset(manifestLists, Encoders.STRING()).toDF("file_path");
   }
 
-  protected Dataset<Row> buildOtherMetadataFileDF(TableOperations ops) {
-    List<String> otherMetadataFiles = getOtherMetadataFilePaths(ops);
-    return spark.createDataset(otherMetadataFiles, Encoders.STRING()).toDF("file_path");
+  protected Dataset<Row> buildOtherMetadataFileDF(Table table) {
+    Set<String> otherMetadataFiles = ReachableFileUtil.metadataFileLocations(table, false);

Review comment:
       nit: I think it is a little confusing to call metadata file locations as otherMetadataFiles and then add version hint to it. Plus, the set returned from the utility may be immutable in the future. 
   
   What about this?
   
   ```
   List<String> otherMetadataFiles = Lists.newArrayList();
   otherMetadataFiles.addAll(ReachableFileUtil.metadataFileLocations(table, false));
   otherMetadataFiles.add(ReachableFileUtil.versionHintLocation(table));
   return spark.createDataset(otherMetadataFiles, Encoders.STRING()).toDF("file_path");
   ```

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns locations of JSON metadata files in a table.
+   *
+   * @param ops       TableOperations to get JSON metadata files from
+   * @param recursive When true, recursively retrieves all the reachable JSON metadata files.
+   *                 When false, gets the all the JSON metadata files only from the current metadata.
+   * @return locations of JSON metadata files
+   */
+  public static Set<String> metadataFileLocations(TableOperations ops, boolean recursive) {
+    Set<String> metadataFileLocations = Sets.newHashSet();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,
+                                            FileIO io, boolean isRecursive) {
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    if (metadataLogEntries.size() > 0) {
+      for (TableMetadata.MetadataLogEntry metadataLogEntry : metadataLogEntries) {
+        metaFiles.add(metadataLogEntry.file());
+      }
+      if (isRecursive) {

Review comment:
       Sounds good to me. Resolving this.

##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -58,17 +53,22 @@ private static void miscMetadataFiles(String metadataFileLocation, Set<String> m
     if (metadataFileLocation == null) {
       return;
     }
-    try {
-      TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
-      List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
-      List<String> previousMetadataFiles =
-          metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file).collect(Collectors.toList());
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    List<TableMetadata.MetadataLogEntry> metadataLogEntries = metadata.previousFiles();
+    List<String> previousMetadataFiles =
+        metadataLogEntries.stream().map(TableMetadata.MetadataLogEntry::file)
+            .collect(Collectors.toList());
+    if (previousMetadataFiles.size() > 0) {
       metaFiles.addAll(previousMetadataFiles);
-      if (isRecursive && previousMetadataFiles.size() > 0) {
-        miscMetadataFiles(previousMetadataFiles.get(0), metaFiles, io, isRecursive);
+      if (isRecursive) {

Review comment:
       Resolving this.

##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtils.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtils.class);
+
+  private ReachableFileUtils() {
+  }
+
+  /**
+   * Returns the location of version.text file
+   * @param table table whose version.text path needs to be retrieved
+   * @return the path to version.text
+   */
+  public static String versionHintLocation(Table table) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return ops.metadataFileLocation("version-hint.text");
+  }
+
+  /**
+   * Returns the metadata.json files associated with {@code table}
+   * @param table table to get the metadata json files from
+   * @param recursive
+   * <p>When true, recursively retrieves all the reachable metadata.json files.
+   * <p>when false, gets the all the metadata.json files only from the current metadata.
+   * @return a list of paths to metadata files
+   */
+  public static Set<String> metadataFileLocations(Table table, boolean recursive) {
+    Set<String> metadataFileLocations = new HashSet<>();
+    TableOperations ops = ((HasTableOperations) table).operations();
+    TableMetadata tableMetadata = ops.current();
+    metadataFileLocations.add(tableMetadata.metadataFileLocation());
+    metadataFileLocations(tableMetadata, metadataFileLocations, ops.io(), recursive);
+    return metadataFileLocations;
+  }
+
+  private static void metadataFileLocations(TableMetadata metadata, Set<String> metaFiles,

Review comment:
       Resolving this.

##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {
+    Set<String> miscMetadataLocation = new HashSet<>();
+    miscMetadataLocation.add(ops.metadataFileLocation("version-hint.text"));

Review comment:
       This has been addressed.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r629839653



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {

Review comment:
       nit: `isRecursive` -> `recursive`




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r631291187



##########
File path: core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.iceberg;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReachableFileUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReachableFileUtil.class);
+
+  private ReachableFileUtil() {
+  }
+
+  /**
+   * Returns the location of the version.text file
+   *
+   * @param ops tableOperation for which version.text path needs to be retrieved
+   * @return the location of the version hint file
+   */
+  public static String versionHintLocation(TableOperations ops) {
+    return ops.metadataFileLocation("version-hint.text");

Review comment:
       Per @kbendick suggested, it is a good time to make a constant for "version-hint.text". But I'm OK to do this in another 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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2564: ICEBERG-2559: Add utility methods

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2564:
URL: https://github.com/apache/iceberg/pull/2564#discussion_r633706499



##########
File path: core/src/main/java/org/apache/iceberg/MetadataLocationUtils.java
##########
@@ -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.iceberg;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.io.FileIO;
+
+public class MetadataLocationUtils {
+
+  private MetadataLocationUtils() {
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations of the table to get paths from
+   * @param isRecursive when true, recursively retrieves all the metadata.json files using metadata.previousFiles
+   * <p>when false, gets the all the metadata.json files pointed to by the only current metadata.json
+   * @return a list of paths to metadata files
+   */
+  public static List<String> miscMetadataFiles(TableOperations ops, boolean isRecursive) {
+    Set<String> miscMetadataLocation = new HashSet<>();
+    miscMetadataLocation.add(ops.metadataFileLocation("version-hint.text"));

Review comment:
       This has been addressed.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org