You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by "zhangbutao (via GitHub)" <gi...@apache.org> on 2023/04/11 09:02:52 UTC

[GitHub] [hive] zhangbutao opened a new pull request, #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   <!--
   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?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description, screenshot and/or a reproducable example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Hive versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   


-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -676,6 +678,35 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
     }
   }
 
+  @Override
+  public void createBranchOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
+      AlterTableCreateBranchSpec createBranchSpec) {
+    TableDesc tableDesc = Utilities.getTableDesc(hmsTable);
+    Table icebergTable = IcebergTableUtil.getTable(conf, tableDesc.getProperties());
+
+    String branchName = createBranchSpec.getBranchName();
+    Optional.ofNullable(icebergTable.currentSnapshot()).orElseThrow(() -> new UnsupportedOperationException(
+        String.format("Cannot create branch %s on iceberg table %s.%s which has no snapshot",
+            branchName, hmsTable.getDbName(), hmsTable.getTableName())));
+    Long snapshotId = Optional.ofNullable(createBranchSpec.getSnapshotId())
+        .orElse(icebergTable.currentSnapshot().snapshotId());
+    LOG.info("Creating branch {} on iceberg table {}.{}", branchName, hmsTable.getDbName(),

Review Comment:
   Could we add `snapshotId` to the log output? Btw, should we log under DEBUG? 



##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -676,6 +678,35 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
     }
   }
 
+  @Override
+  public void createBranchOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
+      AlterTableCreateBranchSpec createBranchSpec) {
+    TableDesc tableDesc = Utilities.getTableDesc(hmsTable);
+    Table icebergTable = IcebergTableUtil.getTable(conf, tableDesc.getProperties());
+
+    String branchName = createBranchSpec.getBranchName();
+    Optional.ofNullable(icebergTable.currentSnapshot()).orElseThrow(() -> new UnsupportedOperationException(
+        String.format("Cannot create branch %s on iceberg table %s.%s which has no snapshot",
+            branchName, hmsTable.getDbName(), hmsTable.getTableName())));
+    Long snapshotId = Optional.ofNullable(createBranchSpec.getSnapshotId())
+        .orElse(icebergTable.currentSnapshot().snapshotId());
+    LOG.info("Creating branch {} on iceberg table {}.{}", branchName, hmsTable.getDbName(),
+        hmsTable.getTableName());
+    ManageSnapshots manageSnapshots = icebergTable.manageSnapshots();
+    manageSnapshots.createBranch(branchName, snapshotId);

Review Comment:
   what happens if supplied `snapshotId` doesn't exist?



##########
parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g:
##########
@@ -477,6 +478,34 @@ alterStatementSuffixExecute
     -> ^(TOK_ALTERTABLE_EXECUTE KW_SET_CURRENT_SNAPSHOT $snapshotParam)
     ;
 
+alterStatementSuffixCreateBranch
+@init { gParent.pushMsg("alter table create branch", state); }
+@after { gParent.popMsg(state); }
+    : KW_CREATE KW_BRANCH branchName=identifier snapshotIdOfBranch? branchRetain? retentionOfSnapshots?
+    -> ^(TOK_ALTERTABLE_CREATE_BRANCH $branchName snapshotIdOfBranch? branchRetain? retentionOfSnapshots?)
+    ;
+
+snapshotIdOfBranch
+@init { gParent.pushMsg("alter table create branch as of version", state); }
+@after { gParent.popMsg(state); }
+    : KW_AS KW_OF KW_VERSION snapshotId=Number
+    -> ^(TOK_AS_OF_VERSION_BRANCH $snapshotId)

Review Comment:
   can't we reuse `TOK_AS_OF_VERSION`?



##########
parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g:
##########
@@ -477,6 +478,34 @@ alterStatementSuffixExecute
     -> ^(TOK_ALTERTABLE_EXECUTE KW_SET_CURRENT_SNAPSHOT $snapshotParam)
     ;
 
+alterStatementSuffixCreateBranch
+@init { gParent.pushMsg("alter table create branch", state); }
+@after { gParent.popMsg(state); }
+    : KW_CREATE KW_BRANCH branchName=identifier snapshotIdOfBranch? branchRetain? retentionOfSnapshots?
+    -> ^(TOK_ALTERTABLE_CREATE_BRANCH $branchName snapshotIdOfBranch? branchRetain? retentionOfSnapshots?)
+    ;
+
+snapshotIdOfBranch
+@init { gParent.pushMsg("alter table create branch as of version", state); }

Review Comment:
   should we support `as of timestamp` as well? Note, that syntax is different from the 'select' statement:
   ````
   FOR SYSTEM_VERSION AS OF
   FOR SYSTEM_TIME AS OF
   ````
   i think we should be consistent here. cc @ayushtkn 



##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AlterTableType.java:
##########
@@ -40,6 +40,7 @@ public enum AlterTableType {
   ALTERPARTITION("alter partition"), // Note: this is never used in AlterTableDesc.
   SETPARTITIONSPEC("set partition spec"),
   EXECUTE("execute"),
+  CREATEBRANCH("create branch"),

Review Comment:
   could we use a separator here, like `CREATE_BRANCH`



##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.HiveMetaHook;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    validateAlterTableType(table, AlterTableType.CREATEBRANCH, false);
+    if (!HiveMetaHook.ICEBERG.equalsIgnoreCase(table.getParameters().get(HiveMetaHook.TABLE_TYPE))) {

Review Comment:
   we should move this logic into a helper class



##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableCreateBranchSpec.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.parse;
+
+import com.google.common.base.MoreObjects;
+
+public class AlterTableCreateBranchSpec {

Review Comment:
   can we reuse `AlterTableExecuteSpec` and define inner `CreateBranchSpec`, similar to `RollbackSpec`?



-- 
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] zhangbutao commented on pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

Posted by "zhangbutao (via GitHub)" <gi...@apache.org>.
zhangbutao commented on PR #4216:
URL: https://github.com/apache/hive/pull/4216#issuecomment-1547413623

   @deniskuzZ @ayushtkn Can we merge this PR first? Thanks.


-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)

Review Comment:
   Thanks for comment @TuroczyX . IIUC, I think no need check null here as the parser moduel will do this.  We have defined all required and optional parameters in [`AlterClauseParser.g` ](https://github.com/apache/hive/pull/4216/files#diff-3b0a324eebcad6e32a1bf8bf42499df0739207dbe96f32a3d5e64a44eb002d0fR473), and the then Antlr parser will check if the parameter is correct.



-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   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=4216)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java:
##########
@@ -6709,6 +6710,15 @@ public void alterTableExecuteOperation(Table table, AlterTableExecuteSpec execut
     }
   }
 
+  public void alterTableCreateBranchOperation(Table table, AlterTableCreateBranchSpec createBranchSpec) throws HiveException {

Review Comment:
   I have rename it to 'alterTableBranchOperation' which can be used to create/drop/replace branch. Is it ok?



-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   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=4216)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   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=4216)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLUtils.java:
##########
@@ -217,4 +218,11 @@ public static void setColumnsAndStorePartitionTransformSpecOfTable(
       partCols.ifPresent(tbl::setPartCols);
     }
   }
