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/05/31 02:42:23 UTC

[GitHub] [hive] zhangbutao opened a new pull request, #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

   <!--
   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.
   -->
   Added support for creating iceberg tag.
   
   ```
   ALTER TABLE tableName
   CREATE TAG tagName [FOR SYSTEM_VERSION AS OF {snapshotId} | FOR SYSTEM_TIME AS OF {timestamp}]
   [RETAIN interval {DAYS | HOURS | MINUTES}]
   ```
   
   ### 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'.
   -->
   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.
   -->
   UT & qtest
   
   mvn test -Dtest.output.overwrite -Dtest=TestIcebergCliDriver -Dqfile=alter_table_create_tag.q -pl itests/qtest-iceberg -Piceberg -Pitests -Drat.skip
   


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

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

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


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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;
+
+import java.time.ZoneId;
+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.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.branch.create.AlterTableCreateBranchDesc;
+import org.apache.hadoop.hive.ql.ddl.table.tag.create.AlterTableCreateTagDesc;
+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.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_BRANCH;
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_TAG;
+
+public abstract class AlterTableCreateMetaRefAnalyzer extends AbstractAlterTableAnalyzer {
+
+  private static AbstractAlterTableDesc alterTableDesc;
+
+  public AlterTableCreateMetaRefAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    DDLUtils.validateTableIsIceberg(table);
+    inputs.add(new ReadEntity(table));
+
+    String metaRefName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long asOfTime = null;
+    Long maxRefAgeMs = null;
+    Integer minSnapshotsToKeep = null;
+    Long maxSnapshotAgeMs = null;
+    AlterTableType alterTableType =
+        command.getType() == HiveParser.TOK_ALTERTABLE_CREATE_BRANCH ? AlterTableType.CREATE_BRANCH : AlterTableType.CREATE_TAG;
+    for (int i = 1; i < command.getChildCount(); i++) {
+      ASTNode childNode = (ASTNode) command.getChild(i);
+      switch (childNode.getToken().getType()) {
+      case HiveParser.TOK_AS_OF_VERSION:
+        snapshotId = Long.parseLong(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_AS_OF_TIME:
+        ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() :
+            SessionState.get().getConf().getLocalTimeZone();
+        TimestampTZ ts = TimestampTZUtil.parse(stripQuotes(childNode.getChild(0).getText()), timeZone);
+        asOfTime = ts.toEpochMilli();
+        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.parseLong(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.parseLong(maxSnapshotAge));
+        }
+        break;
+      default:
+        throw new SemanticException("Unrecognized token in ALTER " + alterTableType.getName() + " statement");
+      }
+    }
+
+    if (alterTableType == AlterTableType.CREATE_BRANCH) {

Review Comment:
   why not keep it as abstract and create TAG and BRANCH implementations?



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

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

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableMetaRefSpec.java:
##########
@@ -48,17 +57,17 @@ public String toString() {
         .add("operationParams", operationParams).toString();
   }
 
-  public static class CreateBranchSpec {
+  public static class CreateMetaRefSpec {

Review Comment:
   in iceberg, it's called SnapshotRef



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/snapshotref/AlterTableCreateSnapshotRefAnalyzer.java:
##########
@@ -16,50 +16,48 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.branch.create;
+package org.apache.hadoop.hive.ql.ddl.table.snapshotref;
 
 import java.time.ZoneId;
 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.common.type.TimestampTZ;
 import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.QueryState;
-import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
 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.AbstractAlterTableDesc;
 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.AlterTableBranchSpec;
+import org.apache.hadoop.hive.ql.parse.AlterTableSnapshotRefSpec;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
-import static org.apache.hadoop.hive.ql.parse.AlterTableBranchSpec.AlterBranchOperationType.CREATE_BRANCH;
-
-@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
-public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+public abstract class AlterTableCreateSnapshotRefAnalyzer extends AbstractAlterTableAnalyzer {
+  protected static AbstractAlterTableDesc alterTableDesc;
+  protected static AlterTableType alterTableType;

Review Comment:
   remove `static`



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/snapshotref/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.snapshotref.branch.create;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.table.snapshotref.AlterTableCreateSnapshotRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+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 AlterTableCreateSnapshotRefAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+    super.alterTableType = AlterTableType.CREATE_BRANCH;

Review Comment:
   Removed



##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/snapshotref/tag/create/AlterTableCreateTagAnalyzer.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.snapshotref.tag.create;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.table.snapshotref.AlterTableCreateSnapshotRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_TAG)
+public class AlterTableCreateTagAnalyzer extends AlterTableCreateSnapshotRefAnalyzer {
+
+  public AlterTableCreateTagAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+    super.alterTableType = AlterTableType.CREATE_TAG;

Review Comment:
   Removed



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -57,90 +59,55 @@ protected void analyzeCommand(TableName tableName, Map<String, String> partition
     DDLUtils.validateTableIsIceberg(table);
     inputs.add(new ReadEntity(table));
     validateAlterTableType(table, alterTableType, false);
-    AlterTableTypeReq alterTableTypeReq = new AlterTableTypeReq();
 
     String metaRefName = command.getChild(0).getText();
-    alterTableTypeReq.tableName = tableName;
-    alterTableTypeReq.metaRefName = metaRefName;
     Long snapshotId = null;
     Long asOfTime = null;
     Long maxRefAgeMs = null;
     Integer minSnapshotsToKeep = null;
     Long maxSnapshotAgeMs = null;
     AlterTableType alterTableType = command.getType()

Review Comment:
   why do you need this? alterTableType is set in children



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

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

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableMetaRefSpec.java:
##########
@@ -48,17 +57,17 @@ public String toString() {
         .add("operationParams", operationParams).toString();
   }
 
-  public static class CreateBranchSpec {
+  public static class CreateMetaRefSpec {

Review Comment:
   > in iceberg project, it's called SnapshotRef, MetaRef is too wide
   
   Renamed MetaRef to SnapshotRef.
   
   > could we create also builders for tag and branch similar to SnapshotRef.tagBuilder(snapshotId).build();
   
   Maybe i misunderstand what your meaning, and i think no need to create builder like iceberg project. 
   
   `Builder `in iceberg SnapshotRef is used to create multiple constructor. e.g.  `SnapshotRef.tagBuilder(snapshotId).build()` is used to create a tag without any properties(maxRefAgeMs, minSnapshotsToKeep), `SnapshotRef.tagBuilder(1L).maxRefAgeMs(10L).build()` is used to create a tag with property `maxRefAgeMs`. 
   
   But what we need here is just only one constructor with all properties(maxRefAgeMs, minSnapshotsToKeep..), and pass these properties to the final branch& tag creation stage,` IcebergBranchExec.java` &  `IcebergTagExec.java`, and set these properties when invoking iceberg native api to create banch & tag.



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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;
+
+import java.time.ZoneId;
+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.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.branch.create.AlterTableCreateBranchDesc;
+import org.apache.hadoop.hive.ql.ddl.table.tag.create.AlterTableCreateTagDesc;
+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.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_BRANCH;
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_TAG;
+
+public abstract class AlterTableCreateMetaRefAnalyzer extends AbstractAlterTableAnalyzer {
+
+  private static AbstractAlterTableDesc alterTableDesc;
+
+  public AlterTableCreateMetaRefAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    DDLUtils.validateTableIsIceberg(table);
+    inputs.add(new ReadEntity(table));
+
+    String metaRefName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long asOfTime = null;
+    Long maxRefAgeMs = null;
+    Integer minSnapshotsToKeep = null;
+    Long maxSnapshotAgeMs = null;
+    AlterTableType alterTableType =
+        command.getType() == HiveParser.TOK_ALTERTABLE_CREATE_BRANCH ? AlterTableType.CREATE_BRANCH : AlterTableType.CREATE_TAG;
+    for (int i = 1; i < command.getChildCount(); i++) {
+      ASTNode childNode = (ASTNode) command.getChild(i);
+      switch (childNode.getToken().getType()) {
+      case HiveParser.TOK_AS_OF_VERSION:
+        snapshotId = Long.parseLong(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_AS_OF_TIME:
+        ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() :
+            SessionState.get().getConf().getLocalTimeZone();
+        TimestampTZ ts = TimestampTZUtil.parse(stripQuotes(childNode.getChild(0).getText()), timeZone);
+        asOfTime = ts.toEpochMilli();
+        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.parseLong(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.parseLong(maxSnapshotAge));
+        }
+        break;
+      default:
+        throw new SemanticException("Unrecognized token in ALTER " + alterTableType.getName() + " statement");
+      }
+    }
+
+    if (alterTableType == AlterTableType.CREATE_BRANCH) {

Review Comment:
   sorry, I don't get it
   ````
   class public abstract class AlterTableCreateMetaRefAnalyzer extends AbstractAlterTableAnalyzer {
       abstract AbstractAlterTableDesc  getAlterTableDesc(AlterTableTypeReq req)
   
       class AlterTableTypeReq{
           snapshotId, asOfTime, maxRefAgeMs, minSnapshotsToKeep, maxSnapshotAgeMs
       }
   }
   
   class AlterTableCreateBranchAnalyzer extends AlterTableCreateMetaRefAnalyzer {
       ...
       public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
           super(queryState);
           alterTableType = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH;
       }
    ....
       AbstractAlterTableDesc  getAlterTableDesc(AlterTableTypeReq req){
            AlterTableMetaRefSpec.CreateTagSpec createTagspec =
             new AlterTableMetaRefSpec.CreateTagSpec(req.getMetaRefName(), req.getSnapshotId(), req.getAsOfTime(), req.getMaxRefAgeMs());
         AlterTableMetaRefSpec<AlterTableMetaRefSpec.CreateTagSpec> alterTableTagSpec
             = new AlterTableMetaRefSpec(CREATE_TAG, createTagspec);
         return new AlterTableCreateTagDesc(tableName, alterTableTagSpec);
   }
   ````



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -763,6 +764,27 @@ public void alterTableBranchOperation(org.apache.hadoop.hive.ql.metadata.Table h
 
   }
 
+  @Override
+  public void alterTableTagOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
+      AlterTableTagSpec alterTagSpec) {
+    TableDesc tableDesc = Utilities.getTableDesc(hmsTable);
+    Table icebergTable = IcebergTableUtil.getTable(conf, tableDesc.getProperties());
+    Optional.ofNullable(icebergTable.currentSnapshot()).orElseThrow(() ->
+        new UnsupportedOperationException(String.format("Cannot alter tag on iceberg table" +
+            " %s.%s which has no snapshot", hmsTable.getDbName(), hmsTable.getTableName())));
+
+    switch (alterTagSpec.getOperationType()) {

Review Comment:
   could we make it reusable and extract common part in `alterTableOperation(hmsTable, alterTag/BranchSpec.getOperationType())`?



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

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

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

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

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


##########
ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java:
##########
@@ -6724,6 +6725,15 @@ public void alterTableBranchOperation(Table table, AlterTableBranchSpec createBr
     }
   }
 
+  public void alterTableTagOperation(Table table, AlterTableTagSpec createTagSpec) throws HiveException {
+    try {
+      HiveStorageHandler storageHandler = createStorageHandler(table.getTTable());

Review Comment:
   gotcha, we can ignore that in the current PR, but I think that should be refactored to a service that operates on the table object



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableTagSpec.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 AlterTableTagSpec <T>{

Review Comment:
   Sorry for my less rigorous coding. 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] zhangbutao commented on pull request #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

   > hi @zhangbutao, sorry for the review delay, added a few comments
   
   @deniskuzZ Never mind. I really appreciate your patient and detail review. Thanks a lot!


-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/tag/create/AlterTableCreateTagAnalyzer.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.metaref.tag.create;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableDesc;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefDesc;
+import org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_TAG;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_TAG)
+public class AlterTableCreateTagAnalyzer extends AlterTableCreateMetaRefAnalyzer {
+
+  public AlterTableCreateTagAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+    super.alterTableType = AlterTableType.CREATE_TAG;
+  }
+
+  @Override
+  protected AbstractAlterTableDesc getAlterTableDesc(AlterTableTypeReq req) throws SemanticException {
+    AlterTableMetaRefSpec.CreateMetaRefSpec createTagspec =

Review Comment:
   createTag**S**pec



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

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

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/snapshotref/AlterTableCreateSnapshotRefAnalyzer.java:
##########
@@ -16,50 +16,48 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.branch.create;
+package org.apache.hadoop.hive.ql.ddl.table.snapshotref;
 
 import java.time.ZoneId;
 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.common.type.TimestampTZ;
 import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.QueryState;
-import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
 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.AbstractAlterTableDesc;
 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.AlterTableBranchSpec;
+import org.apache.hadoop.hive.ql.parse.AlterTableSnapshotRefSpec;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
-import static org.apache.hadoop.hive.ql.parse.AlterTableBranchSpec.AlterBranchOperationType.CREATE_BRANCH;
-
-@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
-public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+public abstract class AlterTableCreateSnapshotRefAnalyzer extends AbstractAlterTableAnalyzer {
+  protected static AbstractAlterTableDesc alterTableDesc;
+  protected static AlterTableType alterTableType;

Review Comment:
   Removed



##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/snapshotref/AlterTableCreateSnapshotRefAnalyzer.java:
##########
@@ -16,50 +16,48 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.branch.create;
+package org.apache.hadoop.hive.ql.ddl.table.snapshotref;
 
 import java.time.ZoneId;
 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.common.type.TimestampTZ;
 import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.QueryState;
-import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
 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.AbstractAlterTableDesc;
 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.AlterTableBranchSpec;
+import org.apache.hadoop.hive.ql.parse.AlterTableSnapshotRefSpec;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
-import static org.apache.hadoop.hive.ql.parse.AlterTableBranchSpec.AlterBranchOperationType.CREATE_BRANCH;
-
-@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
-public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+public abstract class AlterTableCreateSnapshotRefAnalyzer extends AbstractAlterTableAnalyzer {
+  protected static AbstractAlterTableDesc alterTableDesc;

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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

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

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

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

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

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

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

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

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

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

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


##########
ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java:
##########
@@ -6724,6 +6725,15 @@ public void alterTableBranchOperation(Table table, AlterTableBranchSpec createBr
     }
   }
 
+  public void alterTableTagOperation(Table table, AlterTableTagSpec createTagSpec) throws HiveException {
+    try {
+      HiveStorageHandler storageHandler = createStorageHandler(table.getTTable());

Review Comment:
   Good suggestion. I tried to use Caffeine cache to get storage handler, but it failed unfortunately.
   Here it is what i have done:
   
   ```
   // Define storage handler Caffine cache.
     private static final Cache<String, HiveStorageHandler> storageHandlerCache = Caffeine.newBuilder()
         .expireAfterAccess(10, TimeUnit.MINUTES).build();
   
   // Get storage handler from cache.
     private HiveStorageHandler getCachedStorageHandler(org.apache.hadoop.hive.metastore.api.Table tbl) {
      return storageHandlerCache.get(new Table(tbl).getFullTableName().toString(), k -> {
         try {
           // create a new storage handler if it is not in cache.
           return createStorageHandler(tbl);
         } catch (MetaException ex) {
           LOG.error("Failed to get storage handler", ex);
           throw new RuntimeException("Failed to get storage handler:  " + ex.getMessage());
         }
       });
     }
   ```
   
   And then i replaced the invoked method `HiveIcebergStorageHandler::createStorageHandler` with `HiveIcebergStorageHandler::getCachedStorageHandler`, like this:
   ```
   @@ -5699,7 +5704,7 @@ private IMetaStoreClient createMetaStoreClient(boolean allowEmbedded) throws Met
          public HiveMetaHook getHook(
                  org.apache.hadoop.hive.metastore.api.Table tbl)
                  throws MetaException {
   -        HiveStorageHandler storageHandler = createStorageHandler(tbl);
   +        HiveStorageHandler storageHandler = getCachedStorageHandler(tbl);
            return storageHandler == null ? null : storageHandler.getMetaHook();
          }
        };
   ```
   
   Then I analyzed the reason for the cached storage handler. It is mostly because `HiveIcebergStorageHandler ` has a class-level `conf ` which can be used to load a iceberg table and the `conf`  storage the latest `METADATA `location, and if the iceberg table is modified, we should use the modified `conf ` to get a new  `METADATA `location.  If we always use the cached storageHandler, we maybe get a old iceberg table whose latest snapshot is fake.
   
   // Here it is a class-level `conf`
   https://github.com/apache/hive/blob/e0bf9609585d671b936c95ef5ccd4dcadc9be71a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java#L192
   
   Here we use method `IcebergTableUtil.getTable(conf properties)` and `conf  `to get the iceberg table:
   https://github.com/apache/hive/blob/e0bf9609585d671b936c95ef5ccd4dcadc9be71a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java#L751
   
   // Here we use the `conf `to get the latest `METADATA `location
   https://github.com/apache/hive/blob/e0bf9609585d671b936c95ef5ccd4dcadc9be71a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java#L88
   



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTagExec.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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 org.apache.hadoop.hive.ql.parse.AlterTableTagSpec;
+import org.apache.iceberg.ManageSnapshots;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class IcebergTagExec {
+
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergTagExec.class);
+
+  private IcebergTagExec() {
+  }
+
+  public static void createTag(Table table, AlterTableTagSpec.CreateTagSpec createBranchSpec) {

Review Comment:
   The main difference is that `create tag` operation needs less fields, e.g. `minSnapshotsToKeep `and `maxSnapshotAgeMs`. But as the final code execution stage, i want to keep the two separate to make code more cleaner.
   
   In this commit https://github.com/apache/hive/pull/4372/commits/6ef586f028f1d14a7cfbb21fef5b881d48d694a8, I also try my best to reuse code. Especilly, i reused one method `Hive::alterTableMetaRefOperation` & `HiveStorageHandler::alterTableMetaRefOperation` to operate tag&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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/tag/create/AlterTableCreateTagAnalyzer.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.tag.create;
+
+import java.util.Map;
+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.table.AlterTableCreateMetaRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_TAG)
+public class AlterTableCreateTagAnalyzer extends AlterTableCreateMetaRefAnalyzer {
+  public AlterTableCreateTagAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)

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] deniskuzZ commented on a diff in pull request #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -16,92 +16,159 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.branch.create;
+package org.apache.hadoop.hive.ql.ddl.table.metaref;
 
 import java.time.ZoneId;
 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.common.type.TimestampTZ;
 import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.QueryState;
-import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
 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.AbstractAlterTableDesc;
 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.AlterTableBranchSpec;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
-import static org.apache.hadoop.hive.ql.parse.AlterTableBranchSpec.AlterBranchOperationType.CREATE_BRANCH;
-
-@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
-public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+public abstract class AlterTableCreateMetaRefAnalyzer extends AbstractAlterTableAnalyzer {
+  protected static AbstractAlterTableDesc alterTableDesc;
+  protected static AlterTableType alterTableType;
+  protected abstract AbstractAlterTableDesc getAlterTableDesc(AlterTableTypeReq alterTableTypeReq)
+      throws SemanticException;
 
-  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+  public AlterTableCreateMetaRefAnalyzer(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.CREATE_BRANCH, false);
     DDLUtils.validateTableIsIceberg(table);
     inputs.add(new ReadEntity(table));
+    validateAlterTableType(table, alterTableType, false);
+    AlterTableTypeReq alterTableTypeReq = new AlterTableTypeReq();
 
-    String branchName = command.getChild(0).getText();
+    String metaRefName = command.getChild(0).getText();
+    alterTableTypeReq.setTableName(tableName);
+    alterTableTypeReq.setMetaRefName(metaRefName);
     Long snapshotId = null;
     Long asOfTime = null;
     Long maxRefAgeMs = null;
     Integer minSnapshotsToKeep = null;
     Long maxSnapshotAgeMs = null;
+    AlterTableType alterTableType = command.getType()
+        == HiveParser.TOK_ALTERTABLE_CREATE_BRANCH ? AlterTableType.CREATE_BRANCH : AlterTableType.CREATE_TAG;
     for (int i = 1; i < command.getChildCount(); i++) {
       ASTNode childNode = (ASTNode) command.getChild(i);
       switch (childNode.getToken().getType()) {
       case HiveParser.TOK_AS_OF_VERSION:
         snapshotId = Long.parseLong(childNode.getChild(0).getText());
+        alterTableTypeReq.setSnapshotId(snapshotId);
         break;
       case HiveParser.TOK_AS_OF_TIME:
         ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() :
             SessionState.get().getConf().getLocalTimeZone();
         TimestampTZ ts = TimestampTZUtil.parse(stripQuotes(childNode.getChild(0).getText()), timeZone);
         asOfTime = ts.toEpochMilli();
+        alterTableTypeReq.setAsOfTime(asOfTime);
         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.parseLong(maxRefAge));
+        maxRefAgeMs =
+            TimeUnit.valueOf(timeUnitOfBranchRetain.toUpperCase(Locale.ENGLISH)).toMillis(Long.parseLong(maxRefAge));
+        alterTableTypeReq.setMaxRefAgeMs(maxRefAgeMs);
         break;
       case HiveParser.TOK_WITH_SNAPSHOT_RETENTION:
         minSnapshotsToKeep = Integer.valueOf(childNode.getChild(0).getText());
+        alterTableTypeReq.setMinSnapshotsToKeep(minSnapshotsToKeep);
         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.parseLong(maxSnapshotAge));
+          alterTableTypeReq.setMaxSnapshotAgeMs(maxSnapshotAgeMs);
         }
         break;
       default:
-        throw new SemanticException("Unrecognized token in ALTER CREATE BRANCH statement");
+        throw new SemanticException("Unrecognized token in ALTER " + alterTableType.getName() + " statement");
       }
     }
+    alterTableDesc = getAlterTableDesc(alterTableTypeReq);
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterTableDesc)));
+  }
+
+  public class AlterTableTypeReq{

Review Comment:
   could we mark it as a package-private? also not sure if you need to have public setters, you can directly access the properties.



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableMetaRefSpec.java:
##########
@@ -20,21 +20,30 @@
 
 import com.google.common.base.MoreObjects;
 
-public class AlterTableBranchSpec<T> {
+public class AlterTableMetaRefSpec<T> {

Review Comment:
   why do you need `AlterTableMetaRefSpec` wrapper at all, can't we just use parametrized `CreateMetaRefSpec<T>`
   ````
   CreateMetaRefSpec<MetaRef.BRANCH> createBranchSpec = new CreateMetaRefSpec<>(..)
   ````



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableMetaRefSpec.java:
##########
@@ -20,21 +20,30 @@
 
 import com.google.common.base.MoreObjects;
 
-public class AlterTableBranchSpec<T> {
+public class AlterTableMetaRefSpec<T> {

Review Comment:
   I want to use `AlterTableMetaRefSpec `to wrap multiple branch/tag operation, e.g. `CreateMetaRefSpec `as well as the following task `ReplaceMetaRefSpec`, and maybe other operations.
   I will remove the wrapper if these operation have same fields in the future work and will change this as you sugested.



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/snapshotref/AlterTableCreateSnapshotRefAnalyzer.java:
##########
@@ -16,50 +16,48 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.branch.create;
+package org.apache.hadoop.hive.ql.ddl.table.snapshotref;
 
 import java.time.ZoneId;
 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.common.type.TimestampTZ;
 import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.QueryState;
-import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
 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.AbstractAlterTableDesc;
 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.AlterTableBranchSpec;
+import org.apache.hadoop.hive.ql.parse.AlterTableSnapshotRefSpec;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
-import static org.apache.hadoop.hive.ql.parse.AlterTableBranchSpec.AlterBranchOperationType.CREATE_BRANCH;
-
-@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
-public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+public abstract class AlterTableCreateSnapshotRefAnalyzer extends AbstractAlterTableAnalyzer {
+  protected static AbstractAlterTableDesc alterTableDesc;

Review Comment:
   remove `protected` & `static`, add `private`



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/tag/create/AlterTableCreateTagAnalyzer.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.tag.create;
+
+import java.time.ZoneId;
+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.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+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.AlterTableTagSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableTagSpec.AlterTagOperationType.CREATE_TAG;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_TAG)
+public class AlterTableCreateTagAnalyzer extends AbstractAlterTableAnalyzer {
+  public AlterTableCreateTagAnalyzer(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.CREATE_TAG, false);
+    DDLUtils.validateTableIsIceberg(table);
+    inputs.add(new ReadEntity(table));
+
+    String tagName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long asOfTime = null;
+    Long maxRefAgeMs = 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:
+        snapshotId = Long.parseLong(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_AS_OF_TIME:
+        ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() :
+            SessionState.get().getConf().getLocalTimeZone();
+        TimestampTZ ts = TimestampTZUtil.parse(stripQuotes(childNode.getChild(0).getText()), timeZone);
+        asOfTime = ts.toEpochMilli();
+        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.parseLong(maxRefAge));
+        break;
+      default:
+        throw new SemanticException("Unrecognized token in ALTER CREATE TAG statement");
+      }
+    }
+
+    AlterTableTagSpec.CreateTagSpec createTagspec =
+        new AlterTableTagSpec.CreateTagSpec(tagName, snapshotId, asOfTime, maxRefAgeMs);
+    AlterTableTagSpec alterTableTagSpec = new AlterTableTagSpec(CREATE_TAG, createTagspec);
+    AlterTableCreateTagDesc desc = new AlterTableCreateTagDesc(tableName, alterTableTagSpec);

Review Comment:
   I create a new class `ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AlterTableCreateMetaRefAnalyzer.java` to make code reusable.



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableTagSpec.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 AlterTableTagSpec <T>{

Review Comment:
   ok, makes sense, but I still do not see where type information is being used
   https://github.com/apache/hive/pull/4372/files#diff-eac7a1ce623e75ae7789cba72422c240ec9d4f877db53280251a6ad9816e8bf5R110 uses a diamond operator, but declared var is missing the generic type. 



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

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

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


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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;
+
+import java.time.ZoneId;
+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.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.branch.create.AlterTableCreateBranchDesc;
+import org.apache.hadoop.hive.ql.ddl.table.tag.create.AlterTableCreateTagDesc;
+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.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_BRANCH;
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_TAG;
+
+public abstract class AlterTableCreateMetaRefAnalyzer extends AbstractAlterTableAnalyzer {
+
+  private static AbstractAlterTableDesc alterTableDesc;
+
+  public AlterTableCreateMetaRefAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    DDLUtils.validateTableIsIceberg(table);
+    inputs.add(new ReadEntity(table));
+
+    String metaRefName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long asOfTime = null;
+    Long maxRefAgeMs = null;
+    Integer minSnapshotsToKeep = null;
+    Long maxSnapshotAgeMs = null;
+    AlterTableType alterTableType =
+        command.getType() == HiveParser.TOK_ALTERTABLE_CREATE_BRANCH ? AlterTableType.CREATE_BRANCH : AlterTableType.CREATE_TAG;
+    for (int i = 1; i < command.getChildCount(); i++) {
+      ASTNode childNode = (ASTNode) command.getChild(i);
+      switch (childNode.getToken().getType()) {
+      case HiveParser.TOK_AS_OF_VERSION:
+        snapshotId = Long.parseLong(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_AS_OF_TIME:
+        ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() :
+            SessionState.get().getConf().getLocalTimeZone();
+        TimestampTZ ts = TimestampTZUtil.parse(stripQuotes(childNode.getChild(0).getText()), timeZone);
+        asOfTime = ts.toEpochMilli();
+        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.parseLong(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.parseLong(maxSnapshotAge));
+        }
+        break;
+      default:
+        throw new SemanticException("Unrecognized token in ALTER " + alterTableType.getName() + " statement");
+      }
+    }
+
+    if (alterTableType == AlterTableType.CREATE_BRANCH) {

Review Comment:
   sorry, I don't get it
   ````
   abstract class AlterTableCreateMetaRefAnalyzer extends AbstractAlterTableAnalyzer {
       protected final AbstractAlterTableDesc alterTableDesc;
   
       abstract AbstractAlterTableDesc getAlterTableDesc(AlterTableTypeReq req)
   
       class AlterTableTypeReq{
           snapshotId, asOfTime, maxRefAgeMs, minSnapshotsToKeep, maxSnapshotAgeMs
       }
   
       @Override
     protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
         throws SemanticException {
            Table table = getTable(tableName);
           validateAlterTableType(table, alterTableDesc, false);
            .....
            AlterTableTypeReq req = AlterTableTypeReq();
            req.setXXX
            ...
            rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), getAlterTableDesc(req))));
     }
   }
   
   class AlterTableCreateBranchAnalyzer extends AlterTableCreateMetaRefAnalyzer {
       ...
       public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
           super(queryState);
           alterTableType = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH;
       }
    ....
       AbstractAlterTableDesc  getAlterTableDesc(AlterTableTypeReq req){
            AlterTableMetaRefSpec.CreateTagSpec createTagspec =
             new AlterTableMetaRefSpec.CreateTagSpec(req.getMetaRefName(), req.getSnapshotId(), req.getAsOfTime(), req.getMaxRefAgeMs());
         AlterTableMetaRefSpec<AlterTableMetaRefSpec.CreateTagSpec> alterTableTagSpec
             = new AlterTableMetaRefSpec(CREATE_TAG, createTagspec);
         return new AlterTableCreateTagDesc(tableName, alterTableTagSpec);
   }
   ````



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/tag/create/AlterTableCreateTagAnalyzer.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.tag.create;
+
+import java.util.Map;
+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.table.AlterTableCreateMetaRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_TAG)
+public class AlterTableCreateTagAnalyzer extends AlterTableCreateMetaRefAnalyzer {
+  public AlterTableCreateTagAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)

Review Comment:
   no need to overwrite, see the above snippet. 



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -57,90 +59,55 @@ protected void analyzeCommand(TableName tableName, Map<String, String> partition
     DDLUtils.validateTableIsIceberg(table);
     inputs.add(new ReadEntity(table));
     validateAlterTableType(table, alterTableType, false);
-    AlterTableTypeReq alterTableTypeReq = new AlterTableTypeReq();
 
     String metaRefName = command.getChild(0).getText();
-    alterTableTypeReq.tableName = tableName;
-    alterTableTypeReq.metaRefName = metaRefName;
     Long snapshotId = null;
     Long asOfTime = null;
     Long maxRefAgeMs = null;
     Integer minSnapshotsToKeep = null;
     Long maxSnapshotAgeMs = null;
     AlterTableType alterTableType = command.getType()

Review Comment:
   why do you need this? alterTableType is set in concrete implementation (branch & tag)



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.metaref.branch.create;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableDesc;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefDesc;
+import org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_BRANCH;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AlterTableCreateMetaRefAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+    super.alterTableType = AlterTableType.CREATE_BRANCH;
+  }
+
+  @Override
+  protected AbstractAlterTableDesc getAlterTableDesc(AlterTableCreateMetaRefAnalyzer.AlterTableTypeReq req)

Review Comment:
   since you declared a common class for both branch and tag (i.e. `CreateMetaRefSpec`) looks like we do not need to have custom getAlterTableDesc anymore, just supply a proper `alterTableType`



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java:
##########
@@ -6724,6 +6725,15 @@ public void alterTableBranchOperation(Table table, AlterTableBranchSpec createBr
     }
   }
 
+  public void alterTableTagOperation(Table table, AlterTableTagSpec createTagSpec) throws HiveException {
+    try {
+      HiveStorageHandler storageHandler = createStorageHandler(table.getTTable());

Review Comment:
   why do we need to create new storageHandler each time?



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

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

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableTagSpec.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 AlterTableTagSpec <T>{

Review Comment:
   why do you need a generic type here? i don't see it used anywhere (same for AlterTableBranchSpec)



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

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

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

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

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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;
+
+import java.time.ZoneId;
+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.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.branch.create.AlterTableCreateBranchDesc;
+import org.apache.hadoop.hive.ql.ddl.table.tag.create.AlterTableCreateTagDesc;
+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.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_BRANCH;
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_TAG;
+
+public abstract class AlterTableCreateMetaRefAnalyzer extends AbstractAlterTableAnalyzer {
+
+  private static AbstractAlterTableDesc alterTableDesc;
+
+  public AlterTableCreateMetaRefAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    DDLUtils.validateTableIsIceberg(table);
+    inputs.add(new ReadEntity(table));
+
+    String metaRefName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long asOfTime = null;
+    Long maxRefAgeMs = null;
+    Integer minSnapshotsToKeep = null;
+    Long maxSnapshotAgeMs = null;
+    AlterTableType alterTableType =
+        command.getType() == HiveParser.TOK_ALTERTABLE_CREATE_BRANCH ? AlterTableType.CREATE_BRANCH : AlterTableType.CREATE_TAG;
+    for (int i = 1; i < command.getChildCount(); i++) {
+      ASTNode childNode = (ASTNode) command.getChild(i);
+      switch (childNode.getToken().getType()) {
+      case HiveParser.TOK_AS_OF_VERSION:
+        snapshotId = Long.parseLong(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_AS_OF_TIME:
+        ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() :
+            SessionState.get().getConf().getLocalTimeZone();
+        TimestampTZ ts = TimestampTZUtil.parse(stripQuotes(childNode.getChild(0).getText()), timeZone);
+        asOfTime = ts.toEpochMilli();
+        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.parseLong(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.parseLong(maxSnapshotAge));
+        }
+        break;
+      default:
+        throw new SemanticException("Unrecognized token in ALTER " + alterTableType.getName() + " statement");
+      }
+    }
+
+    if (alterTableType == AlterTableType.CREATE_BRANCH) {

Review Comment:
   Cool! Thanks for your guidance. 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] deniskuzZ commented on pull request #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

   hi @zhangbutao, sorry for the review delay, added a few comments 


-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/tag/create/AlterTableCreateTagAnalyzer.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.metaref.tag.create;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableDesc;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefDesc;
+import org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_TAG;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_TAG)
+public class AlterTableCreateTagAnalyzer extends AlterTableCreateMetaRefAnalyzer {
+
+  public AlterTableCreateTagAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+    super.alterTableType = AlterTableType.CREATE_TAG;
+  }
+
+  @Override
+  protected AbstractAlterTableDesc getAlterTableDesc(AlterTableTypeReq req) throws SemanticException {
+    AlterTableMetaRefSpec.CreateMetaRefSpec createTagspec =

Review Comment:
   Have removed `getAlterTableDesc`



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableMetaRefSpec.java:
##########
@@ -48,17 +57,17 @@ public String toString() {
         .add("operationParams", operationParams).toString();
   }
 
-  public static class CreateBranchSpec {
+  public static class CreateMetaRefSpec {

Review Comment:
   in iceberg project, it's called SnapshotRef, MetaRef is too wide
   type = SnapshotRefType.TAG & SnapshotRefType.BRANCH
   
   could we create builders for tag and branch similar to SnapshotRef.tagBuilder(snapshotId).build(); 
   ````
       refName
       snapshotId;
       type;
       asOfTime;
       minSnapshotsToKeep;
       maxSnapshotAgeMs;
       maxRefAgeMs;
   ````



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableMetaRefSpec.java:
##########
@@ -48,17 +57,17 @@ public String toString() {
         .add("operationParams", operationParams).toString();
   }
 
-  public static class CreateBranchSpec {
+  public static class CreateMetaRefSpec {

Review Comment:
   in iceberg project, it's called SnapshotRef, MetaRef is too wide
   type = SnapshotRefType.TAG & SnapshotRefType.BRANCH
   
   could we create builders for tag and branch similar to SnapshotRef.tagBuilder(snapshotId).build(); 



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -16,92 +16,159 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.branch.create;
+package org.apache.hadoop.hive.ql.ddl.table.metaref;
 
 import java.time.ZoneId;
 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.common.type.TimestampTZ;
 import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.QueryState;
-import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
 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.AbstractAlterTableDesc;
 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.AlterTableBranchSpec;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
-import static org.apache.hadoop.hive.ql.parse.AlterTableBranchSpec.AlterBranchOperationType.CREATE_BRANCH;
-
-@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
-public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+public abstract class AlterTableCreateMetaRefAnalyzer extends AbstractAlterTableAnalyzer {
+  protected static AbstractAlterTableDesc alterTableDesc;
+  protected static AlterTableType alterTableType;
+  protected abstract AbstractAlterTableDesc getAlterTableDesc(AlterTableTypeReq alterTableTypeReq)
+      throws SemanticException;
 
-  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+  public AlterTableCreateMetaRefAnalyzer(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.CREATE_BRANCH, false);
     DDLUtils.validateTableIsIceberg(table);
     inputs.add(new ReadEntity(table));
+    validateAlterTableType(table, alterTableType, false);
+    AlterTableTypeReq alterTableTypeReq = new AlterTableTypeReq();
 
-    String branchName = command.getChild(0).getText();
+    String metaRefName = command.getChild(0).getText();
+    alterTableTypeReq.setTableName(tableName);
+    alterTableTypeReq.setMetaRefName(metaRefName);
     Long snapshotId = null;
     Long asOfTime = null;
     Long maxRefAgeMs = null;
     Integer minSnapshotsToKeep = null;
     Long maxSnapshotAgeMs = null;
+    AlterTableType alterTableType = command.getType()
+        == HiveParser.TOK_ALTERTABLE_CREATE_BRANCH ? AlterTableType.CREATE_BRANCH : AlterTableType.CREATE_TAG;
     for (int i = 1; i < command.getChildCount(); i++) {
       ASTNode childNode = (ASTNode) command.getChild(i);
       switch (childNode.getToken().getType()) {
       case HiveParser.TOK_AS_OF_VERSION:
         snapshotId = Long.parseLong(childNode.getChild(0).getText());
+        alterTableTypeReq.setSnapshotId(snapshotId);
         break;
       case HiveParser.TOK_AS_OF_TIME:
         ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() :
             SessionState.get().getConf().getLocalTimeZone();
         TimestampTZ ts = TimestampTZUtil.parse(stripQuotes(childNode.getChild(0).getText()), timeZone);
         asOfTime = ts.toEpochMilli();
+        alterTableTypeReq.setAsOfTime(asOfTime);
         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.parseLong(maxRefAge));
+        maxRefAgeMs =
+            TimeUnit.valueOf(timeUnitOfBranchRetain.toUpperCase(Locale.ENGLISH)).toMillis(Long.parseLong(maxRefAge));
+        alterTableTypeReq.setMaxRefAgeMs(maxRefAgeMs);
         break;
       case HiveParser.TOK_WITH_SNAPSHOT_RETENTION:
         minSnapshotsToKeep = Integer.valueOf(childNode.getChild(0).getText());
+        alterTableTypeReq.setMinSnapshotsToKeep(minSnapshotsToKeep);
         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.parseLong(maxSnapshotAge));
+          alterTableTypeReq.setMaxSnapshotAgeMs(maxSnapshotAgeMs);
         }
         break;
       default:
-        throw new SemanticException("Unrecognized token in ALTER CREATE BRANCH statement");
+        throw new SemanticException("Unrecognized token in ALTER " + alterTableType.getName() + " statement");
       }
     }
+    alterTableDesc = getAlterTableDesc(alterTableTypeReq);
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterTableDesc)));
+  }
+
+  public class AlterTableTypeReq{

Review Comment:
   Removed the public setters. Make it protected as the child class `AlterTableCreateBranchAnalyzer `& `AlterTableCreateTagAnalyzer` will use 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] deniskuzZ commented on a diff in pull request #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchDesc.java:
##########
@@ -30,20 +30,20 @@
 public class AlterTableCreateBranchDesc extends AbstractAlterTableDesc {

Review Comment:
   AlterTableCreateBranchDesc & AlterTableCreateTagDesc are the same. 



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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;
+
+import java.time.ZoneId;
+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.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.branch.create.AlterTableCreateBranchDesc;
+import org.apache.hadoop.hive.ql.ddl.table.tag.create.AlterTableCreateTagDesc;
+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.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_BRANCH;
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_TAG;
+
+public abstract class AlterTableCreateMetaRefAnalyzer extends AbstractAlterTableAnalyzer {
+
+  private static AbstractAlterTableDesc alterTableDesc;
+
+  public AlterTableCreateMetaRefAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    DDLUtils.validateTableIsIceberg(table);
+    inputs.add(new ReadEntity(table));
+
+    String metaRefName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long asOfTime = null;
+    Long maxRefAgeMs = null;
+    Integer minSnapshotsToKeep = null;
+    Long maxSnapshotAgeMs = null;
+    AlterTableType alterTableType =
+        command.getType() == HiveParser.TOK_ALTERTABLE_CREATE_BRANCH ? AlterTableType.CREATE_BRANCH : AlterTableType.CREATE_TAG;
+    for (int i = 1; i < command.getChildCount(); i++) {
+      ASTNode childNode = (ASTNode) command.getChild(i);
+      switch (childNode.getToken().getType()) {
+      case HiveParser.TOK_AS_OF_VERSION:
+        snapshotId = Long.parseLong(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_AS_OF_TIME:
+        ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() :
+            SessionState.get().getConf().getLocalTimeZone();
+        TimestampTZ ts = TimestampTZUtil.parse(stripQuotes(childNode.getChild(0).getText()), timeZone);
+        asOfTime = ts.toEpochMilli();
+        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.parseLong(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.parseLong(maxSnapshotAge));
+        }
+        break;
+      default:
+        throw new SemanticException("Unrecognized token in ALTER " + alterTableType.getName() + " statement");
+      }
+    }
+
+    if (alterTableType == AlterTableType.CREATE_BRANCH) {

Review Comment:
   Branch have more fields than Tag, we must handle Branch and tag respectively.



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/branch/create/AlterTableCreateBranchDesc.java:
##########
@@ -30,20 +30,20 @@
 public class AlterTableCreateBranchDesc extends AbstractAlterTableDesc {

Review Comment:
   Done. Replaced the two class with a new class `AlterTableCreateMetaRefDesc.java` to handle branch&tag.



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.metaref.branch.create;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableDesc;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefDesc;
+import org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_BRANCH;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AlterTableCreateMetaRefAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+    super.alterTableType = AlterTableType.CREATE_BRANCH;
+  }
+
+  @Override
+  protected AbstractAlterTableDesc getAlterTableDesc(AlterTableCreateMetaRefAnalyzer.AlterTableTypeReq req)

Review Comment:
   hi @zhangbutao, sorry for the review delay, added a few comments 



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableMetaRefSpec.java:
##########
@@ -20,21 +20,30 @@
 
 import com.google.common.base.MoreObjects;
 
-public class AlterTableBranchSpec<T> {
+public class AlterTableMetaRefSpec<T> {

Review Comment:
   why do you need `AlterTableMetaRefSpec` wrapper at all, can't we just use parametrized CreateMetaRefSpec<T>
   new CreateMetaRefSpec<MetaRef.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] deniskuzZ commented on a diff in pull request #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/snapshotref/AlterTableCreateSnapshotRefAnalyzer.java:
##########
@@ -16,50 +16,48 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.branch.create;
+package org.apache.hadoop.hive.ql.ddl.table.snapshotref;
 
 import java.time.ZoneId;
 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.common.type.TimestampTZ;
 import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.QueryState;
-import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
 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.AbstractAlterTableDesc;
 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.AlterTableBranchSpec;
+import org.apache.hadoop.hive.ql.parse.AlterTableSnapshotRefSpec;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
-import static org.apache.hadoop.hive.ql.parse.AlterTableBranchSpec.AlterBranchOperationType.CREATE_BRANCH;
-
-@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
-public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+public abstract class AlterTableCreateSnapshotRefAnalyzer extends AbstractAlterTableAnalyzer {
+  protected static AbstractAlterTableDesc alterTableDesc;

Review Comment:
   remove `protected` & `static`



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableMetaRefSpec.java:
##########
@@ -48,17 +57,17 @@ public String toString() {
         .add("operationParams", operationParams).toString();
   }
 
-  public static class CreateBranchSpec {
+  public static class CreateMetaRefSpec {

Review Comment:
   makes sence



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

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

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTagExec.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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 org.apache.hadoop.hive.ql.parse.AlterTableTagSpec;
+import org.apache.iceberg.ManageSnapshots;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class IcebergTagExec {
+
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergTagExec.class);
+
+  private IcebergTagExec() {
+  }
+
+  public static void createTag(Table table, AlterTableTagSpec.CreateTagSpec createBranchSpec) {

Review Comment:
   looks like a copy-paste: createBranchSpec -> createTagSpec. How different are the 2 spec classes?



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/tag/create/AlterTableCreateTagAnalyzer.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.metaref.tag.create;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableDesc;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefDesc;
+import org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_TAG;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_TAG)
+public class AlterTableCreateTagAnalyzer extends AlterTableCreateMetaRefAnalyzer {
+
+  public AlterTableCreateTagAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+    super.alterTableType = AlterTableType.CREATE_TAG;
+  }
+
+  @Override
+  protected AbstractAlterTableDesc getAlterTableDesc(AlterTableTypeReq req) throws SemanticException {
+    AlterTableMetaRefSpec.CreateMetaRefSpec createTagspec =

Review Comment:
   createTag<S>pec



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableMetaRefSpec.java:
##########
@@ -48,17 +57,17 @@ public String toString() {
         .add("operationParams", operationParams).toString();
   }
 
-  public static class CreateBranchSpec {
+  public static class CreateMetaRefSpec {

Review Comment:
   in iceberg, it's called SnapshotRef, MetaRef is too wide



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

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

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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;
+
+import java.time.ZoneId;
+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.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.branch.create.AlterTableCreateBranchDesc;
+import org.apache.hadoop.hive.ql.ddl.table.tag.create.AlterTableCreateTagDesc;
+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.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_BRANCH;
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_TAG;
+
+public abstract class AlterTableCreateMetaRefAnalyzer extends AbstractAlterTableAnalyzer {
+
+  private static AbstractAlterTableDesc alterTableDesc;
+
+  public AlterTableCreateMetaRefAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    DDLUtils.validateTableIsIceberg(table);
+    inputs.add(new ReadEntity(table));
+
+    String metaRefName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long asOfTime = null;
+    Long maxRefAgeMs = null;
+    Integer minSnapshotsToKeep = null;
+    Long maxSnapshotAgeMs = null;
+    AlterTableType alterTableType =
+        command.getType() == HiveParser.TOK_ALTERTABLE_CREATE_BRANCH ? AlterTableType.CREATE_BRANCH : AlterTableType.CREATE_TAG;
+    for (int i = 1; i < command.getChildCount(); i++) {
+      ASTNode childNode = (ASTNode) command.getChild(i);
+      switch (childNode.getToken().getType()) {
+      case HiveParser.TOK_AS_OF_VERSION:
+        snapshotId = Long.parseLong(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_AS_OF_TIME:
+        ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() :
+            SessionState.get().getConf().getLocalTimeZone();
+        TimestampTZ ts = TimestampTZUtil.parse(stripQuotes(childNode.getChild(0).getText()), timeZone);
+        asOfTime = ts.toEpochMilli();
+        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.parseLong(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.parseLong(maxSnapshotAge));
+        }
+        break;
+      default:
+        throw new SemanticException("Unrecognized token in ALTER " + alterTableType.getName() + " statement");
+      }
+    }
+
+    if (alterTableType == AlterTableType.CREATE_BRANCH) {

Review Comment:
   sorry, I don't get it
   ````
   abstract class AlterTableCreateMetaRefAnalyzer extends AbstractAlterTableAnalyzer {
       abstract AbstractAlterTableDesc  getAlterTableDesc(AlterTableTypeReq req)
   
       class AlterTableTypeReq{
           snapshotId, asOfTime, maxRefAgeMs, minSnapshotsToKeep, maxSnapshotAgeMs
       }
   }
   
   class AlterTableCreateBranchAnalyzer extends AlterTableCreateMetaRefAnalyzer {
       ...
       public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
           super(queryState);
           alterTableType = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH;
       }
    ....
       AbstractAlterTableDesc  getAlterTableDesc(AlterTableTypeReq req){
            AlterTableMetaRefSpec.CreateTagSpec createTagspec =
             new AlterTableMetaRefSpec.CreateTagSpec(req.getMetaRefName(), req.getSnapshotId(), req.getAsOfTime(), req.getMaxRefAgeMs());
         AlterTableMetaRefSpec<AlterTableMetaRefSpec.CreateTagSpec> alterTableTagSpec
             = new AlterTableMetaRefSpec(CREATE_TAG, createTagspec);
         return new AlterTableCreateTagDesc(tableName, alterTableTagSpec);
   }
   ````



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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;
+
+import java.time.ZoneId;
+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.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.table.branch.create.AlterTableCreateBranchDesc;
+import org.apache.hadoop.hive.ql.ddl.table.tag.create.AlterTableCreateTagDesc;
+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.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_BRANCH;
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_TAG;
+
+public abstract class AlterTableCreateMetaRefAnalyzer extends AbstractAlterTableAnalyzer {
+
+  private static AbstractAlterTableDesc alterTableDesc;
+
+  public AlterTableCreateMetaRefAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  protected void analyzeCommand(TableName tableName, Map<String, String> partitionSpec, ASTNode command)
+      throws SemanticException {
+    Table table = getTable(tableName);
+    DDLUtils.validateTableIsIceberg(table);
+    inputs.add(new ReadEntity(table));
+
+    String metaRefName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long asOfTime = null;
+    Long maxRefAgeMs = null;
+    Integer minSnapshotsToKeep = null;
+    Long maxSnapshotAgeMs = null;
+    AlterTableType alterTableType =
+        command.getType() == HiveParser.TOK_ALTERTABLE_CREATE_BRANCH ? AlterTableType.CREATE_BRANCH : AlterTableType.CREATE_TAG;
+    for (int i = 1; i < command.getChildCount(); i++) {
+      ASTNode childNode = (ASTNode) command.getChild(i);
+      switch (childNode.getToken().getType()) {
+      case HiveParser.TOK_AS_OF_VERSION:
+        snapshotId = Long.parseLong(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_AS_OF_TIME:
+        ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() :
+            SessionState.get().getConf().getLocalTimeZone();
+        TimestampTZ ts = TimestampTZUtil.parse(stripQuotes(childNode.getChild(0).getText()), timeZone);
+        asOfTime = ts.toEpochMilli();
+        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.parseLong(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.parseLong(maxSnapshotAge));
+        }
+        break;
+      default:
+        throw new SemanticException("Unrecognized token in ALTER " + alterTableType.getName() + " statement");
+      }
+    }
+
+    if (alterTableType == AlterTableType.CREATE_BRANCH) {

Review Comment:
   sorry, I don't get it
   ````
   abstract class AlterTableCreateMetaRefAnalyzer extends AbstractAlterTableAnalyzer {
       protected AbstractAlterTableDesc alterTableDesc;
   
       abstract AbstractAlterTableDesc  getAlterTableDesc(AlterTableTypeReq req)
   
       class AlterTableTypeReq{
           snapshotId, asOfTime, maxRefAgeMs, minSnapshotsToKeep, maxSnapshotAgeMs
       }
   }
   
   class AlterTableCreateBranchAnalyzer extends AlterTableCreateMetaRefAnalyzer {
       ...
       public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
           super(queryState);
           alterTableType = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH;
       }
    ....
       AbstractAlterTableDesc  getAlterTableDesc(AlterTableTypeReq req){
            AlterTableMetaRefSpec.CreateTagSpec createTagspec =
             new AlterTableMetaRefSpec.CreateTagSpec(req.getMetaRefName(), req.getSnapshotId(), req.getAsOfTime(), req.getMaxRefAgeMs());
         AlterTableMetaRefSpec<AlterTableMetaRefSpec.CreateTagSpec> alterTableTagSpec
             = new AlterTableMetaRefSpec(CREATE_TAG, createTagspec);
         return new AlterTableCreateTagDesc(tableName, alterTableTagSpec);
   }
   ````



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -16,92 +16,159 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.branch.create;
+package org.apache.hadoop.hive.ql.ddl.table.metaref;
 
 import java.time.ZoneId;
 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.common.type.TimestampTZ;
 import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.QueryState;
-import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
 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.AbstractAlterTableDesc;
 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.AlterTableBranchSpec;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
-import static org.apache.hadoop.hive.ql.parse.AlterTableBranchSpec.AlterBranchOperationType.CREATE_BRANCH;
-
-@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
-public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+public abstract class AlterTableCreateMetaRefAnalyzer extends AbstractAlterTableAnalyzer {
+  protected static AbstractAlterTableDesc alterTableDesc;
+  protected static AlterTableType alterTableType;
+  protected abstract AbstractAlterTableDesc getAlterTableDesc(AlterTableTypeReq alterTableTypeReq)
+      throws SemanticException;
 
-  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+  public AlterTableCreateMetaRefAnalyzer(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.CREATE_BRANCH, false);
     DDLUtils.validateTableIsIceberg(table);
     inputs.add(new ReadEntity(table));
+    validateAlterTableType(table, alterTableType, false);
+    AlterTableTypeReq alterTableTypeReq = new AlterTableTypeReq();
 
-    String branchName = command.getChild(0).getText();
+    String metaRefName = command.getChild(0).getText();
+    alterTableTypeReq.setTableName(tableName);
+    alterTableTypeReq.setMetaRefName(metaRefName);
     Long snapshotId = null;
     Long asOfTime = null;
     Long maxRefAgeMs = null;
     Integer minSnapshotsToKeep = null;
     Long maxSnapshotAgeMs = null;
+    AlterTableType alterTableType = command.getType()
+        == HiveParser.TOK_ALTERTABLE_CREATE_BRANCH ? AlterTableType.CREATE_BRANCH : AlterTableType.CREATE_TAG;
     for (int i = 1; i < command.getChildCount(); i++) {
       ASTNode childNode = (ASTNode) command.getChild(i);
       switch (childNode.getToken().getType()) {
       case HiveParser.TOK_AS_OF_VERSION:
         snapshotId = Long.parseLong(childNode.getChild(0).getText());
+        alterTableTypeReq.setSnapshotId(snapshotId);
         break;
       case HiveParser.TOK_AS_OF_TIME:
         ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() :
             SessionState.get().getConf().getLocalTimeZone();
         TimestampTZ ts = TimestampTZUtil.parse(stripQuotes(childNode.getChild(0).getText()), timeZone);
         asOfTime = ts.toEpochMilli();
+        alterTableTypeReq.setAsOfTime(asOfTime);
         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.parseLong(maxRefAge));
+        maxRefAgeMs =
+            TimeUnit.valueOf(timeUnitOfBranchRetain.toUpperCase(Locale.ENGLISH)).toMillis(Long.parseLong(maxRefAge));
+        alterTableTypeReq.setMaxRefAgeMs(maxRefAgeMs);
         break;
       case HiveParser.TOK_WITH_SNAPSHOT_RETENTION:
         minSnapshotsToKeep = Integer.valueOf(childNode.getChild(0).getText());
+        alterTableTypeReq.setMinSnapshotsToKeep(minSnapshotsToKeep);
         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.parseLong(maxSnapshotAge));
+          alterTableTypeReq.setMaxSnapshotAgeMs(maxSnapshotAgeMs);
         }
         break;
       default:
-        throw new SemanticException("Unrecognized token in ALTER CREATE BRANCH statement");
+        throw new SemanticException("Unrecognized token in ALTER " + alterTableType.getName() + " statement");
       }
     }
+    alterTableDesc = getAlterTableDesc(alterTableTypeReq);
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterTableDesc)));
+  }
+
+  public class AlterTableTypeReq{

Review Comment:
   could we mark it as a package-private?



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -57,90 +59,55 @@ protected void analyzeCommand(TableName tableName, Map<String, String> partition
     DDLUtils.validateTableIsIceberg(table);
     inputs.add(new ReadEntity(table));
     validateAlterTableType(table, alterTableType, false);
-    AlterTableTypeReq alterTableTypeReq = new AlterTableTypeReq();
 
     String metaRefName = command.getChild(0).getText();
-    alterTableTypeReq.tableName = tableName;
-    alterTableTypeReq.metaRefName = metaRefName;
     Long snapshotId = null;
     Long asOfTime = null;
     Long maxRefAgeMs = null;
     Integer minSnapshotsToKeep = null;
     Long maxSnapshotAgeMs = null;
     AlterTableType alterTableType = command.getType()

Review Comment:
   why do you need this? alterTableType is set in concrete implementation (branch & tag)
   also, why do we need `AlterMetaRefOperationType`, can't we use `AlterTableType`?



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

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

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/snapshotref/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.snapshotref.branch.create;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.table.snapshotref.AlterTableCreateSnapshotRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+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 AlterTableCreateSnapshotRefAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+    super.alterTableType = AlterTableType.CREATE_BRANCH;

Review Comment:
   why `super`



##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/snapshotref/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.snapshotref.branch.create;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.table.snapshotref.AlterTableCreateSnapshotRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+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 AlterTableCreateSnapshotRefAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+    super.alterTableType = AlterTableType.CREATE_BRANCH;

Review Comment:
   why `super`?



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java:
##########
@@ -6724,6 +6725,15 @@ public void alterTableBranchOperation(Table table, AlterTableBranchSpec createBr
     }
   }
 
+  public void alterTableTagOperation(Table table, AlterTableTagSpec createTagSpec) throws HiveException {
+    try {
+      HiveStorageHandler storageHandler = createStorageHandler(table.getTTable());

Review Comment:
   why do we need to create a new storageHandler each time? should we use some cache for those?



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTagExec.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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 org.apache.hadoop.hive.ql.parse.AlterTableTagSpec;
+import org.apache.iceberg.ManageSnapshots;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class IcebergTagExec {
+
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergTagExec.class);
+
+  private IcebergTagExec() {
+  }
+
+  public static void createTag(Table table, AlterTableTagSpec.CreateTagSpec createBranchSpec) {

Review Comment:
   Have fixed in commit https://github.com/apache/hive/commit/6ef586f028f1d14a7cfbb21fef5b881d48d694a8.



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

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

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

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

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableTagSpec.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 AlterTableTagSpec <T>{

Review Comment:
   https://github.com/apache/hive/pull/4372/files#diff-ca555c7ed1f86abc1a58feeb22bf661a84c657a3791944c872a94d68c8309e8eR32  It is used here: Line-32.
   I want to use the generic type to delegate multiple alter tag operation, e.g. `CreateTagSpec` and the further work
    `DropTagSpec` & `ReplaceTagSpec`.



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -763,6 +764,27 @@ public void alterTableBranchOperation(org.apache.hadoop.hive.ql.metadata.Table h
 
   }
 
+  @Override
+  public void alterTableTagOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
+      AlterTableTagSpec alterTagSpec) {
+    TableDesc tableDesc = Utilities.getTableDesc(hmsTable);
+    Table icebergTable = IcebergTableUtil.getTable(conf, tableDesc.getProperties());
+    Optional.ofNullable(icebergTable.currentSnapshot()).orElseThrow(() ->
+        new UnsupportedOperationException(String.format("Cannot alter tag on iceberg table" +
+            " %s.%s which has no snapshot", hmsTable.getDbName(), hmsTable.getTableName())));
+
+    switch (alterTagSpec.getOperationType()) {

Review Comment:
   We may have multiple `OperationType` , e.g. `CREATE_BRANCH` & `DROP_BRANCH` & `REPLACE_BRANCH`. I think switch is more cleaner.



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -763,6 +764,27 @@ public void alterTableBranchOperation(org.apache.hadoop.hive.ql.metadata.Table h
 
   }
 
+  @Override
+  public void alterTableTagOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
+      AlterTableTagSpec alterTagSpec) {
+    TableDesc tableDesc = Utilities.getTableDesc(hmsTable);
+    Table icebergTable = IcebergTableUtil.getTable(conf, tableDesc.getProperties());
+    Optional.ofNullable(icebergTable.currentSnapshot()).orElseThrow(() ->
+        new UnsupportedOperationException(String.format("Cannot alter tag on iceberg table" +
+            " %s.%s which has no snapshot", hmsTable.getDbName(), hmsTable.getTableName())));
+
+    switch (alterTagSpec.getOperationType()) {

Review Comment:
   won't simple if-else be sufficient here? same for alterTableBranchOperation



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/tag/create/AlterTableCreateTagAnalyzer.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.tag.create;
+
+import java.time.ZoneId;
+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.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+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.AlterTableTagSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableTagSpec.AlterTagOperationType.CREATE_TAG;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_TAG)
+public class AlterTableCreateTagAnalyzer extends AbstractAlterTableAnalyzer {
+  public AlterTableCreateTagAnalyzer(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.CREATE_TAG, false);
+    DDLUtils.validateTableIsIceberg(table);
+    inputs.add(new ReadEntity(table));
+
+    String tagName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long asOfTime = null;
+    Long maxRefAgeMs = 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:
+        snapshotId = Long.parseLong(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_AS_OF_TIME:
+        ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() :
+            SessionState.get().getConf().getLocalTimeZone();
+        TimestampTZ ts = TimestampTZUtil.parse(stripQuotes(childNode.getChild(0).getText()), timeZone);
+        asOfTime = ts.toEpochMilli();
+        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.parseLong(maxRefAge));
+        break;
+      default:
+        throw new SemanticException("Unrecognized token in ALTER CREATE TAG statement");
+      }
+    }
+
+    AlterTableTagSpec.CreateTagSpec createTagspec =
+        new AlterTableTagSpec.CreateTagSpec(tagName, snapshotId, asOfTime, maxRefAgeMs);
+    AlterTableTagSpec alterTableTagSpec = new AlterTableTagSpec(CREATE_TAG, createTagspec);
+    AlterTableCreateTagDesc desc = new AlterTableCreateTagDesc(tableName, alterTableTagSpec);

Review Comment:
   I create a new abtract class `ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AlterTableCreateMetaRefAnalyzer.java` to make code reusable.
   https://github.com/apache/hive/pull/4372/files#diff-eac7a1ce623e75ae7789cba72422c240ec9d4f877db53280251a6ad9816e8bf5



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableMetaRefSpec.java:
##########
@@ -20,21 +20,30 @@
 
 import com.google.common.base.MoreObjects;
 
-public class AlterTableBranchSpec<T> {
+public class AlterTableMetaRefSpec<T> {

Review Comment:
   should we mark it as abstract and extend CreateTagSpec and CreateBranchSpec from it? operationType could be set in the constructor



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableMetaRefSpec.java:
##########
@@ -48,17 +57,17 @@ public String toString() {
         .add("operationParams", operationParams).toString();
   }
 
-  public static class CreateBranchSpec {
+  public static class CreateMetaRefSpec {

Review Comment:
   in iceberg project, it's called SnapshotRef, MetaRef is too wide
   type = SnapshotRefType.TAG & SnapshotRefType.BRANCH
   
   could we create also builders for tag and branch similar to SnapshotRef.tagBuilder(snapshotId).build(); 
   ````
       refName
       snapshotId;
       type;
       asOfTime;
       minSnapshotsToKeep;
       maxSnapshotAgeMs;
       maxRefAgeMs;
   ````



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -57,90 +59,55 @@ protected void analyzeCommand(TableName tableName, Map<String, String> partition
     DDLUtils.validateTableIsIceberg(table);
     inputs.add(new ReadEntity(table));
     validateAlterTableType(table, alterTableType, false);
-    AlterTableTypeReq alterTableTypeReq = new AlterTableTypeReq();
 
     String metaRefName = command.getChild(0).getText();
-    alterTableTypeReq.tableName = tableName;
-    alterTableTypeReq.metaRefName = metaRefName;
     Long snapshotId = null;
     Long asOfTime = null;
     Long maxRefAgeMs = null;
     Integer minSnapshotsToKeep = null;
     Long maxSnapshotAgeMs = null;
     AlterTableType alterTableType = command.getType()

Review Comment:
   why do you need this? alterTableType is set in implementing classes



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/AlterTableCreateMetaRefAnalyzer.java:
##########
@@ -57,90 +59,55 @@ protected void analyzeCommand(TableName tableName, Map<String, String> partition
     DDLUtils.validateTableIsIceberg(table);
     inputs.add(new ReadEntity(table));
     validateAlterTableType(table, alterTableType, false);
-    AlterTableTypeReq alterTableTypeReq = new AlterTableTypeReq();
 
     String metaRefName = command.getChild(0).getText();
-    alterTableTypeReq.tableName = tableName;
-    alterTableTypeReq.metaRefName = metaRefName;
     Long snapshotId = null;
     Long asOfTime = null;
     Long maxRefAgeMs = null;
     Integer minSnapshotsToKeep = null;
     Long maxSnapshotAgeMs = null;
     AlterTableType alterTableType = command.getType()

Review Comment:
   Done.
   Removed `AlterMetaRefOperationType` and Use `alterTableType ` which was set in implementation (branch & tag).
   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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

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

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/snapshotref/tag/create/AlterTableCreateTagAnalyzer.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.snapshotref.tag.create;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.table.snapshotref.AlterTableCreateSnapshotRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_TAG)
+public class AlterTableCreateTagAnalyzer extends AlterTableCreateSnapshotRefAnalyzer {
+
+  public AlterTableCreateTagAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+    super.alterTableType = AlterTableType.CREATE_TAG;

Review Comment:
   why `super`?



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/snapshotref/AlterTableCreateSnapshotRefAnalyzer.java:
##########
@@ -16,50 +16,48 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hive.ql.ddl.table.branch.create;
+package org.apache.hadoop.hive.ql.ddl.table.snapshotref;
 
 import java.time.ZoneId;
 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.common.type.TimestampTZ;
 import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.QueryState;
-import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
 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.AbstractAlterTableDesc;
 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.AlterTableBranchSpec;
+import org.apache.hadoop.hive.ql.parse.AlterTableSnapshotRefSpec;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
-import static org.apache.hadoop.hive.ql.parse.AlterTableBranchSpec.AlterBranchOperationType.CREATE_BRANCH;
-
-@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
-public class AlterTableCreateBranchAnalyzer extends AbstractAlterTableAnalyzer {
+public abstract class AlterTableCreateSnapshotRefAnalyzer extends AbstractAlterTableAnalyzer {
+  protected static AbstractAlterTableDesc alterTableDesc;

Review Comment:
   transform into local var



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableMetaRefSpec.java:
##########
@@ -20,21 +20,30 @@
 
 import com.google.common.base.MoreObjects;
 
-public class AlterTableBranchSpec<T> {
+public class AlterTableMetaRefSpec<T> {

Review Comment:
   I have replaced `CreateBranchSpec `& `CreateTagSpec ` with one inner class `CreateMetaRefSpec`. I think this will be more simpler. Fixed in https://github.com/apache/hive/pull/4372/commits/a63d3708d75a577d1f58a72455e2516a80a2fa09
   Please take a look again. 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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

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

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/tag/create/AlterTableCreateTagOperation.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.tag.create;
+
+import org.apache.hadoop.hive.ql.ddl.DDLOperation;
+import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
+import org.apache.hadoop.hive.ql.metadata.Table;
+
+public class AlterTableCreateTagOperation extends DDLOperation<AlterTableCreateTagDesc> {

Review Comment:
   Done. Replaced the two class `AlterTableCreateTagOperation&AlterTableCreateBranchOperation`  with a new class AlterTableCreateMetaRefOperation.java to handle branch&tag.



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/tag/create/AlterTableCreateTagOperation.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.tag.create;
+
+import org.apache.hadoop.hive.ql.ddl.DDLOperation;
+import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
+import org.apache.hadoop.hive.ql.metadata.Table;
+
+public class AlterTableCreateTagOperation extends DDLOperation<AlterTableCreateTagDesc> {

Review Comment:
   why do we need this if there is no diff except type?



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

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

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


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/tag/create/AlterTableCreateTagAnalyzer.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.tag.create;
+
+import java.time.ZoneId;
+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.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+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.AlterTableTagSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableTagSpec.AlterTagOperationType.CREATE_TAG;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_TAG)
+public class AlterTableCreateTagAnalyzer extends AbstractAlterTableAnalyzer {
+  public AlterTableCreateTagAnalyzer(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.CREATE_TAG, false);
+    DDLUtils.validateTableIsIceberg(table);
+    inputs.add(new ReadEntity(table));
+
+    String tagName = command.getChild(0).getText();
+    Long snapshotId = null;
+    Long asOfTime = null;
+    Long maxRefAgeMs = 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:
+        snapshotId = Long.parseLong(childNode.getChild(0).getText());
+        break;
+      case HiveParser.TOK_AS_OF_TIME:
+        ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() :
+            SessionState.get().getConf().getLocalTimeZone();
+        TimestampTZ ts = TimestampTZUtil.parse(stripQuotes(childNode.getChild(0).getText()), timeZone);
+        asOfTime = ts.toEpochMilli();
+        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.parseLong(maxRefAge));
+        break;
+      default:
+        throw new SemanticException("Unrecognized token in ALTER CREATE TAG statement");
+      }
+    }
+
+    AlterTableTagSpec.CreateTagSpec createTagspec =
+        new AlterTableTagSpec.CreateTagSpec(tagName, snapshotId, asOfTime, maxRefAgeMs);
+    AlterTableTagSpec alterTableTagSpec = new AlterTableTagSpec(CREATE_TAG, createTagspec);
+    AlterTableCreateTagDesc desc = new AlterTableCreateTagDesc(tableName, alterTableTagSpec);

Review Comment:
   could we extend from AlterTableCreateBranchAnalyzer and override just AlterTableCreateTagDesc creation (i.e. getAlterTableDesc or something)? 



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java:
##########
@@ -6724,6 +6725,15 @@ public void alterTableBranchOperation(Table table, AlterTableBranchSpec createBr
     }
   }
 
+  public void alterTableTagOperation(Table table, AlterTableTagSpec createTagSpec) throws HiveException {
+    try {
+      HiveStorageHandler storageHandler = createStorageHandler(table.getTTable());

Review Comment:
   https://issues.apache.org/jira/browse/HIVE-27435 I have filed a ticket to trace this issue. I think it can be fixed soon.



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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

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

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTagExec.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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 org.apache.hadoop.hive.ql.parse.AlterTableTagSpec;
+import org.apache.iceberg.ManageSnapshots;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class IcebergTagExec {
+
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergTagExec.class);
+
+  private IcebergTagExec() {
+  }
+
+  public static void createTag(Table table, AlterTableTagSpec.CreateTagSpec createBranchSpec) {

Review Comment:
   that's fine, param names a copy-pasted, see createBranchSpec -> createTagSpec



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.metaref.branch.create;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableDesc;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefDesc;
+import org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_BRANCH;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AlterTableCreateMetaRefAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+    super.alterTableType = AlterTableType.CREATE_BRANCH;
+  }
+
+  @Override
+  protected AbstractAlterTableDesc getAlterTableDesc(AlterTableCreateMetaRefAnalyzer.AlterTableTypeReq req)

Review Comment:
   since you declared a common class for both branch and tag (i.e. `CreateMetaRefSpec`) we do not need custom getAlterTableDesc anymore, we could just supply a proper `alterTableType`



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.metaref.branch.create;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableDesc;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefDesc;
+import org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_BRANCH;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AlterTableCreateMetaRefAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+    super.alterTableType = AlterTableType.CREATE_BRANCH;
+  }
+
+  @Override
+  protected AbstractAlterTableDesc getAlterTableDesc(AlterTableCreateMetaRefAnalyzer.AlterTableTypeReq req)

Review Comment:
   since you declared a common class for both branch and tag (i.e. `CreateMetaRefSpec`) we do not need custom `getAlterTableDesc` anymore, we could drop `AlterTableTypeReq` and supply a proper `alterTableType`



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/metaref/branch/create/AlterTableCreateBranchAnalyzer.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.metaref.branch.create;
+
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableDesc;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefAnalyzer;
+import org.apache.hadoop.hive.ql.ddl.table.AlterTableType;
+import org.apache.hadoop.hive.ql.ddl.table.metaref.AlterTableCreateMetaRefDesc;
+import org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+import static org.apache.hadoop.hive.ql.parse.AlterTableMetaRefSpec.AlterMetaRefOperationType.CREATE_BRANCH;
+
+@DDLSemanticAnalyzerFactory.DDLType(types = HiveParser.TOK_ALTERTABLE_CREATE_BRANCH)
+public class AlterTableCreateBranchAnalyzer extends AlterTableCreateMetaRefAnalyzer {
+
+  public AlterTableCreateBranchAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+    super.alterTableType = AlterTableType.CREATE_BRANCH;
+  }
+
+  @Override
+  protected AbstractAlterTableDesc getAlterTableDesc(AlterTableCreateMetaRefAnalyzer.AlterTableTypeReq req)

Review Comment:
   Fixed in https://github.com/apache/hive/pull/4372/commits/e32b10a64775518cffb5dfac99a5b1455f53d5d0



-- 
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 #4372: HIVE-27235: Iceberg:CREATE TAG SQL implementation

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


##########
ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableMetaRefSpec.java:
##########
@@ -48,17 +57,17 @@ public String toString() {
         .add("operationParams", operationParams).toString();
   }
 
-  public static class CreateBranchSpec {
+  public static class CreateMetaRefSpec {

Review Comment:
   in iceberg project, it's called SnapshotRef, MetaRef is too wide
   type = SnapshotRefType.TAG & SnapshotRefType.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