simhadri-g commented on code in PR #4761:
URL: https://github.com/apache/hive/pull/4761#discussion_r1365054556


##########
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestConflictingDataFiles.java:
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.mr.hive;
+
+import java.util.List;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.mr.TestHelper;
+import org.apache.iceberg.relocated.com.google.common.base.Throwables;
+import org.apache.iceberg.util.Tasks;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static 
org.apache.iceberg.mr.hive.HiveIcebergStorageHandlerTestUtils.init;
+
+public class TestConflictingDataFiles extends 
HiveIcebergStorageHandlerWithEngineBase {
+
+  @Test
+  public void testSingleFilterUpdate() {
+
+    PartitionSpec spec =
+        
PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA).identity("last_name")
+            .bucket("customer_id", 16).build();
+
+    // create and insert an initial batch of records
+    testTables.createTable(shell, "customers", 
HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, spec, fileFormat,
+        HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2);
+    // insert one more batch so that we have multiple data files within the 
same partition
+    
shell.executeStatement(testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1,
+        TableIdentifier.of("default", "customers"), false));
+
+    String[] singleFilterQuery = new String[] { "UPDATE customers SET 
first_name='Changed' WHERE  last_name='Taylor'",
+        "UPDATE customers SET first_name='Changed' WHERE  last_name='Donnel'" 
};
+
+    try {
+      Tasks.range(2).executeWith(Executors.newFixedThreadPool(2)).run(i -> {
+        init(shell, testTables, temp, executionEngine);
+        HiveConf.setBoolVar(shell.getHiveConf(), 
HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized);
+        HiveConf.setVar(shell.getHiveConf(), 
HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none");
+        HiveConf.setVar(shell.getHiveConf(), 
HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES,
+            RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT);
+        shell.executeStatement(singleFilterQuery[i]);
+        shell.closeSession();
+      });
+    } catch (Throwable ex) {
+      // If retry succeeds then it should not throw an ValidationException.
+      Throwable cause = Throwables.getRootCause(ex);
+      if (cause instanceof ValidationException && 
cause.getMessage().matches("^Found.*conflicting.*files(.*)")) {
+        Assert.fail();
+      }
+    }
+
+    List<Object[]> objects =
+        shell.executeStatement("SELECT * FROM customers ORDER BY customer_id, 
last_name, first_name");
+    Assert.assertEquals(12, objects.size());
+    List<Record> expected = 
TestHelper.RecordsBuilder.newInstance(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA)
+        .add(1L, "Joanna", "Pierce")
+        .add(1L, "Changed", "Taylor")
+        .add(2L, "Changed", "Donnel")
+        .add(2L, "Susan", "Morrison")
+        .add(2L, "Bob", "Silver")
+        .add(2L, "Joanna", "Silver")
+        .add(3L, "Marci", "Barna")
+        .add(3L, "Blake", "Burr")
+        .add(3L, "Trudy", "Henderson")
+        .add(3L, "Trudy", "Johnson")
+        .add(4L, "Laci", "Zold")
+        .add(5L, "Peti", "Rozsaszin").build();
+    HiveIcebergTestUtils.validateData(expected,
+        
HiveIcebergTestUtils.valueForRow(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
 objects), 0);
+
+  }
+
+  @Test
+  public void testMultiFiltersUpdate() {
+
+    PartitionSpec spec =
+        
PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA).identity("last_name")
+            .bucket("customer_id", 16).build();
+
+    // create and insert an initial batch of records
+    testTables.createTable(shell, "customers", 
HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, spec, fileFormat,
+        HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2);
+    // insert one more batch so that we have multiple data files within the 
same partition
+    
shell.executeStatement(testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1,
+        TableIdentifier.of("default", "customers"), false));
+
+    String[] multiFilterQuery = new String[] {
+        "UPDATE customers SET first_name='Changed' WHERE  
last_name='Henderson' " + "OR last_name='Johnson'",
+        "UPDATE customers SET first_name='Changed' WHERE  last_name='Taylor' 
AND customer_id=1" };
+
+    try {
+      Tasks.range(2).executeWith(Executors.newFixedThreadPool(2)).run(i -> {
+        init(shell, testTables, temp, executionEngine);
+        HiveConf.setBoolVar(shell.getHiveConf(), 
HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized);
+        HiveConf.setVar(shell.getHiveConf(), 
HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none");
+        HiveConf.setVar(shell.getHiveConf(), 
HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES,
+            RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT);
+        shell.executeStatement(multiFilterQuery[i]);
+        shell.closeSession();
+      });
+    } catch (Throwable ex) {
+      // If retry succeeds then it should not throw an ValidationException.
+      Throwable cause = Throwables.getRootCause(ex);
+      if (cause instanceof ValidationException && 
cause.getMessage().matches("^Found.*conflicting.*files(.*)")) {
+        Assert.fail();
+      }
+    }
+
+
+    List<Object[]> objects =
+        shell.executeStatement("SELECT * FROM customers ORDER BY customer_id, 
last_name, first_name");
+    Assert.assertEquals(12, objects.size());
+    List<Record> expected = 
TestHelper.RecordsBuilder.newInstance(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA)
+        .add(1L, "Joanna", "Pierce")
+        .add(1L, "Changed", "Taylor")
+        .add(2L, "Jake", "Donnel")
+        .add(2L, "Susan", "Morrison")
+        .add(2L, "Bob", "Silver")
+        .add(2L, "Joanna", "Silver")
+        .add(3L, "Marci", "Barna")
+        .add(3L, "Blake", "Burr")
+        .add(3L, "Changed", "Henderson")
+        .add(3L, "Changed", "Johnson")
+        .add(4L, "Laci", "Zold")
+        .add(5L, "Peti", "Rozsaszin").build();
+    HiveIcebergTestUtils.validateData(expected,
+        
HiveIcebergTestUtils.valueForRow(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
 objects), 0);
+  }
+
+  @Test
+  public void testDeleteFilters() {
+
+    PartitionSpec spec =
+        
PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA).identity("last_name")
+            .bucket("customer_id", 16).build();
+
+    // create and insert an initial batch of records
+    testTables.createTable(shell, "customers", 
HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, spec, fileFormat,
+        HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2);
+    // insert one more batch so that we have multiple data files within the 
same partition
+    
shell.executeStatement(testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1,
+        TableIdentifier.of("default", "customers"), false));
+
+    String[] sql = new String[] { "DELETE FROM customers WHERE  
last_name='Taylor'",
+        "DELETE FROM customers WHERE last_name='Donnel'",
+        "DELETE FROM customers WHERE last_name='Henderson' OR 
last_name='Johnson'" };
+
+    try {
+      Tasks.range(3).executeWith(Executors.newFixedThreadPool(2)).run(i -> {
+        init(shell, testTables, temp, executionEngine);
+        HiveConf.setBoolVar(shell.getHiveConf(), 
HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized);
+        HiveConf.setVar(shell.getHiveConf(), 
HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none");
+        HiveConf.setVar(shell.getHiveConf(), 
HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES,
+            RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT);
+        shell.executeStatement(sql[i]);
+        shell.closeSession();
+      });
+    } catch (Throwable ex) {
+      // If retry succeeds then it should not throw an ValidationException.
+      Throwable cause = Throwables.getRootCause(ex);
+      if (cause instanceof ValidationException && 
cause.getMessage().matches("^Found.*conflicting.*files(.*)")) {
+        Assert.fail();
+      }
+    }
+
+    List<Object[]> objects =
+        shell.executeStatement("SELECT * FROM customers ORDER BY customer_id, 
last_name, first_name");
+    Assert.assertEquals(8, objects.size());
+    List<Record> expected = 
TestHelper.RecordsBuilder.newInstance(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA)
+        .add(1L, "Joanna", "Pierce").add(2L, "Susan", "Morrison").add(2L, 
"Bob", "Silver").add(2L, "Joanna", "Silver")
+        .add(3L, "Marci", "Barna").add(3L, "Blake", "Burr").add(4L, "Laci", 
"Zold").add(5L, "Peti", "Rozsaszin")
+        .build();
+    HiveIcebergTestUtils.validateData(expected,
+        
HiveIcebergTestUtils.valueForRow(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
 objects), 0);
+
+  }
+
+  @Test
+  public void testConflictingUpdates() {
+    PartitionSpec spec =
+        
PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA).identity("last_name")
+            .bucket("customer_id", 16).build();
+
+    // create and insert an initial batch of records
+    testTables.createTable(shell, "customers", 
HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, spec, fileFormat,
+        HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2);
+    // insert one more batch so that we have multiple data files within the 
same partition
+    
shell.executeStatement(testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1,
+        TableIdentifier.of("default", "customers"), false));
+
+    String[] singleFilterQuery = new String[] { "UPDATE customers SET 
first_name='Changed' WHERE  last_name='Taylor'",
+        "UPDATE customers SET first_name='Changed' WHERE  last_name='Taylor'" 
};
+
+    try {
+      Tasks.range(2).executeWith(Executors.newFixedThreadPool(2)).run(i -> {
+        init(shell, testTables, temp, executionEngine);
+        HiveConf.setBoolVar(shell.getHiveConf(), 
HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized);
+        HiveConf.setVar(shell.getHiveConf(), 
HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none");
+        HiveConf.setVar(shell.getHiveConf(), 
HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES,
+            RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT);
+        shell.executeStatement(singleFilterQuery[i]);
+        shell.closeSession();
+      });
+    } catch (Throwable ex) {
+      // If retry succeeds then it should not throw an ValidationException.
+      Throwable cause = Throwables.getRootCause(ex);
+      Assert.assertTrue(
+          cause instanceof ValidationException && 
cause.getMessage().matches("^Found.*conflicting" + ".*files(.*)"));

Review Comment:
   fixed. In this case with conflicting queries it should throw 
ValidationException.



-- 
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: gitbox-unsubscr...@hive.apache.org

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


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

Reply via email to