amogh-jahagirdar commented on code in PR #12111:
URL: https://github.com/apache/iceberg/pull/12111#discussion_r1929803827


##########
spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java:
##########
@@ -0,0 +1,1081 @@
+/*
+ * 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.spark.actions;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.GenericStatisticsFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StaticTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.actions.ActionsProvider;
+import org.apache.iceberg.actions.RewriteTablePath;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkCatalog;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.Pair;
+import org.apache.spark.SparkEnv;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.storage.BlockId;
+import org.apache.spark.storage.BlockInfoManager;
+import org.apache.spark.storage.BlockManager;
+import org.apache.spark.storage.BroadcastBlockId;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import scala.Tuple2;
+
+public class TestRewriteTablePathsAction extends SparkTestBase {
+
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  protected ActionsProvider actions() {
+    return SparkActions.get();
+  }
+
+  private static final HadoopTables TABLES = new HadoopTables(new 
Configuration());
+  protected static final Schema SCHEMA =
+      new Schema(
+          optional(1, "c1", Types.IntegerType.get()),
+          optional(2, "c2", Types.StringType.get()),
+          optional(3, "c3", Types.StringType.get()));
+
+  protected String tableLocation = null;
+  public String staging = null;
+  public String tableDir = null;
+  public String newTableDir = null;
+  public String targetTableDir = null;
+  private Table table = null;
+
+  private final String ns = "testns";
+  private final String backupNs = "backupns";
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableLocation = temp.newFolder().toURI().toString();
+    this.staging = temp.newFolder("staging").toURI().toString();
+    this.tableDir = temp.newFolder("table").toURI().toString();
+    this.newTableDir = temp.newFolder("newTable").toURI().toString();
+    this.targetTableDir = temp.newFolder("targetTable").toURI().toString();
+    this.table = createATableWith2Snapshots(tableLocation);
+    createNameSpaces();
+  }
+
+  @After
+  public void cleanupTableSetup() throws Exception {
+    dropNameSpaces();
+  }
+
+  private Table createATableWith2Snapshots(String location) {
+    return createTableWithSnapshots(location, 2);
+  }
+
+  private Table createTableWithSnapshots(String location, int snapshotNumber) {
+    return createTableWithSnapshots(location, snapshotNumber, 
Maps.newHashMap());
+  }
+
+  protected Table createTableWithSnapshots(
+      String location, int snapshotNumber, Map<String, String> properties) {
+    Table newTable = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), 
properties, location);
+
+    List<ThreeColumnRecord> records =
+        Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA"));
+
+    Dataset<Row> df = spark.createDataFrame(records, 
ThreeColumnRecord.class).coalesce(1);
+
+    for (int i = 0; i < snapshotNumber; i++) {
+      df.select("c1", "c2", 
"c3").write().format("iceberg").mode("append").save(location);
+    }
+
+    return newTable;
+  }
+
+  private void createNameSpaces() {
+    sql("CREATE DATABASE IF NOT EXISTS %s", ns);
+    sql("CREATE DATABASE IF NOT EXISTS %s", backupNs);
+  }
+
+  private void dropNameSpaces() {
+    sql("DROP DATABASE IF EXISTS %s CASCADE", ns);
+    sql("DROP DATABASE IF EXISTS %s CASCADE", backupNs);
+  }
+
+  @Test
+  public void testRewritePath() throws Exception {
+    String targetTableLocation = targetTableLocation();
+
+    // check the data file location before the rebuild
+    List<String> validDataFiles =
+        spark
+            .read()
+            .format("iceberg")
+            .load(tableLocation + "#files")
+            .select("file_path")
+            .as(Encoders.STRING())
+            .collectAsList();
+    assertThat(validDataFiles.size()).isEqualTo(2);
+
+    RewriteTablePath.Result result =
+        actions()
+            .rewriteTablePath(table)
+            .rewriteLocationPrefix(tableLocation, targetTableLocation)
+            .endVersion("v3.metadata.json")
+            .execute();
+
+    assertThat(result.latestVersion()).isEqualTo("v3.metadata.json");
+
+    checkFileNum(3, 2, 2, 9, result);
+
+    // copy the metadata files and data files
+    copyTableFiles(result);
+
+    // verify the data file path after the rebuild
+    List<String> validDataFilesAfterRebuilt =
+        spark
+            .read()
+            .format("iceberg")
+            .load(targetTableLocation + "#files")
+            .select("file_path")
+            .as(Encoders.STRING())
+            .collectAsList();
+    assertThat(validDataFilesAfterRebuilt)
+        .hasSize(2)
+        .allMatch(item -> item.startsWith(targetTableLocation));
+
+    // verify data rows
+    List<Object[]> actual = rows(targetTableLocation);
+    List<Object[]> expected = rows(tableLocation);
+    assertEquals("Rows should match after copy", expected, actual);
+  }
+
+  @Test
+  public void testSameLocations() {
+    assertThatThrownBy(
+            () ->
+                actions()
+                    .rewriteTablePath(table)
+                    .rewriteLocationPrefix(tableLocation, tableLocation)
+                    .endVersion("v1.metadata.json")
+                    .execute())
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessageContaining("Source prefix cannot be the same as target 
prefix");
+  }
+
+  @Test
+  public void testStartVersion() throws Exception {
+    RewriteTablePath.Result result =
+        actions()
+            .rewriteTablePath(table)
+            .rewriteLocationPrefix(tableLocation, targetTableLocation())
+            .startVersion("v2.metadata.json")
+            .execute();
+
+    checkFileNum(1, 1, 1, 4, result);
+
+    List<Tuple2<String, String>> paths = 
readPathPairList(result.fileListLocation());
+
+    String currentSnapshotId = 
String.valueOf(table.currentSnapshot().snapshotId());
+    assertThat(paths.stream().filter(c -> 
c._2().contains(currentSnapshotId)).count())
+        .withFailMessage("Should have the current snapshot file")
+        .isEqualTo(1);
+
+    String parentSnapshotId = 
String.valueOf(table.currentSnapshot().parentId());
+    assertThat(paths.stream().filter(c -> 
c._2().contains(parentSnapshotId)).count())
+        .withFailMessage("Should NOT have the parent snapshot file")
+        .isEqualTo(0);
+  }
+
+  @Test
+  //  public void testTableWith3Snapshots(@TempDir Path location1, @TempDir 
Path location2)

Review Comment:
   I think this was probably accidentally commented out as part of a 
copy/paste. Could we remove 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-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

Reply via email to