+
+  public static void validateTableIsIceberg(org.apache.hadoop.hive.ql.metadata.Table table)

Review Comment:
   yep, it duplicates `HiveTableOperations.validateTableIsIceberg` as DDLTask in`ql `  package can not import iceberg class.
    
   I changed the thrown exception to` Not an iceberg table: %s (type=%s)` , this means the same thing as `HiveTableOperations.validateTableIsIceberg`



-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    validateAlterTableType(table, AlterTableType.CREATEBRANCH, false);
+    if (!"ICEBERG".equalsIgnoreCase(table.getParameters().get("table_type"))) {
+      throw new SemanticException("Cannot perform ALTER CREATE BRANCH statement on non-iceberg table.");
+    }
+    inputs.add(new ReadEntity(table));
+
+    String branchName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long maxRefAgeMs = null;
+    Integer minSnapshotsToKeep = null;
+    Long maxSnapshotAgeMs = null;
+    for (int i = 1; i < command.getChildCount(); i++) {
+      ASTNode childNode = (ASTNode) command.getChild(i);
+      switch (childNode.getToken().getType()) {
+      case HiveParser.TOK_AS_OF_VERSION_BRANCH:
+        snapshotId = Long.valueOf(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_RETAIN:
+        String maxRefAge = childNode.getChild(0).getText();
+        String timeUnitOfBranchRetain = childNode.getChild(1).getText();
+        maxRefAgeMs = TimeUnit.valueOf(timeUnitOfBranchRetain.toUpperCase(Locale.ENGLISH)).toMillis(Long.valueOf(maxRefAge));
+        break;
+      case HiveParser.TOK_WITH_SNAPSHOT_RETENTION:
+        minSnapshotsToKeep = Integer.valueOf(childNode.getChild(0).getText());
+        if (childNode.getChildren().size() > 1) {
+          String maxSnapshotAge = childNode.getChild(1).getText();
+          String timeUnitOfSnapshotsRetention = childNode.getChild(2).getText();
+          maxSnapshotAgeMs = TimeUnit.valueOf(timeUnitOfSnapshotsRetention.toUpperCase(Locale.ENGLISH)).toMillis(Long.valueOf(maxSnapshotAge));
+        }
+        break;
+      default:
+        throw new SemanticException("Unrecognized token in ALTER CREATE BRANCH statement");
+      }
+    }
+
+    AlterTableCreateBranchSpec spec = new AlterTableCreateBranchSpec(branchName, snapshotId, maxRefAgeMs, minSnapshotsToKeep, maxSnapshotAgeMs);

Review Comment:
   fixed. Thx.



-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -676,6 +678,32 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
     }
   }
 
+  @Override
+  public void createBranchOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
+      AlterTableCreateBranchSpec createBranchSpec) {
+    TableDesc tableDesc = Utilities.getTableDesc(hmsTable);
+    Table icebergTable = IcebergTableUtil.getTable(conf, tableDesc.getProperties());
+
+    String branchName = createBranchSpec.getBranchName();
+    ManageSnapshots manageSnapshots = icebergTable.manageSnapshots();
+    Long snapShotId = Optional.ofNullable(createBranchSpec.getSnapshotId())
+        .orElse(icebergTable.currentSnapshot().snapshotId());

Review Comment:
   Nice catch! fixed. Thanks very much!



-- 
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] zhangbutao closed pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

Posted by "zhangbutao (via GitHub)" <gi...@apache.org>.
zhangbutao closed pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation
URL: https://github.com/apache/hive/pull/4216


-- 
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] zhangbutao commented on pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

Posted by "zhangbutao (via GitHub)" <gi...@apache.org>.
zhangbutao commented on PR #4216:
URL: https://github.com/apache/hive/pull/4216#issuecomment-1528833729

   > Thanx @zhangbutao, overall changes looks good, dropped some comments.
   > 
   > Additionally your code ain't formatted according to hive formatting rules, line length should be less than 120 and other, can refer here: https://github.com/apache/hive/blob/master/dev-support/eclipse-styles.xml
   > 
   > Additionally add a q test. Where you do an explain of the create branch statement before creating the branch.
   > 
   > Add some failure test cases: Branch already exist and you try to create a branch Attempt to create a branch on a non iceberg table
   > 
   > In the positive test case check for the data as well.
   
   I think i have addressed these comments. Please take a look again if you have time. Thanks @ayushtkn 


-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g:
##########
@@ -477,6 +478,34 @@ alterStatementSuffixExecute
     -> ^(TOK_ALTERTABLE_EXECUTE KW_SET_CURRENT_SNAPSHOT $snapshotParam)
     ;
 
+alterStatementSuffixCreateBranch
+@init { gParent.pushMsg("alter table create branch", state); }
+@after { gParent.popMsg(state); }
+    : KW_CREATE KW_BRANCH branchName=identifier snapshotIdOfBranch? branchRetain? retentionOfSnapshots?
+    -> ^(TOK_ALTERTABLE_CREATE_BRANCH $branchName snapshotIdOfBranch? branchRetain? retentionOfSnapshots?)
+    ;
+
+snapshotIdOfBranch
+@init { gParent.pushMsg("alter table create branch as of version", state); }

Review Comment:
   👍 



-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java:
##########
@@ -6709,6 +6710,15 @@ public void alterTableExecuteOperation(Table table, AlterTableExecuteSpec execut
     }
   }
 
