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/06/01 17:48:13 UTC

[GitHub] [iceberg] anuragmantri opened a new pull request #2658: Support relative paths in metadata

anuragmantri opened a new pull request #2658:
URL: https://github.com/apache/iceberg/pull/2658


   Add initial support for relative paths in metadata(https://github.com/apache/iceberg/issues/1617).
   
   Summary of changes:
   - Introduce `write.metadata.use.relative-path` property that governs the use of relative paths
   - Modify metadata writers and readers to use relative paths when the above property is enabled
   - No changes were made to public APIs


-- 
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] anuragmantri commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: spark2/src/test/java/org/apache/iceberg/examples/RelativePathHiveFunctionalityTest.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.examples;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hive.HiveMetastoreTest;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.SparkConf;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public class RelativePathHiveFunctionalityTest extends HiveMetastoreTest {

Review comment:
       You are right. Renamed 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] anuragmantri commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/TableProperties.java
##########
@@ -177,6 +177,9 @@ private TableProperties() {
   public static final String UPDATE_ISOLATION_LEVEL = "write.update.isolation-level";
   public static final String UPDATE_ISOLATION_LEVEL_DEFAULT = "serializable";
 
+  public static final String WRITE_METADATA_USE_RELATIVE_PATH = "write.metadata.use.relative-path";

Review comment:
       Done




-- 
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] pvary commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: spark2/src/test/java/org/apache/iceberg/examples/RelativePathHiveFunctionalityTest.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.examples;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hive.HiveMetastoreTest;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.SparkConf;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public class RelativePathHiveFunctionalityTest extends HiveMetastoreTest {

Review comment:
       If I am correct then we try to name test to start with `Test`, like `TestRelativePathHiveFunctionality`




-- 
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] pvary commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/MetadataPaths.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPaths {
+
+  private MetadataPaths() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param properties table properties
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, Map<String, String> properties) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return useRelativePath(properties) && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;

Review comment:
       You are right. Would it be better to introduce something similar for metadata file locations as well? We can have 2 implementations:
   * AbsoluteMetadataLocationProvider
   * RelativeMetadataLocationProvider
   
   Or even a single implementation at this point handling both?
   This could hide the complexity of calculating the path whenever it is needed and we can just pass down this provider for every metadata file (even encapsulate the io, so we do not have to pass that around 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] pvary commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/ManifestEntriesTable.java
