You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2020/12/07 15:45:52 UTC

[GitHub] [hive] pvary commented on a change in pull request #1660: HIVE-23410: ACID: Improve the delete and update operations to avoid t…

pvary commented on a change in pull request #1660:
URL: https://github.com/apache/hive/pull/1660#discussion_r537566412



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/QueryPlan.java
##########
@@ -189,6 +191,49 @@ public WriteEntity getAcidAnalyzeTable() {
     return acidSinks;
   }
 
+  public Integer getStatementIdForAcidWriteType(long writeId, String moveTaskId, AcidUtils.Operation acidOperation, Path path) {
+    FileSinkDesc result = null;
+    for (FileSinkDesc acidSink : acidSinks) {
+      if (acidOperation.equals(acidSink.getAcidOperation()) && path.equals(acidSink.getDestPath())
+          && acidSink.getTableWriteId() == writeId
+          && (moveTaskId == null || acidSink.getMoveTaskId() == null || moveTaskId.equals(acidSink.getMoveTaskId()))) {
+        // There is a problem with the union all optimisation. In this case, there will be multiple FileSinkOperators

Review comment:
       Maybe javadoc instead of a comment?

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/Context.java
##########
@@ -105,6 +105,7 @@
 
   private Configuration conf;
   protected int pathid = 10000;
+  private int moveTaskId = 100;

Review comment:
       Could we start it from 0?

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
##########
@@ -563,6 +564,21 @@ else if (filename.startsWith(BUCKET_PREFIX)) {
     return result;
   }
 
+  public static Map<String, Integer> getDeltaToAttemptIdMap(

Review comment:
       Javadoc please

##########
File path: ql/src/test/queries/clientpositive/materialized_view_create_rewrite_4.q
##########
@@ -3,6 +3,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.strict.checks.cartesian.product=false;
 set hive.materializedview.rewriting=true;
+set hive.acid.direct.insert.enabled=false;

Review comment:
       Please file a jira

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
##########
@@ -232,9 +236,25 @@ public void closeWriters(boolean abort) throws HiveException {
       for (int i = 0; i < updaters.length; i++) {
         if (updaters[i] != null) {
           SerDeStats stats = updaters[i].getStats();
-          // Ignore 0 row files except in case of insert overwrite
-          if (isDirectInsert && (stats.getRowCount() > 0 || isInsertOverwrite)) {
-            outPathsCommitted[i] = updaters[i].getUpdatedFilePath();
+          // Ignore 0 row files except in case of insert overwrite or delete or update
+          if (isDirectInsert
+              && (stats.getRowCount() > 0 || isInsertOverwrite || AcidUtils.Operation.DELETE.equals(acidOperation)
+                  || AcidUtils.Operation.UPDATE.equals(acidOperation))) {
+            // In case of delete operation, the deleteFilePath has to be used, not the updatedFilePath
+            // In case of update operation, we need both paths. The updateFilePath will be added
+            // to the outPathsCommitted array and the deleteFilePath will be collected in a separate list.
+            OrcRecordUpdater recordUpdater = (OrcRecordUpdater) updaters[i];
+            outPathsCommitted[i] = recordUpdater.getUpdatedFilePath();

Review comment:
       Could we do something like a switch?

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/QueryPlan.java
##########
@@ -189,6 +191,49 @@ public WriteEntity getAcidAnalyzeTable() {
     return acidSinks;
   }
 
+  public Integer getStatementIdForAcidWriteType(long writeId, String moveTaskId, AcidUtils.Operation acidOperation, Path path) {
+    FileSinkDesc result = null;
+    for (FileSinkDesc acidSink : acidSinks) {
+      if (acidOperation.equals(acidSink.getAcidOperation()) && path.equals(acidSink.getDestPath())
+          && acidSink.getTableWriteId() == writeId
+          && (moveTaskId == null || acidSink.getMoveTaskId() == null || moveTaskId.equals(acidSink.getMoveTaskId()))) {
+        // There is a problem with the union all optimisation. In this case, there will be multiple FileSinkOperators
+        // with the same operation, writeId and moveTaskId. But one of these FSOs doesn't write data and its statementId
+        // is not valid, so if this FSO is selected and its statementId is returned, the file listing will find nothing.
+        // So check the acidSinks and if two of them have the same writeId, path and moveTaskId, then return -1 as statementId.
+        // Like this, the file listing will find all partitions and files correctly.
+        if (result != null) {
+          return -1;
+        }
+        result = acidSink;
+      }
+    }
+    if (result != null) {
+      return result.getStatementId();
+    } else {
+      return -1;
+    }
+  }
+
+  public Set<String> getDynamicPartitionSpecs(long writeId, String moveTaskId, AcidUtils.Operation acidOperation, Path path) {

Review comment:
       Javadoc

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
##########
@@ -1895,7 +1901,20 @@ public static boolean isSkewedStoredAsDirs(FileSinkDesc fsInputDesc) {
       }
 
       if ((srcDir != null) && srcDir.equals(fsopFinalDir)) {
-        return mvTsk;
+        if (isDirectInsert || isMmFsop) {
+          if (moveTaskId != null && fsoMoveTaskId != null && moveTaskId.equals(fsoMoveTaskId)) {
+            // If the ACID direct insert is on, the MoveTasks cannot be identified by the srcDir as
+            // in this case the srcDir is always the root directory of the table.
+            // We need to consider the ACID write type to identify the MoveTasks.
+            return mvTsk;
+          }
+          if ((moveTaskId == null || fsoMoveTaskId == null) && moveTaskWriteType != null

Review comment:
       Double check if this is needed

##########
File path: ql/src/test/queries/clientpositive/sort_acid.q
##########
@@ -16,7 +16,7 @@ explain cbo
 update acidtlb set b=777;
 update acidtlb set b=777;
 
-select * from acidtlb;
+select * from acidtlb order by a;

Review comment:
       Can we sort the result file instead?

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
##########
@@ -251,7 +271,7 @@ public void closeWriters(boolean abort) throws HiveException {
       }
     }
 
-    private void commit(FileSystem fs, List<Path> commitPaths) throws HiveException {
+    private void commit(FileSystem fs, List<Path> commitPaths, List<Path> deleteDeltas) throws HiveException {

Review comment:
       Think through 1 more time. I can accept that this is the best solution, but this is UGLY 😄 

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -2694,7 +2699,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
    */
   private Set<Path> getValidPartitionsInPath(
       int numDP, int numLB, Path loadPath, Long writeId, int stmtId,
-      boolean isMmTable, boolean isInsertOverwrite, boolean isDirectInsert) throws HiveException {
+      boolean isMmTable, boolean isInsertOverwrite, boolean isDirectInsert, AcidUtils.Operation operation, Set<String> dynamiPartitionSpecs) throws HiveException {

Review comment:
       Too long line

##########
File path: ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidInputFormat.java
##########
@@ -52,6 +52,8 @@
   @Mock
   private DataInput mockDataInput;
 
+  // IRJUNK IDE TESZTET!!!

Review comment:
       Remove




----------------------------------------------------------------
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: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org