+  public void alterTableCreateBranchOperation(Table table, AlterTableCreateBranchSpec createBranchSpec) throws HiveException {

Review Comment:
   maybe simply 'alterTableCreateBranch' ?



-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableCreateBranchSpec.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.parse;
+
+import com.google.common.base.MoreObjects;
+
+public class AlterTableCreateBranchSpec {

Review Comment:
   Finally, i decide not to reuse codes about `alter table execute` syntax , as i think it is more important to make code clear and readable.
    I  used `AlterTableBranchSpec ` which looks like the pattern `AlterTableExecuteSpec` , and then define inner `CreateBranchSpec` which type is `CREATE_BRANCH`. We can continue to add other brancn operation e.g `DROP_BRANCH`, `REPLACE_BRANCH`.
   
   Please take a look again to see if it is make more sense. Thanks.



-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   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=4216)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   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=4216)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&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] ayushtkn commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    validateAlterTableType(table, AlterTableType.CREATEBRANCH, false);
+    if (!"ICEBERG".equalsIgnoreCase(table.getParameters().get("table_type"))) {
+      throw new SemanticException("Cannot perform ALTER CREATE BRANCH statement on non-iceberg table.");
+    }
+    inputs.add(new ReadEntity(table));
+
+    String branchName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long maxRefAgeMs = null;
+    Integer minSnapshotsToKeep = null;
+    Long maxSnapshotAgeMs = null;
+    for (int i = 1; i < command.getChildCount(); i++) {
+      ASTNode childNode = (ASTNode) command.getChild(i);
+      switch (childNode.getToken().getType()) {
+      case HiveParser.TOK_AS_OF_VERSION_BRANCH:
+        snapshotId = Long.valueOf(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_RETAIN:
+        String maxRefAge = childNode.getChild(0).getText();
+        String timeUnitOfBranchRetain = childNode.getChild(1).getText();
+        maxRefAgeMs = TimeUnit.valueOf(timeUnitOfBranchRetain.toUpperCase(Locale.ENGLISH)).toMillis(Long.valueOf(maxRefAge));
+        break;
+      case HiveParser.TOK_WITH_SNAPSHOT_RETENTION:
+        minSnapshotsToKeep = Integer.valueOf(childNode.getChild(0).getText());
+        if (childNode.getChildren().size() > 1) {
+          String maxSnapshotAge = childNode.getChild(1).getText();
+          String timeUnitOfSnapshotsRetention = childNode.getChild(2).getText();
+          maxSnapshotAgeMs = TimeUnit.valueOf(timeUnitOfSnapshotsRetention.toUpperCase(Locale.ENGLISH)).toMillis(Long.valueOf(maxSnapshotAge));

Review Comment:
   use. ``Long.parseLong(maxSnapshotAge)``



-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   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=4216)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -676,6 +678,35 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
     }
   }
 
+  @Override
+  public void createBranchOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
+      AlterTableCreateBranchSpec createBranchSpec) {
+    TableDesc tableDesc = Utilities.getTableDesc(hmsTable);
+    Table icebergTable = IcebergTableUtil.getTable(conf, tableDesc.getProperties());
+
+    String branchName = createBranchSpec.getBranchName();
+    Optional.ofNullable(icebergTable.currentSnapshot()).orElseThrow(() -> new UnsupportedOperationException(
+        String.format("Cannot create branch %s on iceberg table %s.%s which has no snapshot",
+            branchName, hmsTable.getDbName(), hmsTable.getTableName())));
+    Long snapshotId = Optional.ofNullable(createBranchSpec.getSnapshotId())
+        .orElse(icebergTable.currentSnapshot().snapshotId());
+    LOG.info("Creating branch {} on iceberg table {}.{}", branchName, hmsTable.getDbName(),
+        hmsTable.getTableName());
+    ManageSnapshots manageSnapshots = icebergTable.manageSnapshots();
+    manageSnapshots.createBranch(branchName, snapshotId);

Review Comment:
   If user omits ` snapshotId ` when creating branch, then current snapshotId will be used.
   if user passes a non-existent `snapshotId` when creating branch, exception `unknown snapshot` will be thrown from iceberg lib.



-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g:
##########
@@ -477,6 +478,34 @@ alterStatementSuffixExecute
     -> ^(TOK_ALTERTABLE_EXECUTE KW_SET_CURRENT_SNAPSHOT $snapshotParam)
     ;
 
+alterStatementSuffixCreateBranch
+@init { gParent.pushMsg("alter table create branch", state); }
+@after { gParent.popMsg(state); }
+    : KW_CREATE KW_BRANCH branchName=identifier snapshotIdOfBranch? branchRetain? retentionOfSnapshots?
+    -> ^(TOK_ALTERTABLE_CREATE_BRANCH $branchName snapshotIdOfBranch? branchRetain? retentionOfSnapshots?)
+    ;
+
+snapshotIdOfBranch
+@init { gParent.pushMsg("alter table create branch as of version", state); }

Review Comment:
   should we support `as of timestamp` as well? Note, that syntax is different from the 'select' statement:
   ````
   FOR SYSTEM_VERSION AS OF
   FOR SYSTEM_TIME AS OF
   ````
   i think we should be consistent here, WDYT? cc @ayushtkn 



-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -676,6 +678,35 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
     }
   }
 
+  @Override
+  public void createBranchOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
+      AlterTableCreateBranchSpec createBranchSpec) {
+    TableDesc tableDesc = Utilities.getTableDesc(hmsTable);
+    Table icebergTable = IcebergTableUtil.getTable(conf, tableDesc.getProperties());
+
+    String branchName = createBranchSpec.getBranchName();
+    Optional.ofNullable(icebergTable.currentSnapshot()).orElseThrow(() -> new UnsupportedOperationException(
+        String.format("Cannot create branch %s on iceberg table %s.%s which has no snapshot",
+            branchName, hmsTable.getDbName(), hmsTable.getTableName())));
+    Long snapshotId = Optional.ofNullable(createBranchSpec.getSnapshotId())
+        .orElse(icebergTable.currentSnapshot().snapshotId());
+    LOG.info("Creating branch {} on iceberg table {}.{}", branchName, hmsTable.getDbName(),

Review Comment:
   Added `snapshotId `. I think INFO is enough as this DDL is rarely executed.



-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    validateAlterTableType(table, AlterTableType.CREATEBRANCH, false);
+    if (!"ICEBERG".equalsIgnoreCase(table.getParameters().get("table_type"))) {

Review Comment:
   fixed. Thx



-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -676,6 +678,32 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
     }
   }
 
+  @Override
+  public void createBranchOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
+      AlterTableCreateBranchSpec createBranchSpec) {
+    TableDesc tableDesc = Utilities.getTableDesc(hmsTable);
+    Table icebergTable = IcebergTableUtil.getTable(conf, tableDesc.getProperties());
+
+    String branchName = createBranchSpec.getBranchName();
+    ManageSnapshots manageSnapshots = icebergTable.manageSnapshots();
+    Long snapShotId = Optional.ofNullable(createBranchSpec.getSnapshotId())
+        .orElse(icebergTable.currentSnapshot().snapshotId());
+    LOG.info("Creating branch {} on iceberg table {}.{}", branchName, hmsTable.getDbName(),
+        hmsTable.getTableName());
+    manageSnapshots.createBranch(branchName, snapShotId);

Review Comment:
   fixed.



-- 
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] TuroczyX commented on pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

Posted by "TuroczyX (via GitHub)" <gi...@apache.org>.
TuroczyX commented on PR #4216:
URL: https://github.com/apache/hive/pull/4216#issuecomment-1515464168

   ![image](https://user-images.githubusercontent.com/7687174/233213641-1439d6c1-b799-4939-830a-177b7f733f86.png)
   


-- 
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] zhangbutao commented on pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

Posted by "zhangbutao (via GitHub)" <gi...@apache.org>.
zhangbutao commented on PR #4216:
URL: https://github.com/apache/hive/pull/4216#issuecomment-1527752401

   @ayushtkn Thanks for review! I will probably fix these comments after a few days of vacation.


-- 
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] ayushtkn commented on pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