##########
@@ -126,10 +126,10 @@ public long targetSplitSize() {
     @Override
     public CloseableIterable<StructLike> rows() {
       if (manifest.content() == ManifestContent.DATA) {
-        return CloseableIterable.transform(ManifestFiles.read(manifest, io).project(fileSchema).entries(),
+        return CloseableIterable.transform(ManifestFiles.read(manifest, io, null, null).project(fileSchema).entries(),

Review comment:
       Again, keep the original signatures 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] szehon-ho commented on a change in pull request #2658: Core: Support relative paths in metadata

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on a change in pull request #2658:
URL: https://github.com/apache/iceberg/pull/2658#discussion_r665076345



##########
File path: core/src/main/java/org/apache/iceberg/BaseSnapshot.java
##########
@@ -78,6 +91,19 @@
     this.operation = operation;
     this.summary = summary;
     this.manifestListLocation = manifestList;
+    this.tableLocation = tableLocation;
+    this.shouldUseRelativePaths = shouldUseRelativePaths;

Review comment:
       Nit: was there a reason its 'shouldUseRelativePaths' vs 'useRelativePaths'?  Save a bit of ink ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] anuragmantri edited a comment on pull request #2658: Core: Support relative paths in metadata

Posted by GitBox <gi...@apache.org>.
anuragmantri edited a comment on pull request #2658:
URL: https://github.com/apache/iceberg/pull/2658#issuecomment-868771782


   CC: @dpaani - Who has also contributed to this work.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/TableProperties.java
##########
@@ -177,6 +177,9 @@ private TableProperties() {
   public static final String UPDATE_ISOLATION_LEVEL = "write.update.isolation-level";
   public static final String UPDATE_ISOLATION_LEVEL_DEFAULT = "serializable";
 
+  public static final String WRITE_METADATA_USE_RELATIVE_PATH = "write.metadata.use.relative-path";

Review comment:
       It is worth to comment out in the code that this property is immutable once the table is created.




-- 
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] pvary commented on pull request #2658: Core: Support relative paths in metadata

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


   @anuragmantri: Tried to go through the PR, but this one is huge 😄 
   
   Some general takeaways:
   - Can we keep the original signatures for the extended methods as well? They still can call the new ones, but if we kept the original ones then we do not have to change the callers to add `null, null` which could reduce the number of changes greatly
   - Can we decrease the number of parameters we push down the new methods? I would prefer to send a boolean `useRelativePath` instead of the whole property list, or even only send just the baseLocation if we have to use it.
   - Would it be possible to hide the whole stuff inside a LocationProvider? It could be a base implementation for all of the LocationProviders, and would be a natural place in my opinion. We can avoid changing the public APIs if we provide a default implementation.
   - If there is any way to split this PR to smaller, more reviewer friendly ones that would be a big help for us, reviewers.
   
   Thanks,
   Peter


-- 
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] pvary commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/MetadataPaths.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPaths {
+
+  private MetadataPaths() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param properties table properties
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, Map<String, String> properties) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return useRelativePath(properties) && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;

Review comment:
       For me extending the LocationProvider with additional default implementation which handles relative patch would be more natural than handling it in some random place in the code




-- 
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] pvary commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/MetadataPathUtils.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH;
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH_DEFAULT;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPathUtils {
+
+  private MetadataPathUtils() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param shouldUseRelativePaths whether relative paths should be used
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return shouldUseRelativePaths && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;

Review comment:
       I do not like concatenating "/" in path. Maybe using `Path` object would be better.




-- 
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 #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/MetadataPathUtils.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH;
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH_DEFAULT;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPathUtils {
+
+  private MetadataPathUtils() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param shouldUseRelativePaths whether relative paths should be used
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return shouldUseRelativePaths && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;
+  }
+
+  /**
+   * Convert a given absolute path to relative path with respect to base table location
+   * @param path the absolute path
+   * @param tableLocation the base table location
+   * @return relative path with respect to the base table location
+   */
+  public static String toRelativePath(String path, String tableLocation, boolean shouldUseRelativePaths) {

Review comment:
       do we need shouldUseRelativePaths here? Shouldn't the function toRelativePath always return a relative path? The caller would know whether it needs to call toRelativePath or toAbsolutePath?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/MetadataPathUtils.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH;
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH_DEFAULT;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPathUtils {
+
+  private MetadataPathUtils() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param shouldUseRelativePaths whether relative paths should be used
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return shouldUseRelativePaths && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;
+  }
+
+  /**
+   * Convert a given absolute path to relative path with respect to base table location
+   * @param path the absolute path
+   * @param tableLocation the base table location
+   * @return relative path with respect to the base table location
+   */
+  public static String toRelativePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to relative path by removing the table location
+    return shouldUseRelativePaths && path.startsWith(tableLocation) ?

Review comment:
       I think for both of these functions we are basically care more about the output than the input
   
   toAbsolute
   (Absolute Path | Relative to table Location) -> Absolute Path
   
   toRelative
   (Absolute Path | Relative to table location) -> relativizedPath to table location | absolutePath not including table location
   
   As @pvary notes it's possible for AbsolutePath to be in a completely different file system, in that case this method sometimes returns relative paths, but sometimes returns absolute paths.
   
   I think this is actually the right thing to do for this method since we can only ever deal with paths relative to the table location ... but maybe this needs another name though? Since currently it doesn't always return relative paths




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/MetadataPathUtils.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH;
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH_DEFAULT;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPathUtils {
+
+  private MetadataPathUtils() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param shouldUseRelativePaths whether relative paths should be used
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return shouldUseRelativePaths && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;

Review comment:
       Paths.get?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 #2658: Core: Support relative paths in metadata

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



##########
File path: api/src/main/java/org/apache/iceberg/Snapshot.java
##########
@@ -126,4 +127,10 @@
    * @return the location of the manifest list for this Snapshot
    */
   String manifestListLocation();
+
+  /**
+   *
+   * @return file io used for this snapshot
+   */
+  FileIO io();

Review comment:
       Why do we need 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] anuragmantri commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/ManifestEntriesTable.java
##########
@@ -126,10 +126,10 @@ public long targetSplitSize() {
     @Override
     public CloseableIterable<StructLike> rows() {
       if (manifest.content() == ManifestContent.DATA) {
-        return CloseableIterable.transform(ManifestFiles.read(manifest, io).project(fileSchema).entries(),
+        return CloseableIterable.transform(ManifestFiles.read(manifest, io, null, null).project(fileSchema).entries(),

Review comment:
       Done

##########
File path: core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java
##########
@@ -46,18 +49,28 @@ static InheritableMetadata forCopy(long snapshotId) {
     private final int specId;
     private final long snapshotId;
     private final long sequenceNumber;
+    private final String tableLocation;
+    private final Map<String, String> tableProperties;
 
-    private BaseInheritableMetadata(int specId, long snapshotId, long sequenceNumber) {
+    private BaseInheritableMetadata(int specId, long snapshotId, long sequenceNumber, String tableLocation,
+        Map<String, String> tableProperties) {
       this.specId = specId;
       this.snapshotId = snapshotId;
       this.sequenceNumber = sequenceNumber;
+      this.tableLocation = tableLocation;
+      this.tableProperties = tableProperties;
     }
 
     @Override
     public <F extends ContentFile<F>> ManifestEntry<F> apply(ManifestEntry<F> manifestEntry) {
       if (manifestEntry.file() instanceof BaseFile) {
         BaseFile<?> file = (BaseFile<?>) manifestEntry.file();
         file.setSpecId(specId);
+        if (MetadataPaths.useRelativePath(tableProperties)) {
+          if (!file.path().toString().startsWith(tableLocation)) {

Review comment:
       You are right. Pushed the logic into `toAbsolutePath()`.

##########
File path: core/src/main/java/org/apache/iceberg/MetadataPaths.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPaths {

Review comment:
       Done




-- 
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] rdblue commented on pull request #2658: Core: Support relative paths in metadata

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


   @flyrain, thanks for pointing out this PR. Looks like the design doc was updated, so I'll make some time to take another look at that.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] anuragmantri commented on pull request #2658: Core: Support relative paths in metadata

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


   CC: @dpaani 
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/ManifestLists.java
##########
@@ -43,22 +47,54 @@ private ManifestLists() {
         .reuseContainers(false)
         .build()) {
 
-      return Lists.newLinkedList(files);
+      return shouldUseRelativePaths ? updateManifestFilePathsToAbsolutePaths(files,

Review comment:
       I believe this was commented on the issue as well, but shouldn't we always call updateManifestFilePathstoAbsolute here? Instead of passing through a boolean we could just see if we need to update any files and return the updated set? So a use with only absolute paths would essentially just pass through the list and not change anything?
   
   This would remove our requirement to pass through "shouldUseRelativePaths"




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] pvary commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/MetadataPathUtils.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH;
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH_DEFAULT;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPathUtils {
+
+  private MetadataPathUtils() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param shouldUseRelativePaths whether relative paths should be used
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return shouldUseRelativePaths && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;
+  }
+
+  /**
+   * Convert a given absolute path to relative path with respect to base table location
+   * @param path the absolute path
+   * @param tableLocation the base table location
+   * @return relative path with respect to the base table location
+   */
+  public static String toRelativePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to relative path by removing the table location
+    return shouldUseRelativePaths && path.startsWith(tableLocation) ?
+        path.substring(tableLocation.length() + 1) : path;
+  }
+
+  /**
+   * Return true if relative paths is enabled on this table. Return false otherwise
+   * @param properties table properties
+   * @return true if "write.metadata.use.relative-path" property is true, false otherwise
+   */
+  public static boolean shouldUseRelativePath(Map<String, String> properties) {

Review comment:
       Maybe `Table` as a parameter as mentioned in the comments?




-- 
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 #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/MetadataPathUtils.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH;
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH_DEFAULT;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPathUtils {
+
+  private MetadataPathUtils() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param shouldUseRelativePaths whether relative paths should be used
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return shouldUseRelativePaths && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;

Review comment:
       do we need shouldUseRelativePaths here? Can't we just use the return !path.startsWith(tableLocation) ? Paths.join(tableLocation, path) : 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] anuragmantri commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/MetadataPaths.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPaths {
+
+  private MetadataPaths() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param properties table properties
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, Map<String, String> properties) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return useRelativePath(properties) && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;

Review comment:
       IIUC, the `LocationProvider` class is for data files and is used to provide locations to write the files. In this PR, we do not wish to change where data files and/or manifest files are written. We just want to change the references to any paths inside `metadata` files to relative paths. This way, if a table is relocated/moved, we don't have to change a bunch of metadata files. 




-- 
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] anuragmantri commented on pull request #2658: Core: Support relative paths in metadata

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


   Thanks for the review @pvary :)
   
   > Would it be possible to hide the whole stuff inside a LocationProvider? It could be a base implementation for all of the LocationProviders, and would be a natural place in my opinion. We can avoid changing the public APIs if we provide a default implementation.
   
   - In this change, we only want to replace the references of paths in the content of `metadata` files. We do not wish to change where data/metadata is actually written. My understanding of `LocationProviders` is that they are used to get locations to write the data files and hence for this change, we don't want to change any LocationProvider implementation. 
   
   > If there is any way to split this PR to smaller, more reviewer friendly ones that would be a big help for us, reviewers.
   
   - I tried to keep this change small initially. Specifically, in this PR, I tried to just introduce the new property and modify the read and write paths to use table property to do the necessary changes. Property is turned off my default. Yet, this turned out to be huge :) There will be several follow-up PRs such has handling metadata tables and writing unit tests. I will be happy to split this further if you think we can logically do so. 


-- 
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] pvary commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/MetadataPaths.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPaths {
+
+  private MetadataPaths() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param properties table properties
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, Map<String, String> properties) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return useRelativePath(properties) && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;

Review comment:
       Shouldn't we add this method to the LocationProviders?
   For me extending the LocationProvider with additional default implementation which handles relative patch would be more natural.




-- 
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 pull request #2658: Core: Support relative paths in metadata

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


   Hi @aokolnychyi/@rdblue, can you take a look at this PR? Data replication is a pretty common request for any serious Iceberg user. Let's see how can we move forward with https://github.com/apache/iceberg/issues/1617. 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] pvary commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/DataTableScan.java
##########
@@ -71,7 +71,8 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche
   public CloseableIterable<FileScanTask> planFiles(TableOperations ops, Snapshot snapshot,
                                                    Expression rowFilter, boolean ignoreResiduals,
                                                    boolean caseSensitive, boolean colStats) {
-    ManifestGroup manifestGroup = new ManifestGroup(ops.io(), snapshot.dataManifests(), snapshot.deleteManifests())
+    ManifestGroup manifestGroup = new ManifestGroup(ops.io(), snapshot.dataManifests(), snapshot.deleteManifests(),
+        ops.current().location(), ops.current().properties())

Review comment:
       Why do we push the whole `properties`? Shall we just push a flag `useRelative`, or we can just push the `location` and if it is not null then we use it as a base for the relative paths?




-- 
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] pvary commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/BaseSnapshot.java
##########
@@ -180,14 +210,20 @@ public String manifestListLocation() {
     return manifestListLocation;
   }
 
+  @Override
+  public FileIO io() {

Review comment:
       I leave a comment here, because I would rather not leak `io` here if possible




-- 
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 edited a comment on pull request #2658: Core: Support relative paths in metadata

Posted by GitBox <gi...@apache.org>.
flyrain edited a comment on pull request #2658:
URL: https://github.com/apache/iceberg/pull/2658#issuecomment-866194097


   Hi @aokolnychyi, @rdblue, can you take a look at this PR? Data replication is a pretty common request for any serious Iceberg user. Let's see how can we move forward with https://github.com/apache/iceberg/issues/1617. 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] RussellSpitzer commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java
##########
@@ -158,7 +158,8 @@ private ManifestFile copyManifest(ManifestFile manifest) {
     InputFile toCopy = ops.io().newInputFile(manifest.path());
     OutputFile newFile = newManifestOutput();
     return ManifestFiles.copyRewriteManifest(
-        current.formatVersion(), toCopy, specsById, newFile, snapshotId(), summaryBuilder);
+        current.formatVersion(), toCopy, specsById, newFile, snapshotId(), summaryBuilder,
+            current.location(), current.shouldUseRelativePaths());

Review comment:
       shouldn't I shouldUseRelativePaths from the table properties?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] anuragmantri commented on pull request #2658: Core: Support relative paths in metadata

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


   All tests in `spark3/src/test/java/org/apache/iceberg/actions/TestRemoveOrphanFilesAction3.java` are passing for me locally. However, there is a test failure (`testHiveCatalogTable.java`) in CI. Has this happed to anyone before?


-- 
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] pvary commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/MetadataPathUtils.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH;
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH_DEFAULT;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPathUtils {
+
+  private MetadataPathUtils() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param shouldUseRelativePaths whether relative paths should be used
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return shouldUseRelativePaths && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;
+  }
+
+  /**
+   * Convert a given absolute path to relative path with respect to base table location
+   * @param path the absolute path
+   * @param tableLocation the base table location
+   * @return relative path with respect to the base table location
+   */
+  public static String toRelativePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to relative path by removing the table location
+    return shouldUseRelativePaths && path.startsWith(tableLocation) ?

Review comment:
       If we should use relative paths, it might be an error if the path is not starts with location, and it is not relative...
   Do we want that check 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] RussellSpitzer commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/ManifestLists.java
##########
@@ -43,22 +47,54 @@ private ManifestLists() {
         .reuseContainers(false)
         .build()) {
 
-      return Lists.newLinkedList(files);
+      return shouldUseRelativePaths ? updateManifestFilePathsToAbsolutePaths(files,

Review comment:
       I believe this was commented on the issue as well, but shouldn't we always call updateManifestFilePathstoAbsolute here? Instead of passing through a boolean we could just see if we need to update any files and return the updated set? So a use with only absolute paths would essentially just pass through the list and not change anything?
   
   This would remove our requirement to pass through "shouldUseRelativePaths"

##########
File path: core/src/main/java/org/apache/iceberg/MetadataPathUtils.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH;
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH_DEFAULT;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPathUtils {
+
+  private MetadataPathUtils() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param shouldUseRelativePaths whether relative paths should be used
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return shouldUseRelativePaths && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;

Review comment:
       Paths.get?

##########
File path: core/src/main/java/org/apache/iceberg/MetadataPathUtils.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH;
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH_DEFAULT;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPathUtils {
+
+  private MetadataPathUtils() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param shouldUseRelativePaths whether relative paths should be used
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return shouldUseRelativePaths && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;

Review comment:
       do we need shouldUseRelativePaths here? Can't we just use the return !path.startsWith(tableLocation) ? Paths.join(tableLocation, path) : path?

##########
File path: core/src/main/java/org/apache/iceberg/MetadataPathUtils.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH;
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH_DEFAULT;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPathUtils {
+
+  private MetadataPathUtils() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param shouldUseRelativePaths whether relative paths should be used
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return shouldUseRelativePaths && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;
+  }
+
+  /**
+   * Convert a given absolute path to relative path with respect to base table location
+   * @param path the absolute path
+   * @param tableLocation the base table location
+   * @return relative path with respect to the base table location
+   */
+  public static String toRelativePath(String path, String tableLocation, boolean shouldUseRelativePaths) {

Review comment:
       do we need shouldUseRelativePaths here? Shouldn't the function toRelativePath always return a relative path? The caller would know whether it needs to call toRelativePath or toAbsolutePath?

##########
File path: core/src/main/java/org/apache/iceberg/MetadataPathUtils.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH;
+import static org.apache.iceberg.TableProperties.WRITE_METADATA_USE_RELATIVE_PATH_DEFAULT;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPathUtils {
+
+  private MetadataPathUtils() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param shouldUseRelativePaths whether relative paths should be used
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return shouldUseRelativePaths && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;
+  }
+
+  /**
+   * Convert a given absolute path to relative path with respect to base table location
+   * @param path the absolute path
+   * @param tableLocation the base table location
+   * @return relative path with respect to the base table location
+   */
+  public static String toRelativePath(String path, String tableLocation, boolean shouldUseRelativePaths) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to relative path by removing the table location
+    return shouldUseRelativePaths && path.startsWith(tableLocation) ?

Review comment:
       I think for both of these functions we are basically care more about the output than the input
   
   toAbsolute
   (Absolute Path | Relative to table Location) -> Absolute Path
   
   toRelative
   (Absolute Path | Relative to table location) -> relativizedPath to table location | absolutePath not including table location
   
   As @pvary notes it's possible for AbsolutePath to be in a completely different file system, in that case this method sometimes returns relative paths, but sometimes returns absolute paths.
   
   I think this is actually the right thing to do for this method since we can only ever deal with paths relative to the table location ... but maybe this needs another name though? Since currently it doesn't always return relative paths

##########
File path: api/src/main/java/org/apache/iceberg/Snapshot.java
##########
@@ -126,4 +127,10 @@
    * @return the location of the manifest list for this Snapshot
    */
   String manifestListLocation();
+
+  /**
+   *
+   * @return file io used for this snapshot
+   */
+  FileIO io();

Review comment:
       Why do we need this?

##########
File path: core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java
##########
@@ -158,7 +158,8 @@ private ManifestFile copyManifest(ManifestFile manifest) {
     InputFile toCopy = ops.io().newInputFile(manifest.path());
     OutputFile newFile = newManifestOutput();
     return ManifestFiles.copyRewriteManifest(
-        current.formatVersion(), toCopy, specsById, newFile, snapshotId(), summaryBuilder);
+        current.formatVersion(), toCopy, specsById, newFile, snapshotId(), summaryBuilder,
+            current.location(), current.shouldUseRelativePaths());

Review comment:
       shouldn't I shouldUseRelativePaths from the table properties?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] pvary commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java
##########
@@ -46,18 +49,28 @@ static InheritableMetadata forCopy(long snapshotId) {
     private final int specId;
     private final long snapshotId;
     private final long sequenceNumber;
+    private final String tableLocation;
+    private final Map<String, String> tableProperties;
 
-    private BaseInheritableMetadata(int specId, long snapshotId, long sequenceNumber) {
+    private BaseInheritableMetadata(int specId, long snapshotId, long sequenceNumber, String tableLocation,
+        Map<String, String> tableProperties) {
       this.specId = specId;
       this.snapshotId = snapshotId;
       this.sequenceNumber = sequenceNumber;
+      this.tableLocation = tableLocation;
+      this.tableProperties = tableProperties;
     }
 
     @Override
     public <F extends ContentFile<F>> ManifestEntry<F> apply(ManifestEntry<F> manifestEntry) {
       if (manifestEntry.file() instanceof BaseFile) {
         BaseFile<?> file = (BaseFile<?>) manifestEntry.file();
         file.setSpecId(specId);
+        if (MetadataPaths.useRelativePath(tableProperties)) {
+          if (!file.path().toString().startsWith(tableLocation)) {

Review comment:
       Why do not we push this logic inside the `otAbsolutePath` method?
   
   This is only a question which we might to consider, I myself not sure about the answer yet.




-- 
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] pvary commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/MetadataPaths.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPaths {

Review comment:
       Maybe name it as `MetadataPathUtils`?




-- 
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] anuragmantri commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/DataTableScan.java
##########
@@ -71,7 +71,8 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche
   public CloseableIterable<FileScanTask> planFiles(TableOperations ops, Snapshot snapshot,
                                                    Expression rowFilter, boolean ignoreResiduals,
                                                    boolean caseSensitive, boolean colStats) {
-    ManifestGroup manifestGroup = new ManifestGroup(ops.io(), snapshot.dataManifests(), snapshot.deleteManifests())
+    ManifestGroup manifestGroup = new ManifestGroup(ops.io(), snapshot.dataManifests(), snapshot.deleteManifests(),
+        ops.current().location(), ops.current().properties())

Review comment:
       Instead of  pushing `properties` around, I introduced a flag `shouldUseRelativePaths`.

##########
File path: core/src/main/java/org/apache/iceberg/ManifestEntriesTable.java
##########
@@ -126,10 +126,10 @@ public long targetSplitSize() {
     @Override
     public CloseableIterable<StructLike> rows() {
       if (manifest.content() == ManifestContent.DATA) {
-        return CloseableIterable.transform(ManifestFiles.read(manifest, io).project(fileSchema).entries(),
+        return CloseableIterable.transform(ManifestFiles.read(manifest, io, null, null).project(fileSchema).entries(),

Review comment:
       Done

##########
File path: core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java
##########
@@ -46,18 +49,28 @@ static InheritableMetadata forCopy(long snapshotId) {
     private final int specId;
     private final long snapshotId;
     private final long sequenceNumber;
+    private final String tableLocation;
+    private final Map<String, String> tableProperties;
 
-    private BaseInheritableMetadata(int specId, long snapshotId, long sequenceNumber) {
+    private BaseInheritableMetadata(int specId, long snapshotId, long sequenceNumber, String tableLocation,
+        Map<String, String> tableProperties) {
       this.specId = specId;
       this.snapshotId = snapshotId;
       this.sequenceNumber = sequenceNumber;
+      this.tableLocation = tableLocation;
+      this.tableProperties = tableProperties;
     }
 
     @Override
     public <F extends ContentFile<F>> ManifestEntry<F> apply(ManifestEntry<F> manifestEntry) {
       if (manifestEntry.file() instanceof BaseFile) {
         BaseFile<?> file = (BaseFile<?>) manifestEntry.file();
         file.setSpecId(specId);
+        if (MetadataPaths.useRelativePath(tableProperties)) {
+          if (!file.path().toString().startsWith(tableLocation)) {

Review comment:
       You are right. Pushed the logic into `toAbsolutePath()`.

##########
File path: core/src/main/java/org/apache/iceberg/MetadataPaths.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPaths {

Review comment:
       Done

##########
File path: spark2/src/test/java/org/apache/iceberg/examples/RelativePathHiveFunctionalityTest.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.examples;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hive.HiveMetastoreTest;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.SparkConf;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public class RelativePathHiveFunctionalityTest extends HiveMetastoreTest {

Review comment:
       You are right. Renamed the class.

##########
File path: core/src/main/java/org/apache/iceberg/MetadataPaths.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Utility class that contains path conversion methods.
+ */
+public final class MetadataPaths {
+
+  private MetadataPaths() {
+  }
+
+  /**
+   * Convert a given relative path to absolute path for the table by appending the base table location
+   * @param path relative path to be converted
+   * @param tableLocation base table location
+   * @param properties table properties
+   * @return the absolute path
+   */
+  public static String toAbsolutePath(String path, String tableLocation, Map<String, String> properties) {
+    Preconditions.checkArgument(path != null && path.trim().length() > 0);
+    // TODO: Fix this after tests are changed to always pass the table location. Table location cannot be null.
+    if (tableLocation == null) {
+      return path;
+    }
+    // convert to absolute path by appending the table location
+    return useRelativePath(properties) && !path.startsWith(tableLocation) ? tableLocation + "/" + path : path;

Review comment:
       IIUC, the `LocationProvider` class is for data files and is used to provide locations to write the files. In this PR, we do not wish to change where data files and/or manifest files are written. We just want to change the references to any paths inside `metadata` files to relative paths. This way, if a table is relocated/moved, we don't have to change a bunch of metadata files. 




-- 
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] pvary commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/DataFilesTable.java
##########
@@ -122,7 +122,7 @@ public long targetSplitSize() {
     @Override
     public CloseableIterable<StructLike> rows() {
       return CloseableIterable.transform(
-          ManifestFiles.read(manifest, io).project(schema),
+          ManifestFiles.read(manifest, io, null, null).project(schema),

Review comment:
       Could we keep the original signatures as well, so we do not have to rewrite the method calls everywhere and add `null, null`? I think this could greatly reduce the changes needed in this patch




-- 
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] anuragmantri commented on pull request #2658: Core: Support relative paths in metadata

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


   Thanks for the review @pvary :)
   
   > Would it be possible to hide the whole stuff inside a LocationProvider? It could be a base implementation for all of the LocationProviders, and would be a natural place in my opinion. We can avoid changing the public APIs if we provide a default implementation.
   
   - In this change, we only want to replace the references of paths in the content of `metadata` files. We do not wish to change where data/metadata is actually written. My understanding of `LocationProviders` is that they are used to get locations to write the data files and hence for this change, we don't want to change any LocationProvider implementation. 
   
   > If there is any way to split this PR to smaller, more reviewer friendly ones that would be a big help for us, reviewers.
   
   - I tried to keep this change small initially. Specifically, in this PR, I tried to just introduce the new property and modify the read and write paths to use table property to do the necessary changes. Property is turned off my default. Yet, this turned out to be huge :) There will be several follow-up PRs such has handling metadata tables and writing unit tests. I will be happy to split this further if you think we can logically do so. 


-- 
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] anuragmantri commented on a change in pull request #2658: Core: Support relative paths in metadata

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



##########
File path: core/src/main/java/org/apache/iceberg/DataTableScan.java
##########
@@ -71,7 +71,8 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche
   public CloseableIterable<FileScanTask> planFiles(TableOperations ops, Snapshot snapshot,
                                                    Expression rowFilter, boolean ignoreResiduals,
                                                    boolean caseSensitive, boolean colStats) {
-    ManifestGroup manifestGroup = new ManifestGroup(ops.io(), snapshot.dataManifests(), snapshot.deleteManifests())
+    ManifestGroup manifestGroup = new ManifestGroup(ops.io(), snapshot.dataManifests(), snapshot.deleteManifests(),
+        ops.current().location(), ops.current().properties())

Review comment:
       Instead of  pushing `properties` around, I introduced a flag `shouldUseRelativePaths`.




-- 
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] anuragmantri commented on pull request #2658: Core: Support relative paths in metadata

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


   Thank you @RussellSpitzer and @pvary for your comments. I will address the comments after the design is finalized. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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