You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by "kasakrisz (via GitHub)" <gi...@apache.org> on 2023/03/28 15:21:46 UTC

[GitHub] [hive] kasakrisz opened a new pull request, #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

kasakrisz opened a new pull request, #4166:
URL: https://github.com/apache/hive/pull/4166

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://cwiki.apache.org/confluence/display/Hive/HowToContribute
     2. Ensure that you have created an issue on the Hive project JIRA: https://issues.apache.org/jira/projects/HIVE/summary
     3. Ensure you have added or run the appropriate tests for your PR: 
     4. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]HIVE-XXXXX:  Your PR title ...'.
     5. Be sure to keep the PR description updated to reflect all changes.
     6. Please write your PR title to summarize what this PR proposes.
     7. If possible, provide a concise example to reproduce the issue for a faster review.
   
   -->
   
   ### What changes were proposed in this pull request?
   * Enable Incremental rebuild plan transformation of materialized view stored by iceberg and definition contains aggregate.
   * query the acid sort columns from storage handler in case of non native tables when transforming the AST plan of the MV rebuild
   
   ### Why are the changes needed?
   Improve MV rebuild performance
   
   ### Does this PR introduce _any_ user-facing change?
   No.
   
   ### How was this patch tested?
   ```
   mvn test -Dtest.output.overwrite -DskipSparkTests -Dtest=TestMiniLlapLocalCliDriver -Dqfile=materialized_view_create_rewrite_5.q,materialized_view_create_rewrite_6.q -pl itests/qtest -Pitests
   mvn test -Dtest.output.overwrite -Dtest=TestIcebergCliDriver -Dqfile=mv_iceberg_orc7.q -pl itests/qtest-iceberg -Pitests,iceberg -Drat.skip
   ```