Posted by "ayushtkn (via GitHub)" <gi...@apache.org>.
ayushtkn commented on PR #4216:
URL: https://github.com/apache/hive/pull/4216#issuecomment-1515454235

   Didn't check the code, But it was something I was also reading coincidently 2-3 hours before only :) 
   
   The main thing to chase is not creating a branch. But to insert into those branches. @zhangbutao We should chase that once we have the create in. Will review this in a day or two along with Denys. :) 
   
   
   Just FYI. I think Iceberg has options to create Tags as well now, and an option to do cherry-pick to branches as well. They are gonna release it for spark in 1.12.0
   
   Good to have stuff!!!


-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.io.IOException;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.Table;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestHiveIcebergBranchOperation extends HiveIcebergStorageHandlerWithEngineBase {
+
+  @Test

Review Comment:
   I think multiple small tests are better for reading. But if you think one test is ok, I'm fine with that. ;)
   



-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
iceberg/iceberg-handler/src/test/queries/negative/alter_table_create_branch_negative.q:
##########
@@ -0,0 +1,3 @@
+create table ice_tbl (id int, name string) Stored by Iceberg;
+
+alter table ice_tbl create branch test_branch_1;

Review Comment:
   Qtest: creating branch on table without current snapshot will fail



##########
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.io.IOException;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.Table;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestHiveIcebergBranchOperation extends HiveIcebergStorageHandlerWithEngineBase {
+
+  @Test
+  public void testCreateBranchWithDefaultConfig() throws InterruptedException, IOException {
+    Table table =
+        testTables.createTableWithVersions(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
+            fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2);
+
+    String branchName = "test_branch_1";
+    shell.executeStatement(String.format("ALTER TABLE customers CREATE BRANCH %s", branchName));
+    table.refresh();
+    SnapshotRef ref = table.refs().get(branchName);
+    Assert.assertEquals(table.currentSnapshot().snapshotId(), ref.snapshotId());
+    Assert.assertNull(ref.minSnapshotsToKeep());
+    Assert.assertNull(ref.maxSnapshotAgeMs());
+    Assert.assertNull(ref.maxRefAgeMs());
+
+    // creating a branch which is already exists will fail
+    try {
+      shell.executeStatement(String.format("ALTER TABLE customers CREATE BRANCH %s", branchName));

Review Comment:
   Test: creating a branch which is already exists will fail



##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -676,6 +678,35 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
     }
   }
 
+  @Override
+  public void createBranchOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
+      AlterTableCreateBranchSpec createBranchSpec) {
+    TableDesc tableDesc = Utilities.getTableDesc(hmsTable);
+    Table icebergTable = IcebergTableUtil.getTable(conf, tableDesc.getProperties());
+
+    String branchName = createBranchSpec.getBranchName();
+    Optional.ofNullable(icebergTable.currentSnapshot()).orElseThrow(() -> new UnsupportedOperationException(

Review Comment:
   Here give a check if table have current snapshot.



##########
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.io.IOException;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.Table;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestHiveIcebergBranchOperation extends HiveIcebergStorageHandlerWithEngineBase {
+
+  @Test
+  public void testCreateBranchWithDefaultConfig() throws InterruptedException, IOException {
+    Table table =
+        testTables.createTableWithVersions(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
+            fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2);
+
+    String branchName = "test_branch_1";
+    shell.executeStatement(String.format("ALTER TABLE customers CREATE BRANCH %s", branchName));
+    table.refresh();
+    SnapshotRef ref = table.refs().get(branchName);
+    Assert.assertEquals(table.currentSnapshot().snapshotId(), ref.snapshotId());
+    Assert.assertNull(ref.minSnapshotsToKeep());
+    Assert.assertNull(ref.maxSnapshotAgeMs());
+    Assert.assertNull(ref.maxRefAgeMs());
+
+    // creating a branch which is already exists will fail
+    try {
+      shell.executeStatement(String.format("ALTER TABLE customers CREATE BRANCH %s", branchName));
+    } catch (Throwable e) {
+      while (e.getCause() != null) {
+        e = e.getCause();
+      }
+      Assert.assertTrue(e.getMessage().contains("Ref test_branch_1 already exists"));
+    }
+  }
+
+  @Test
+  public void testCreateBranchWithSnapshotId() throws InterruptedException, IOException {
+    Table table =
+        testTables.createTableWithVersions(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
+            fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2);
+
+    String branchName = "test_branch_1";
+    Long snapshotId = table.history().get(0).snapshotId();
+    shell.executeStatement(String.format("ALTER TABLE customers CREATE BRANCH %s AS OF VERSION %d",
+        branchName, snapshotId));
+    table.refresh();
+    SnapshotRef ref = table.refs().get(branchName);
+    Assert.assertEquals(snapshotId.longValue(), ref.snapshotId());
+    Assert.assertNull(ref.minSnapshotsToKeep());
+    Assert.assertNull(ref.maxSnapshotAgeMs());
+    Assert.assertNull(ref.maxRefAgeMs());
+  }
+
+  @Test
+  public void testCreateBranchWithMaxRefAge() throws InterruptedException, IOException {
+    Table table =
+        testTables.createTableWithVersions(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
+            fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2);
+
+    String branchName = "test_branch_1";
+    long maxRefAge = 5L;
+    shell.executeStatement(String.format("ALTER TABLE customers CREATE BRANCH %s RETAIN %d DAYS",
+        branchName, maxRefAge));
+    table.refresh();
+    SnapshotRef ref = table.refs().get(branchName);
+    Assert.assertEquals(table.currentSnapshot().snapshotId(), ref.snapshotId());
+    Assert.assertNull(ref.minSnapshotsToKeep());
+    Assert.assertNull(ref.maxSnapshotAgeMs());
+    Assert.assertEquals(TimeUnit.DAYS.toMillis(maxRefAge), ref.maxRefAgeMs().longValue());
+  }
+
+  @Test
+  public void testCreateBranchWithMinSnapshotsToKeep() throws InterruptedException, IOException {
+    Table table =
+        testTables.createTableWithVersions(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
+            fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2);
+
+    String branchName = "test_branch_1";
+    Integer minSnapshotsToKeep = 2;
+    shell.executeStatement(String.format("ALTER TABLE customers CREATE BRANCH %s WITH SNAPSHOT RETENTION %d SNAPSHOTS",
+        branchName, minSnapshotsToKeep));
+    table.refresh();
+    SnapshotRef ref = table.refs().get(branchName);
+    Assert.assertEquals(table.currentSnapshot().snapshotId(), ref.snapshotId());
+    Assert.assertEquals(minSnapshotsToKeep, ref.minSnapshotsToKeep());
+    Assert.assertNull(ref.maxSnapshotAgeMs());
+    Assert.assertNull(ref.maxRefAgeMs());
+  }
+
+  @Test
+  public void testCreateBranchWithMinSnapshotsToKeepAndMaxSnapshotAge() throws InterruptedException, IOException {
+    Table table =
+        testTables.createTableWithVersions(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
+            fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2);
+
+    String branchName = "test_branch_1";
+    Integer minSnapshotsToKeep = 2;
+    long maxSnapshotAge = 2L;
+    shell.executeStatement(String.format("ALTER TABLE customers CREATE BRANCH %s WITH SNAPSHOT RETENTION %d SNAPSHOTS" +
+        " %d DAYS", branchName, minSnapshotsToKeep, maxSnapshotAge));
+    table.refresh();
+    SnapshotRef ref = table.refs().get(branchName);
+    Assert.assertEquals(table.currentSnapshot().snapshotId(), ref.snapshotId());
+    Assert.assertEquals(minSnapshotsToKeep, ref.minSnapshotsToKeep());
+    Assert.assertEquals(TimeUnit.DAYS.toMillis(maxSnapshotAge), ref.maxSnapshotAgeMs().longValue());
+    Assert.assertNull(ref.maxRefAgeMs());
+  }
+
+  @Test
+  public void testCreateBranchWithAllCustomConfig() throws InterruptedException, IOException {
+    Table table =
+        testTables.createTableWithVersions(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
+            fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2);
+
+    String branchName = "test_branch_1";
+    Long snapshotId = table.history().get(0).snapshotId();
+    Integer minSnapshotsToKeep = 2;
+    long maxSnapshotAge = 2L;
+    long maxRefAge = 5L;
+    shell.executeStatement(String.format("ALTER TABLE customers CREATE BRANCH %s AS OF VERSION %d RETAIN %d DAYS WITH" +
+            " SNAPSHOT RETENTION %d SNAPSHOTS %d days",
+        branchName, snapshotId, maxRefAge, minSnapshotsToKeep, maxSnapshotAge));
+    table.refresh();
+    SnapshotRef ref = table.refs().get(branchName);
+    Assert.assertEquals(snapshotId.longValue(), ref.snapshotId());
+    Assert.assertEquals(minSnapshotsToKeep, ref.minSnapshotsToKeep());
+    Assert.assertEquals(TimeUnit.DAYS.toMillis(maxSnapshotAge), ref.maxSnapshotAgeMs().longValue());
+    Assert.assertEquals(TimeUnit.DAYS.toMillis(maxRefAge), ref.maxRefAgeMs().longValue());
+  }
+
+  @Test
+  public void testCreateBranchWithNonIcebergTable() {
+    shell.executeStatement("create table nonice_tbl (id int, name string)");
+
+    String branchName = "test_branch_1";
+    try {
+      shell.executeStatement(String.format("ALTER TABLE nonice_tbl CREATE BRANCH %s", branchName));

Review Comment:
   Test: creating branch on a non-icebeg table will fail



-- 
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] zhangbutao commented on pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

Posted by "zhangbutao (via GitHub)" <gi...@apache.org>.
zhangbutao commented on PR #4216:
URL: https://github.com/apache/hive/pull/4216#issuecomment-1544002029

   Git rebase to fix code conflicts.


-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   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=4216)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=CODE_SMELL) [38 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&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] zhangbutao closed pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

Posted by "zhangbutao (via GitHub)" <gi...@apache.org>.
zhangbutao closed pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation
URL: https://github.com/apache/hive/pull/4216


-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   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=4216)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AlterTableType.java:
##########
@@ -40,6 +40,7 @@ public enum AlterTableType {
   ALTERPARTITION("alter partition"), // Note: this is never used in AlterTableDesc.
   SETPARTITIONSPEC("set partition spec"),
   EXECUTE("execute"),
+  CREATEBRANCH("create branch"),

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] TuroczyX commented on pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

Posted by "TuroczyX (via GitHub)" <gi...@apache.org>.
TuroczyX commented on PR #4216:
URL: https://github.com/apache/hive/pull/4216#issuecomment-1515449032

   uhhhhh! This is super cool idea!  I want it! Actually with this it is possible to play on dataset without any risk. I love it! I want it!
   @deniskuzZ @ayushtkn @simhadri-g Could you please make this review a priority?
   
   Documents about it:
   https://docs.google.com/document/d/1tbATFPrKF3vNlzkgZQdaW8CAJmbjvryfrlg6C2Ci_aA/edit#heading=h.v8gsu2fe19q2 
   


-- 
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] TuroczyX commented on pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

Posted by "TuroczyX (via GitHub)" <gi...@apache.org>.
TuroczyX commented on PR #4216:
URL: https://github.com/apache/hive/pull/4216#issuecomment-1515852966

   yes, let's upgrade the iceberg version


-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   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=4216)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&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] ayushtkn commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g:
##########
@@ -477,6 +478,34 @@ alterStatementSuffixExecute
     -> ^(TOK_ALTERTABLE_EXECUTE KW_SET_CURRENT_SNAPSHOT $snapshotParam)
     ;
 
