singhpk234 commented on code in PR #5056:
URL: https://github.com/apache/iceberg/pull/5056#discussion_r900040404


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkCatalogTestBase;
+import org.apache.iceberg.spark.source.SimpleRecord;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+
+public class TestDeleteFrom extends SparkCatalogTestBase {
+  public TestDeleteFrom(String catalogName, String implementation, Map<String, 
String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  // revisit
+  @Ignore
+  @Test
+  public void testDeleteFromUnpartitionedTable() throws NoSuchTableException {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    List<SimpleRecord> records = Lists.newArrayList(
+        new SimpleRecord(1, "a"),
+        new SimpleRecord(2, "b"),
+        new SimpleRecord(3, "c")
+    );
+    Dataset<Row> df = spark.createDataFrame(records, SimpleRecord.class);
+    df.coalesce(1).writeTo(tableName).append();
+
+    assertEquals("Should have expected rows",
+        ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c")),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+
+    AssertHelpers.assertThrows("Should not delete when not all rows of a file 
match the filter",
+        AnalysisException.class, "Cannot delete from",
+        () -> sql("DELETE FROM %s WHERE id < 2", tableName));
+
+    sql("DELETE FROM %s WHERE id < 4", tableName);
+
+    assertEquals("Should have no rows after successful delete",
+        ImmutableList.of(),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void testDeleteFromTableAtSnapshot() throws NoSuchTableException {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    List<SimpleRecord> records = Lists.newArrayList(
+        new SimpleRecord(1, "a"),
+        new SimpleRecord(2, "b"),
+        new SimpleRecord(3, "c")
+    );
+    Dataset<Row> df = spark.createDataFrame(records, SimpleRecord.class);
+    df.coalesce(1).writeTo(tableName).append();
+
+    long snapshotId = 
validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId();
+    String prefix = "snapshot_id_";
+    AssertHelpers.assertThrows("Should not be able to delete from a table at a 
specific snapshot",
+        IllegalArgumentException.class, "Cannot delete from table at a 
specific snapshot",
+        () -> sql("DELETE FROM %s.%s WHERE id < 4", tableName, prefix + 
snapshotId));
+  }
+
+  // revisit
+  @Ignore
+  @Test

Review Comment:
   These test's were failing stating  : 
   ```
   Field _file not found in source schema
   java.lang.IllegalArgumentException: Field _file not found in source schema
        at org.apache.iceberg.types.ReassignIds.field(ReassignIds.java:104)
        at org.apache.iceberg.types.ReassignIds.field(ReassignIds.java:28)
          ....
                at 
org.apache.iceberg.spark.SparkSchemaUtil.convert(SparkSchemaUtil.java:192)
        at 
org.apache.iceberg.spark.source.SparkWriteBuilder.validateOrMergeWriteSchema(SparkWriteBuilder.java:247)
   ```
   
   `_file` not found this was happening because in the upstream we were relying 
on `_ _metadata_col` flag in metadata attribute to filter these metadata 
columns out : 
   
https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala#L221-L225
   
   But when we were doing spark schema conversion we were not adding this in 
attribute metadata, and hence it was not getting filtered, when it was trying 
validating with source schema it was failing. 
   
   Added a fix for same, (added for 
[3.2](https://github.com/apache/iceberg/pull/5075) as well) and adapted the UT 
to respect the upstream changes.



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


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

Reply via email to