-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1154631673


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java:
##########
@@ -465,21 +467,37 @@ private RelNode applyIncrementalRebuild(RelNode basePlan, RelMetadataProvider md
   protected ASTNode fixUpAfterCbo(ASTNode originalAst, ASTNode newAst, CalcitePlanner.PreCboCtx cboCtx)
           throws SemanticException {
     ASTNode fixedAST = super.fixUpAfterCbo(originalAst, newAst, cboCtx);
+    if (mvRebuildMode == MaterializationRebuildMode.INSERT_OVERWRITE_REBUILD) {
+      return fixedAST;
+    } else if (mvRebuildMode == MaterializationRebuildMode.JOIN_INSERT_REBUILD) {
+      fixUpASTJoinInsertIncrementalRebuild(fixedAST);
+      return fixedAST;
+    }
+
+    MaterializedViewASTBuilder astBuilder;
+    if (AcidUtils.isFullAcidTable(mvTable.getTTable())) {

Review Comment:
   please extract this into the helper method: getMaterializedViewASTBuilder()



-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4166:
URL: https://github.com/apache/hive/pull/4166#issuecomment-1494575502

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4166)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4166&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4166&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] kasakrisz commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "kasakrisz (via GitHub)" <gi...@apache.org>.
kasakrisz commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1155857426


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java:
##########
@@ -564,30 +582,38 @@ private void fixUpASTAggregateIncrementalRebuild(
             HiveParser.TOK_INSERT, HiveParser.TOK_SELECT);
     ASTNode selectExprNodeInputROJ = (ASTNode) ParseDriver.adaptor.create(
             HiveParser.TOK_SELEXPR, "TOK_SELEXPR");
-    ASTNode tableName = createRowIdNode(TableName.getDbTable(
+    astBuilder.createAcidSortNodes(TableName.getDbTable(
             materializationNode.getChild(0).getText(),
-            materializationNode.getChild(1).getText()));
-    ParseDriver.adaptor.addChild(selectExprNodeInputROJ, tableName);
-    ParseDriver.adaptor.addChild(selectNodeInputROJ, selectExprNodeInputROJ);
+            materializationNode.getChild(1).getText()))
+            .forEach(astNode -> ParseDriver.adaptor.addChild(selectNodeInputROJ, astNode));
     // 4) Transform first INSERT branch into an UPDATE
-    // 4.1) Adding ROW__ID field
-    ASTNode selectNodeInUpdate = (ASTNode) updateNode.getChild(1);
-    if (selectNodeInUpdate.getType() != HiveParser.TOK_SELECT) {
-      throw new SemanticException("TOK_SELECT expected in incremental rewriting");
-    }
-    ASTNode selectExprNodeInUpdate = (ASTNode) ParseDriver.adaptor.dupNode(selectExprNodeInputROJ);
-    ParseDriver.adaptor.addChild(selectExprNodeInUpdate, createRowIdNode((ASTNode) subqueryNodeInputROJ.getChild(1)));
-    selectNodeInUpdate.insertChild(0, selectExprNodeInUpdate);
-    // 4.2) Modifying filter condition.
+    // 4.1) Modifying filter condition.
     ASTNode whereClauseInUpdate = findWhereClause(updateNode);
     if (whereClauseInUpdate.getChild(0).getType() != HiveParser.KW_OR) {
       throw new SemanticException("OR clause expected below TOK_WHERE in incremental rewriting");
     }
     // We bypass the OR clause and select the first disjunct for the Update branch
     ParseDriver.adaptor.setChild(whereClauseInUpdate, 0, disjuncts.get(Context.DestClausePrefix.UPDATE));
+    ASTNode updateDeleteNode = (ASTNode) ParseDriver.adaptor.dupTree(updateNode);
+    // 4.2) Adding ROW__ID field
+    ASTNode selectNodeInUpdateDelete = (ASTNode) updateDeleteNode.getChild(1);
+    if (selectNodeInUpdateDelete.getType() != HiveParser.TOK_SELECT) {
+      throw new SemanticException("TOK_SELECT expected in incremental rewriting");
+    }
+    // Remove children
+    while (selectNodeInUpdateDelete.getChildCount() > 0) {
+      selectNodeInUpdateDelete.deleteChild(0);

Review Comment:
   `org.antlr.runtime.tree.BaseTree` doesn't support removing all children in one call.



-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1154638925


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java:
##########
@@ -564,30 +582,38 @@ private void fixUpASTAggregateIncrementalRebuild(
             HiveParser.TOK_INSERT, HiveParser.TOK_SELECT);
     ASTNode selectExprNodeInputROJ = (ASTNode) ParseDriver.adaptor.create(
             HiveParser.TOK_SELEXPR, "TOK_SELEXPR");
-    ASTNode tableName = createRowIdNode(TableName.getDbTable(
+    astBuilder.createAcidSortNodes(TableName.getDbTable(
             materializationNode.getChild(0).getText(),
-            materializationNode.getChild(1).getText()));
-    ParseDriver.adaptor.addChild(selectExprNodeInputROJ, tableName);
-    ParseDriver.adaptor.addChild(selectNodeInputROJ, selectExprNodeInputROJ);
+            materializationNode.getChild(1).getText()))
+            .forEach(astNode -> ParseDriver.adaptor.addChild(selectNodeInputROJ, astNode));
     // 4) Transform first INSERT branch into an UPDATE
-    // 4.1) Adding ROW__ID field
-    ASTNode selectNodeInUpdate = (ASTNode) updateNode.getChild(1);
-    if (selectNodeInUpdate.getType() != HiveParser.TOK_SELECT) {
-      throw new SemanticException("TOK_SELECT expected in incremental rewriting");
-    }
-    ASTNode selectExprNodeInUpdate = (ASTNode) ParseDriver.adaptor.dupNode(selectExprNodeInputROJ);
-    ParseDriver.adaptor.addChild(selectExprNodeInUpdate, createRowIdNode((ASTNode) subqueryNodeInputROJ.getChild(1)));
-    selectNodeInUpdate.insertChild(0, selectExprNodeInUpdate);
-    // 4.2) Modifying filter condition.
+    // 4.1) Modifying filter condition.
     ASTNode whereClauseInUpdate = findWhereClause(updateNode);
     if (whereClauseInUpdate.getChild(0).getType() != HiveParser.KW_OR) {
       throw new SemanticException("OR clause expected below TOK_WHERE in incremental rewriting");
     }
     // We bypass the OR clause and select the first disjunct for the Update branch
     ParseDriver.adaptor.setChild(whereClauseInUpdate, 0, disjuncts.get(Context.DestClausePrefix.UPDATE));
+    ASTNode updateDeleteNode = (ASTNode) ParseDriver.adaptor.dupTree(updateNode);
+    // 4.2) Adding ROW__ID field
+    ASTNode selectNodeInUpdateDelete = (ASTNode) updateDeleteNode.getChild(1);
+    if (selectNodeInUpdateDelete.getType() != HiveParser.TOK_SELECT) {
+      throw new SemanticException("TOK_SELECT expected in incremental rewriting");

Review Comment:
   should we print the actual type? 



-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] kasakrisz commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "kasakrisz (via GitHub)" <gi...@apache.org>.
kasakrisz commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1155853269


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java:
##########
@@ -564,30 +582,38 @@ private void fixUpASTAggregateIncrementalRebuild(
             HiveParser.TOK_INSERT, HiveParser.TOK_SELECT);
     ASTNode selectExprNodeInputROJ = (ASTNode) ParseDriver.adaptor.create(
             HiveParser.TOK_SELEXPR, "TOK_SELEXPR");
-    ASTNode tableName = createRowIdNode(TableName.getDbTable(
+    astBuilder.createAcidSortNodes(TableName.getDbTable(
             materializationNode.getChild(0).getText(),
-            materializationNode.getChild(1).getText()));
-    ParseDriver.adaptor.addChild(selectExprNodeInputROJ, tableName);
-    ParseDriver.adaptor.addChild(selectNodeInputROJ, selectExprNodeInputROJ);
+            materializationNode.getChild(1).getText()))
+            .forEach(astNode -> ParseDriver.adaptor.addChild(selectNodeInputROJ, astNode));
     // 4) Transform first INSERT branch into an UPDATE
-    // 4.1) Adding ROW__ID field
-    ASTNode selectNodeInUpdate = (ASTNode) updateNode.getChild(1);
-    if (selectNodeInUpdate.getType() != HiveParser.TOK_SELECT) {
-      throw new SemanticException("TOK_SELECT expected in incremental rewriting");
-    }
-    ASTNode selectExprNodeInUpdate = (ASTNode) ParseDriver.adaptor.dupNode(selectExprNodeInputROJ);
-    ParseDriver.adaptor.addChild(selectExprNodeInUpdate, createRowIdNode((ASTNode) subqueryNodeInputROJ.getChild(1)));
-    selectNodeInUpdate.insertChild(0, selectExprNodeInUpdate);
-    // 4.2) Modifying filter condition.
+    // 4.1) Modifying filter condition.
     ASTNode whereClauseInUpdate = findWhereClause(updateNode);
     if (whereClauseInUpdate.getChild(0).getType() != HiveParser.KW_OR) {
       throw new SemanticException("OR clause expected below TOK_WHERE in incremental rewriting");
     }
     // We bypass the OR clause and select the first disjunct for the Update branch
     ParseDriver.adaptor.setChild(whereClauseInUpdate, 0, disjuncts.get(Context.DestClausePrefix.UPDATE));
+    ASTNode updateDeleteNode = (ASTNode) ParseDriver.adaptor.dupTree(updateNode);
+    // 4.2) Adding ROW__ID field
+    ASTNode selectNodeInUpdateDelete = (ASTNode) updateDeleteNode.getChild(1);
+    if (selectNodeInUpdateDelete.getType() != HiveParser.TOK_SELECT) {
+      throw new SemanticException("TOK_SELECT expected in incremental rewriting");

Review Comment:
   I added printing `selectNodeInUpdateDelete.getType()` which is the int representation if the token.



-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4166:
URL: https://github.com/apache/hive/pull/4166#issuecomment-1496319047

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4166)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4166&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4166&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1157183277


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/NonNativeMaterializedViewASTBuilder.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.hadoop.hive.ql.ddl.view.materialized.alter.rebuild;
+
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static java.util.Collections.singletonList;
+
+public class NonNativeMaterializedViewASTBuilder extends MaterializedViewASTBuilder {

Review Comment:
   what about FullAcidMaterializedViewASTBuilder? Should it be `NativeAcidMaterializedViewASTBuilder`?



-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1154629845


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java:
##########
@@ -465,21 +467,37 @@ private RelNode applyIncrementalRebuild(RelNode basePlan, RelMetadataProvider md
   protected ASTNode fixUpAfterCbo(ASTNode originalAst, ASTNode newAst, CalcitePlanner.PreCboCtx cboCtx)
           throws SemanticException {
     ASTNode fixedAST = super.fixUpAfterCbo(originalAst, newAst, cboCtx);
+    if (mvRebuildMode == MaterializationRebuildMode.INSERT_OVERWRITE_REBUILD) {

Review Comment:
   could we use `switch` statement



-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1154650166


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java:
##########
@@ -564,30 +582,38 @@ private void fixUpASTAggregateIncrementalRebuild(
             HiveParser.TOK_INSERT, HiveParser.TOK_SELECT);
     ASTNode selectExprNodeInputROJ = (ASTNode) ParseDriver.adaptor.create(
             HiveParser.TOK_SELEXPR, "TOK_SELEXPR");
-    ASTNode tableName = createRowIdNode(TableName.getDbTable(
+    astBuilder.createAcidSortNodes(TableName.getDbTable(
             materializationNode.getChild(0).getText(),
-            materializationNode.getChild(1).getText()));
-    ParseDriver.adaptor.addChild(selectExprNodeInputROJ, tableName);
-    ParseDriver.adaptor.addChild(selectNodeInputROJ, selectExprNodeInputROJ);
+            materializationNode.getChild(1).getText()))
+            .forEach(astNode -> ParseDriver.adaptor.addChild(selectNodeInputROJ, astNode));
     // 4) Transform first INSERT branch into an UPDATE
-    // 4.1) Adding ROW__ID field
-    ASTNode selectNodeInUpdate = (ASTNode) updateNode.getChild(1);
-    if (selectNodeInUpdate.getType() != HiveParser.TOK_SELECT) {
-      throw new SemanticException("TOK_SELECT expected in incremental rewriting");
-    }
-    ASTNode selectExprNodeInUpdate = (ASTNode) ParseDriver.adaptor.dupNode(selectExprNodeInputROJ);
-    ParseDriver.adaptor.addChild(selectExprNodeInUpdate, createRowIdNode((ASTNode) subqueryNodeInputROJ.getChild(1)));
-    selectNodeInUpdate.insertChild(0, selectExprNodeInUpdate);
-    // 4.2) Modifying filter condition.
+    // 4.1) Modifying filter condition.
     ASTNode whereClauseInUpdate = findWhereClause(updateNode);
     if (whereClauseInUpdate.getChild(0).getType() != HiveParser.KW_OR) {
       throw new SemanticException("OR clause expected below TOK_WHERE in incremental rewriting");
     }
     // We bypass the OR clause and select the first disjunct for the Update branch
     ParseDriver.adaptor.setChild(whereClauseInUpdate, 0, disjuncts.get(Context.DestClausePrefix.UPDATE));
+    ASTNode updateDeleteNode = (ASTNode) ParseDriver.adaptor.dupTree(updateNode);
+    // 4.2) Adding ROW__ID field
+    ASTNode selectNodeInUpdateDelete = (ASTNode) updateDeleteNode.getChild(1);
+    if (selectNodeInUpdateDelete.getType() != HiveParser.TOK_SELECT) {
+      throw new SemanticException("TOK_SELECT expected in incremental rewriting");
+    }
+    // Remove children
+    while (selectNodeInUpdateDelete.getChildCount() > 0) {
+      selectNodeInUpdateDelete.deleteChild(0);

Review Comment:
   can we remove all children in 1 go and don't call freshenParentAndChildIndexes for every child? 



-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] kasakrisz commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "kasakrisz (via GitHub)" <gi...@apache.org>.
kasakrisz commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1155850815


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/NonNativeMaterializedViewASTBuilder.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.hadoop.hive.ql.ddl.view.materialized.alter.rebuild;
+
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static java.util.Collections.singletonList;
+
+public class NonNativeMaterializedViewASTBuilder extends MaterializedViewASTBuilder {

Review Comment:
   Renamed to `NonNativeAcidMaterializedViewASTBuilder`



-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] kasakrisz commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "kasakrisz (via GitHub)" <gi...@apache.org>.
kasakrisz commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1157194454


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/NonNativeMaterializedViewASTBuilder.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.hadoop.hive.ql.ddl.view.materialized.alter.rebuild;
+
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static java.util.Collections.singletonList;
+
+public class NonNativeMaterializedViewASTBuilder extends MaterializedViewASTBuilder {

Review Comment:
   I think this name emphasizes that it is used in case of native a storage format. The same convention is used in case of `ColumnAppender`.



-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] kasakrisz merged pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "kasakrisz (via GitHub)" <gi...@apache.org>.
kasakrisz merged PR #4166:
URL: https://github.com/apache/hive/pull/4166


-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1154626481


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/NonNativeMaterializedViewASTBuilder.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.hadoop.hive.ql.ddl.view.materialized.alter.rebuild;
+
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static java.util.Collections.singletonList;
+
+public class NonNativeMaterializedViewASTBuilder extends MaterializedViewASTBuilder {

Review Comment:
   could we use consistent naming: NativeAcidMaterializedViewASTBuilder & NonNativeAcidMaterializedViewASTBuilder? see ColumnAppender



-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1154628405


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java:
##########
@@ -299,7 +300,8 @@ private RelNode applyRecordIncrementalRebuildPlan(
       // First we need to check if it is valid to convert to MERGE/INSERT INTO.
       // If we succeed, we modify the plan and afterwards the AST.
       // MV should be an acid table.
-      boolean fullAcidView = AcidUtils.isFullAcidTable(mvTable.getTTable());
+      boolean fullAcidView = AcidUtils.isFullAcidTable(mvTable.getTTable())

Review Comment:
   var name is confusing, maybe acidView or txnView?



-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] kasakrisz commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "kasakrisz (via GitHub)" <gi...@apache.org>.
kasakrisz commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1155851502


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java:
##########
@@ -465,21 +467,37 @@ private RelNode applyIncrementalRebuild(RelNode basePlan, RelMetadataProvider md
   protected ASTNode fixUpAfterCbo(ASTNode originalAst, ASTNode newAst, CalcitePlanner.PreCboCtx cboCtx)
           throws SemanticException {
     ASTNode fixedAST = super.fixUpAfterCbo(originalAst, newAst, cboCtx);
+    if (mvRebuildMode == MaterializationRebuildMode.INSERT_OVERWRITE_REBUILD) {
+      return fixedAST;
+    } else if (mvRebuildMode == MaterializationRebuildMode.JOIN_INSERT_REBUILD) {
+      fixUpASTJoinInsertIncrementalRebuild(fixedAST);
+      return fixedAST;
+    }
+
+    MaterializedViewASTBuilder astBuilder;
+    if (AcidUtils.isFullAcidTable(mvTable.getTTable())) {

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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] kasakrisz commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "kasakrisz (via GitHub)" <gi...@apache.org>.
kasakrisz commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1157469710


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/NonNativeMaterializedViewASTBuilder.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.hadoop.hive.ql.ddl.view.materialized.alter.rebuild;
+
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static java.util.Collections.singletonList;
+
+public class NonNativeMaterializedViewASTBuilder extends MaterializedViewASTBuilder {

Review Comment:
   Renamed



-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4166:
URL: https://github.com/apache/hive/pull/4166#issuecomment-1504010533

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4166)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL) [13 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4166&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4166&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4166:
URL: https://github.com/apache/hive/pull/4166#issuecomment-1502711234

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4166)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL) [6 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4166&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4166&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4166:
URL: https://github.com/apache/hive/pull/4166#issuecomment-1488471135

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4166)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL) [4 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4166&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4166&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4166:
URL: https://github.com/apache/hive/pull/4166#issuecomment-1488089545

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4166)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL) [4 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4166&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4166&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] kasakrisz commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "kasakrisz (via GitHub)" <gi...@apache.org>.
kasakrisz commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1155851179


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java:
##########
@@ -299,7 +300,8 @@ private RelNode applyRecordIncrementalRebuildPlan(
       // First we need to check if it is valid to convert to MERGE/INSERT INTO.
       // If we succeed, we modify the plan and afterwards the AST.
       // MV should be an acid table.
-      boolean fullAcidView = AcidUtils.isFullAcidTable(mvTable.getTTable());
+      boolean fullAcidView = AcidUtils.isFullAcidTable(mvTable.getTTable())

Review Comment:
   renamed to `acidView`



##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java:
##########
@@ -465,21 +467,37 @@ private RelNode applyIncrementalRebuild(RelNode basePlan, RelMetadataProvider md
   protected ASTNode fixUpAfterCbo(ASTNode originalAst, ASTNode newAst, CalcitePlanner.PreCboCtx cboCtx)
           throws SemanticException {
     ASTNode fixedAST = super.fixUpAfterCbo(originalAst, newAst, cboCtx);
+    if (mvRebuildMode == MaterializationRebuildMode.INSERT_OVERWRITE_REBUILD) {

Review Comment:
   refactored



-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] kasakrisz commented on a diff in pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "kasakrisz (via GitHub)" <gi...@apache.org>.
kasakrisz commented on code in PR #4166:
URL: https://github.com/apache/hive/pull/4166#discussion_r1157194454


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/NonNativeMaterializedViewASTBuilder.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.hadoop.hive.ql.ddl.view.materialized.alter.rebuild;
+
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static java.util.Collections.singletonList;
+
+public class NonNativeMaterializedViewASTBuilder extends MaterializedViewASTBuilder {

Review Comment:
   I think this name emphasizes that it is used in case of native a storage format. The same convention is used in case of `ColumnAppender`.



-- 
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-unsubscribe@hive.apache.org

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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4166: HIVE-27187: Incremental rebuild of materialized view having aggregate and stored by iceberg

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4166:
URL: https://github.com/apache/hive/pull/4166#issuecomment-1497081163

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4166)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4166&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4166&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4166&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4166&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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-unsubscribe@hive.apache.org

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