+alterStatementSuffixCreateBranch
+@init { gParent.pushMsg("alter table create branch", state); }
+@after { gParent.popMsg(state); }
+    : KW_CREATE KW_BRANCH branchName=identifier snapshotIdOfBranch? branchRetain? retentionOfSnapshots?
+    -> ^(TOK_ALTERTABLE_CREATE_BRANCH $branchName snapshotIdOfBranch? branchRetain? retentionOfSnapshots?)
+    ;
+
+snapshotIdOfBranch
+@init { gParent.pushMsg("alter table create branch as of version", state); }

Review Comment:
   Makes sense to be in sync. @zhangbutao can you change it



-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g:
##########
@@ -477,6 +478,34 @@ alterStatementSuffixExecute
     -> ^(TOK_ALTERTABLE_EXECUTE KW_SET_CURRENT_SNAPSHOT $snapshotParam)
     ;
 
+alterStatementSuffixCreateBranch
+@init { gParent.pushMsg("alter table create branch", state); }
+@after { gParent.popMsg(state); }
+    : KW_CREATE KW_BRANCH branchName=identifier snapshotIdOfBranch? branchRetain? retentionOfSnapshots?
+    -> ^(TOK_ALTERTABLE_CREATE_BRANCH $branchName snapshotIdOfBranch? branchRetain? retentionOfSnapshots?)
+    ;
+
+snapshotIdOfBranch
+@init { gParent.pushMsg("alter table create branch as of version", state); }
+@after { gParent.popMsg(state); }
+    : KW_AS KW_OF KW_VERSION snapshotId=Number
+    -> ^(TOK_AS_OF_VERSION_BRANCH $snapshotId)

Review Comment:
   As mentioned above, we can reuse `TOK_AS_OF_VERSION` if we use `FOR SYSTEM_VERSION AS OF` instead of `AS OF VERSION`.



-- 
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] TuroczyX commented on pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

Posted by "TuroczyX (via GitHub)" <gi...@apache.org>.
TuroczyX commented on PR #4216:
URL: https://github.com/apache/hive/pull/4216#issuecomment-1515462400

   Tags!!!! 😍 


-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   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=4216)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   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=4216)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=CODE_SMELL) [27 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&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] ayushtkn commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.io.IOException;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.Table;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestHiveIcebergBranchOperation extends HiveIcebergStorageHandlerWithEngineBase {
+
+  @Test

Review Comment:
   Do you need so many tests? Just create a table once and can create multiple branches of that with different parameters ?



##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -676,6 +678,32 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
     }
   }
 
+  @Override
+  public void createBranchOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
+      AlterTableCreateBranchSpec createBranchSpec) {
+    TableDesc tableDesc = Utilities.getTableDesc(hmsTable);
+    Table icebergTable = IcebergTableUtil.getTable(conf, tableDesc.getProperties());
+
+    String branchName = createBranchSpec.getBranchName();
+    ManageSnapshots manageSnapshots = icebergTable.manageSnapshots();
+    Long snapShotId = Optional.ofNullable(createBranchSpec.getSnapshotId())
+        .orElse(icebergTable.currentSnapshot().snapshotId());

Review Comment:
   What happens to this in case of any empty table? Just a create iceberg table and nothing. As far as I remember in that case currentSnapshot is null? Just give a check once.



##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -676,6 +678,32 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
     }
   }
 
+  @Override
+  public void createBranchOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
+      AlterTableCreateBranchSpec createBranchSpec) {
+    TableDesc tableDesc = Utilities.getTableDesc(hmsTable);
+    Table icebergTable = IcebergTableUtil.getTable(conf, tableDesc.getProperties());
+
+    String branchName = createBranchSpec.getBranchName();
+    ManageSnapshots manageSnapshots = icebergTable.manageSnapshots();
+    Long snapShotId = Optional.ofNullable(createBranchSpec.getSnapshotId())
+        .orElse(icebergTable.currentSnapshot().snapshotId());
+    LOG.info("Creating branch {} on iceberg table {}.{}", branchName, hmsTable.getDbName(),
+        hmsTable.getTableName());
+    manageSnapshots.createBranch(branchName, snapShotId);

Review Comment:
   nit:
   snapShotId -> snapshotId



##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    validateAlterTableType(table, AlterTableType.CREATEBRANCH, false);
+    if (!"ICEBERG".equalsIgnoreCase(table.getParameters().get("table_type"))) {
+      throw new SemanticException("Cannot perform ALTER CREATE BRANCH statement on non-iceberg table.");
+    }
+    inputs.add(new ReadEntity(table));
+
+    String branchName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long maxRefAgeMs = null;
+    Integer minSnapshotsToKeep = null;
+    Long maxSnapshotAgeMs = null;
+    for (int i = 1; i < command.getChildCount(); i++) {
+      ASTNode childNode = (ASTNode) command.getChild(i);
+      switch (childNode.getToken().getType()) {
+      case HiveParser.TOK_AS_OF_VERSION_BRANCH:
+        snapshotId = Long.valueOf(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_RETAIN:
+        String maxRefAge = childNode.getChild(0).getText();
+        String timeUnitOfBranchRetain = childNode.getChild(1).getText();
+        maxRefAgeMs = TimeUnit.valueOf(timeUnitOfBranchRetain.toUpperCase(Locale.ENGLISH)).toMillis(Long.valueOf(maxRefAge));
+        break;
+      case HiveParser.TOK_WITH_SNAPSHOT_RETENTION:
+        minSnapshotsToKeep = Integer.valueOf(childNode.getChild(0).getText());
+        if (childNode.getChildren().size() > 1) {
+          String maxSnapshotAge = childNode.getChild(1).getText();
+          String timeUnitOfSnapshotsRetention = childNode.getChild(2).getText();
+          maxSnapshotAgeMs = TimeUnit.valueOf(timeUnitOfSnapshotsRetention.toUpperCase(Locale.ENGLISH)).toMillis(Long.valueOf(maxSnapshotAge));
+        }
+        break;
+      default:
+        throw new SemanticException("Unrecognized token in ALTER CREATE BRANCH statement");
+      }
+    }
+
+    AlterTableCreateBranchSpec spec = new AlterTableCreateBranchSpec(branchName, snapshotId, maxRefAgeMs, minSnapshotsToKeep, maxSnapshotAgeMs);

Review Comment:
   line length should be less than 120 in hive



##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    validateAlterTableType(table, AlterTableType.CREATEBRANCH, false);
+    if (!"ICEBERG".equalsIgnoreCase(table.getParameters().get("table_type"))) {
+      throw new SemanticException("Cannot perform ALTER CREATE BRANCH statement on non-iceberg table.");
+    }
+    inputs.add(new ReadEntity(table));
+
+    String branchName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long maxRefAgeMs = null;
+    Integer minSnapshotsToKeep = null;
+    Long maxSnapshotAgeMs = null;
+    for (int i = 1; i < command.getChildCount(); i++) {
+      ASTNode childNode = (ASTNode) command.getChild(i);
+      switch (childNode.getToken().getType()) {
+      case HiveParser.TOK_AS_OF_VERSION_BRANCH:
+        snapshotId = Long.valueOf(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_RETAIN:
+        String maxRefAge = childNode.getChild(0).getText();
+        String timeUnitOfBranchRetain = childNode.getChild(1).getText();
+        maxRefAgeMs = TimeUnit.valueOf(timeUnitOfBranchRetain.toUpperCase(Locale.ENGLISH)).toMillis(Long.valueOf(maxRefAge));
+        break;
+      case HiveParser.TOK_WITH_SNAPSHOT_RETENTION:
+        minSnapshotsToKeep = Integer.valueOf(childNode.getChild(0).getText());
+        if (childNode.getChildren().size() > 1) {
+          String maxSnapshotAge = childNode.getChild(1).getText();
+          String timeUnitOfSnapshotsRetention = childNode.getChild(2).getText();
+          maxSnapshotAgeMs = TimeUnit.valueOf(timeUnitOfSnapshotsRetention.toUpperCase(Locale.ENGLISH)).toMillis(Long.valueOf(maxSnapshotAge));

Review Comment:
   use. ```Long.parseLong(maxSnapshotAge)``



##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    validateAlterTableType(table, AlterTableType.CREATEBRANCH, false);
+    if (!"ICEBERG".equalsIgnoreCase(table.getParameters().get("table_type"))) {

Review Comment:
   Change to 
   ```
       if (!HiveMetaHook.ICEBERG.equalsIgnoreCase(table.getParameters().get(HiveMetaHook.TABLE_TYPE))) {
   ```



##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    validateAlterTableType(table, AlterTableType.CREATEBRANCH, false);
+    if (!"ICEBERG".equalsIgnoreCase(table.getParameters().get("table_type"))) {
+      throw new SemanticException("Cannot perform ALTER CREATE BRANCH statement on non-iceberg table.");
+    }
+    inputs.add(new ReadEntity(table));
+
+    String branchName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long maxRefAgeMs = null;
+    Integer minSnapshotsToKeep = null;
+    Long maxSnapshotAgeMs = null;
+    for (int i = 1; i < command.getChildCount(); i++) {
+      ASTNode childNode = (ASTNode) command.getChild(i);
+      switch (childNode.getToken().getType()) {
+      case HiveParser.TOK_AS_OF_VERSION_BRANCH:
+        snapshotId = Long.valueOf(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_RETAIN:
+        String maxRefAge = childNode.getChild(0).getText();
+        String timeUnitOfBranchRetain = childNode.getChild(1).getText();
+        maxRefAgeMs = TimeUnit.valueOf(timeUnitOfBranchRetain.toUpperCase(Locale.ENGLISH)).toMillis(Long.valueOf(maxRefAge));

Review Comment:
   you need primitive long only, Long.valueOf does a parseLong to get a primitive and then converts to Long, and here you would be converting back to long, because toMillis to take primitive long only.
   
   Use ``Long.parseLong(maxRefAge)``



-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g:
##########
@@ -477,6 +478,34 @@ alterStatementSuffixExecute
     -> ^(TOK_ALTERTABLE_EXECUTE KW_SET_CURRENT_SNAPSHOT $snapshotParam)
     ;
 
+alterStatementSuffixCreateBranch
+@init { gParent.pushMsg("alter table create branch", state); }
+@after { gParent.popMsg(state); }
+    : KW_CREATE KW_BRANCH branchName=identifier snapshotIdOfBranch? branchRetain? retentionOfSnapshots?
+    -> ^(TOK_ALTERTABLE_CREATE_BRANCH $branchName snapshotIdOfBranch? branchRetain? retentionOfSnapshots?)
+    ;
+
+snapshotIdOfBranch
+@init { gParent.pushMsg("alter table create branch as of version", state); }

Review Comment:
   My original thought was to be consistent with spark-iceberg syntax. But it seems more reasonable to be sync with current hive` timet_travel` syntax.
   I will be change the syntax and add the syntax `FOR SYSTEM_TIME AS OF`.
   
   The new syntax will be as follows:
   
   ```
   ALTER TABLE tableName
   {CREATE BRANCH branchName [FOR SYSTEM_VERSION AS OF {snapshotId} | FOR SYSTEM_TIME AS OF {timestamp}]
   [RETAIN interval {DAYS | HOURS | MINUTES}]
   [WITH SNAPSHOT RETENTION {[num_snapshots SNAPSHOTS] [interval {DAYS | HOURS | MINUTES}]}]}]
   ```



-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   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=4216)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&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] zhangbutao closed pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

Posted by "zhangbutao (via GitHub)" <gi...@apache.org>.
zhangbutao closed pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation
URL: https://github.com/apache/hive/pull/4216


-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -676,6 +678,35 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
     }
   }
 
+  @Override
+  public void createBranchOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
+      AlterTableCreateBranchSpec createBranchSpec) {
+    TableDesc tableDesc = Utilities.getTableDesc(hmsTable);
+    Table icebergTable = IcebergTableUtil.getTable(conf, tableDesc.getProperties());
+
+    String branchName = createBranchSpec.getBranchName();
+    Optional.ofNullable(icebergTable.currentSnapshot()).orElseThrow(() -> new UnsupportedOperationException(
+        String.format("Cannot create branch %s on iceberg table %s.%s which has no snapshot",
+            branchName, hmsTable.getDbName(), hmsTable.getTableName())));
+    Long snapshotId = Optional.ofNullable(createBranchSpec.getSnapshotId())
+        .orElse(icebergTable.currentSnapshot().snapshotId());
+    LOG.info("Creating branch {} on iceberg table {}.{}", branchName, hmsTable.getDbName(),
+        hmsTable.getTableName());
+    ManageSnapshots manageSnapshots = icebergTable.manageSnapshots();
+    manageSnapshots.createBranch(branchName, snapshotId);

Review Comment:
   👍 



-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    validateAlterTableType(table, AlterTableType.CREATEBRANCH, false);
+    if (!"ICEBERG".equalsIgnoreCase(table.getParameters().get("table_type"))) {
+      throw new SemanticException("Cannot perform ALTER CREATE BRANCH statement on non-iceberg table.");
+    }
+    inputs.add(new ReadEntity(table));
+
+    String branchName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long maxRefAgeMs = null;
+    Integer minSnapshotsToKeep = null;
+    Long maxSnapshotAgeMs = null;
+    for (int i = 1; i < command.getChildCount(); i++) {
+      ASTNode childNode = (ASTNode) command.getChild(i);
+      switch (childNode.getToken().getType()) {
+      case HiveParser.TOK_AS_OF_VERSION_BRANCH:
+        snapshotId = Long.valueOf(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_RETAIN:
+        String maxRefAge = childNode.getChild(0).getText();
+        String timeUnitOfBranchRetain = childNode.getChild(1).getText();
+        maxRefAgeMs = TimeUnit.valueOf(timeUnitOfBranchRetain.toUpperCase(Locale.ENGLISH)).toMillis(Long.valueOf(maxRefAge));

Review Comment:
   fixed. Thx.



-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g:
##########
@@ -477,6 +478,34 @@ alterStatementSuffixExecute
     -> ^(TOK_ALTERTABLE_EXECUTE KW_SET_CURRENT_SNAPSHOT $snapshotParam)
     ;
 
+alterStatementSuffixCreateBranch
+@init { gParent.pushMsg("alter table create branch", state); }
+@after { gParent.popMsg(state); }
+    : KW_CREATE KW_BRANCH branchName=identifier snapshotIdOfBranch? branchRetain? retentionOfSnapshots?
+    -> ^(TOK_ALTERTABLE_CREATE_BRANCH $branchName snapshotIdOfBranch? branchRetain? retentionOfSnapshots?)
+    ;
+
+snapshotIdOfBranch
+@init { gParent.pushMsg("alter table create branch as of version", state); }

Review Comment:
   done



##########
parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g:
##########
@@ -477,6 +478,34 @@ alterStatementSuffixExecute
     -> ^(TOK_ALTERTABLE_EXECUTE KW_SET_CURRENT_SNAPSHOT $snapshotParam)
     ;
 
+alterStatementSuffixCreateBranch
+@init { gParent.pushMsg("alter table create branch", state); }
+@after { gParent.popMsg(state); }
+    : KW_CREATE KW_BRANCH branchName=identifier snapshotIdOfBranch? branchRetain? retentionOfSnapshots?
+    -> ^(TOK_ALTERTABLE_CREATE_BRANCH $branchName snapshotIdOfBranch? branchRetain? retentionOfSnapshots?)
+    ;
+
+snapshotIdOfBranch
+@init { gParent.pushMsg("alter table create branch as of version", state); }
+@after { gParent.popMsg(state); }
+    : KW_AS KW_OF KW_VERSION snapshotId=Number
+    -> ^(TOK_AS_OF_VERSION_BRANCH $snapshotId)

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] sonarcloud[bot] commented on pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   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=4216)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableCreateBranchSpec.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.parse;
+
+import com.google.common.base.MoreObjects;
+
+public class AlterTableCreateBranchSpec {

Review Comment:
   I think so, ExecuteOperationType was extended with those 3 operations:    
   ````
   ROLLBACK,
   EXPIRE_SNAPSHOT,
   SET_CURRENT_SNAPSHOT
   ````
   note, if you think that create/drop branch doesn't semantically fit this design or complicates it, feel free to create a a new abstraction    



-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.HiveMetaHook;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    validateAlterTableType(table, AlterTableType.CREATEBRANCH, false);
+    if (!HiveMetaHook.ICEBERG.equalsIgnoreCase(table.getParameters().get(HiveMetaHook.TABLE_TYPE))) {

Review Comment:
   Thanks for your tip. Added a helper method:  `DDLUtils::validateTableIsIceberg`



-- 
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 merged pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


-- 
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] zhangbutao commented on pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

Posted by "zhangbutao (via GitHub)" <gi...@apache.org>.
zhangbutao commented on PR #4216:
URL: https://github.com/apache/hive/pull/4216#issuecomment-1515700762

   https://github.com/apache/hive/pull/4252 Upgrade iceberg to 1.2.1 in order to better integrate branch&tag features.


-- 
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] zhangbutao commented on pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

Posted by "zhangbutao (via GitHub)" <gi...@apache.org>.
zhangbutao commented on PR #4216:
URL: https://github.com/apache/hive/pull/4216#issuecomment-1515606342

   @TuroczyX @ayushtkn yes, branch and tag are important feats in iceberg 1.2.0.  I have created a umbrella ticket https://issues.apache.org/jira/browse/HIVE-27233 to track the new feats about branch&tag.
   
   > The main thing to chase is not creating a branch. But to insert into those branches.
   
   I absolutely agree. I am exploring how to achieve this in hive. Next maybe we shoud consider upgrade iceberg to latest version 1.2.1.  In addition, we can refer to this PR to implement the sql syntax for` insert into branch`


-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g:
##########
@@ -477,6 +478,34 @@ alterStatementSuffixExecute
     -> ^(TOK_ALTERTABLE_EXECUTE KW_SET_CURRENT_SNAPSHOT $snapshotParam)
     ;
 
+alterStatementSuffixCreateBranch
+@init { gParent.pushMsg("alter table create branch", state); }
+@after { gParent.popMsg(state); }
+    : KW_CREATE KW_BRANCH branchName=identifier snapshotIdOfBranch? branchRetain? retentionOfSnapshots?
+    -> ^(TOK_ALTERTABLE_CREATE_BRANCH $branchName snapshotIdOfBranch? branchRetain? retentionOfSnapshots?)
+    ;
+
+snapshotIdOfBranch
+@init { gParent.pushMsg("alter table create branch as of version", state); }
+@after { gParent.popMsg(state); }
+    : KW_AS KW_OF KW_VERSION snapshotId=Number
+    -> ^(TOK_AS_OF_VERSION_BRANCH $snapshotId)

Review Comment:
   sure, as mentioned above, we can reuse `TOK_AS_OF_VERSION` if we use `FOR SYSTEM_VERSION AS OF` instead of `AS OF VERSION`.



-- 
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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.HiveMetaHook;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    validateAlterTableType(table, AlterTableType.CREATEBRANCH, false);
+    if (!HiveMetaHook.ICEBERG.equalsIgnoreCase(table.getParameters().get(HiveMetaHook.TABLE_TYPE))) {

Review Comment:
   Sorry, can you elaborate on where this logic shoud move into? This is just a simple iceberg check  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] zhangbutao commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableCreateBranchSpec.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.parse;
+
+import com.google.common.base.MoreObjects;
+
+public class AlterTableCreateBranchSpec {

Review Comment:
   That's what I thought before. Also I want to reuse method: `HiveIcebergStorageHandler::executeOperation` :
   https://github.com/apache/hive/blob/e413b445e95e7c8beb4e43f0f6b08a98934e1c2c/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java#L645
   
   But it  seems to be that these codes are for  syntax `alter table execute`.  Do you think we can reuse them for `alter table create branch
   ` or `alter table drop branch`?



-- 
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] TuroczyX commented on a diff in pull request #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)

Review Comment:
   I know other does not use it, in the code, but in defensive code, when you pass a reference it should be checked if it not a null in the beginning of the method. Just to avoid any unpredicted null reference exception. It is pretty optional, because other commands does not do it at all. 
   
   Btw nice code. 



-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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

   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=4216)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4216&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=4216&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=4216&resolved=false&types=CODE_SMELL) [8 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4216&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=4216&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=4216&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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.HiveMetaHook;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    validateAlterTableType(table, AlterTableType.CREATEBRANCH, false);
+    if (!HiveMetaHook.ICEBERG.equalsIgnoreCase(table.getParameters().get(HiveMetaHook.TABLE_TYPE))) {

Review Comment:
   would be great to have a short helper method like `isIceberg()` instead of this long construct (used in multiple places).
   Found  similar thing in `HiveTableOperations#validateTableIsIceberg(Table table, String fullName)`
     



-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.table.branch.create;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.HiveMetaHook;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AlterTableCreateBranchSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    validateAlterTableType(table, AlterTableType.CREATEBRANCH, false);
+    if (!HiveMetaHook.ICEBERG.equalsIgnoreCase(table.getParameters().get(HiveMetaHook.TABLE_TYPE))) {

Review Comment:
   would be great to have a short helper method like` isIceberg()` instead of this long construct (used in multiple places).
   Found  similar thing in `HiveTableOperations#validateTableIsIceberg(Table table, String fullName)`
     



-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableCreateBranchSpec.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.parse;
+
+import com.google.common.base.MoreObjects;
+
+public class AlterTableCreateBranchSpec {

Review Comment:
   👍 



-- 
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 #4216: HIVE-27234: Iceberg: CREATE BRANCH SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLUtils.java:
##########
@@ -217,4 +218,11 @@ public static void setColumnsAndStorePartitionTransformSpecOfTable(
       partCols.ifPresent(tbl::setPartCols);
     }
   }
+
+  public static void validateTableIsIceberg(org.apache.hadoop.hive.ql.metadata.Table table)

Review Comment:
   doesn't it duplicate `HiveTableOperations.validateTableIsIceberg`, could we keep just one version of it 
   ````
     static void validateTableIsIceberg(Table table, String fullName) {
       String tableType = table.getParameters().get(TABLE_TYPE_PROP);
       NoSuchIcebergTableException.check(tableType != null && tableType.equalsIgnoreCase(ICEBERG_TABLE_TYPE_VALUE),
           "Not an iceberg table: %s (type=%s)", fullName, tableType);
     }
   ````



-- 
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