You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by "simhadri-g (via GitHub)" <gi...@apache.org> on 2023/05/20 20:30:20 UTC

[GitHub] [hive] simhadri-g opened a new pull request, #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

simhadri-g opened a new pull request, #4346:
URL: https://github.com/apache/hive/pull/4346

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://cwiki.apache.org/confluence/display/Hive/HowToContribute
     2. Ensure that you have created an issue on the Hive project JIRA: https://issues.apache.org/jira/projects/HIVE/summary
     3. Ensure you have added or run the appropriate tests for your PR: 
     4. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]HIVE-XXXXX:  Your PR title ...'.
     5. Be sure to keep the PR description updated to reflect all changes.
     6. Please write your PR title to summarize what this PR proposes.
     7. If possible, provide a concise example to reproduce the issue for a faster review.
   
   -->
   
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description, screenshot and/or a reproducable example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Hive versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   
   Q test


-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));

Review Comment:
   could we please move string literals to constants?



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn: Unable to show partitions for iceberg table - ", e);
+    }
+    return parts;
+  }
+
+  @Override
+  public boolean supportsPartitions() {
+    return true;
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,
+      RecordReader<WritableComparable, Writable> reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws Exception {
+
+    List<String> parts = Lists.newArrayList();
+    Writable value = reader.createValue();
+    WritableComparable key = reader.createKey();
+    boolean notEoF = true;
+    String prevRow = "";
+
+    try (FetchFormatter fetcher = new DefaultFetchFormatter()) {
+      fetcher.initialize(job, HiveTableUtil.getProps());
+      org.apache.hadoop.hive.ql.metadata.Table metaDataPartTable =
+          context.getDb().getTable(hmstbl.getDbName(), hmstbl.getTableName(), "partitions", true);
+
+      while (notEoF) {
+        reader.next(key, value);

Review Comment:
   doesn't it return EOF marker?
   ````
   opNotEOF = reader.next(key, value);
   ````



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn: Unable to show partitions for iceberg table - ", e);
+    }
+    return parts;
+  }
+
+  @Override
+  public boolean supportsPartitions() {
+    return true;
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,
+      RecordReader<WritableComparable, Writable> reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws Exception {
+
+    List<String> parts = Lists.newArrayList();
+    Writable value = reader.createValue();
+    WritableComparable key = reader.createKey();
+    boolean notEoF = true;
+    String prevRow = "";
+
+    try (FetchFormatter fetcher = new DefaultFetchFormatter()) {
+      fetcher.initialize(job, HiveTableUtil.getProps());
+      org.apache.hadoop.hive.ql.metadata.Table metaDataPartTable =
+          context.getDb().getTable(hmstbl.getDbName(), hmstbl.getTableName(), "partitions", true);
+
+      while (notEoF) {
+        reader.next(key, value);

Review Comment:
   doesn't it return EOF marker?
   ````
   opNotEOF = reader.next(key, value);
   ````
   
   could be changed to
   ````
   Deserializer currSerDe = metaDataPartTable.getDeserializer();
   while (recordReader.next(key, value)) {
       String[] row = fetcher.convert(
           currSerDe.deserialize(value), currSerDe.getObjectInspector()
       ).toString().split("\t");
       
       parts.add(HiveTableUtil.getParseData(row[PART_IDX], row[SPEC_IDX]));
   }
   ````



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getProps() {

Review Comment:
   the method name is not self-explanatory, maybe rename it to `getSerializationProps()`?



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java:
##########
@@ -638,4 +640,18 @@ default void prepareAlterTableEnvironmentContext(AbstractAlterTableDesc alterTab
   default Boolean hasAppendsOnly(org.apache.hadoop.hive.ql.metadata.Table hmsTable, SnapshotContext since) {
     return null;
   }
+
+  /**
+   * Checks if storage handler supports Show Partitions and returns a list of partitions
+   * @param context
+   * @param tbl
+   * @return List of partitions
+   * @throws UnsupportedOperationException
+   * @throws HiveException
+   */
+  default List<String> showPartitions(DDLOperationContext context,
+      org.apache.hadoop.hive.ql.metadata.Table tbl) throws UnsupportedOperationException, HiveException {

Review Comment:
   👍 



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

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

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


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -159,9 +169,12 @@ public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, H
   private static final String TABLE_NAME_SEPARATOR = "..";
   private static final String ICEBERG = "iceberg";
   private static final String PUFFIN = "puffin";

Review Comment:
   can find any usages of 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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1223239700


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));

Review Comment:
   Done



##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn: Unable to show partitions for iceberg table - ", e);

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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1226758926


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -54,12 +54,12 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d
   @Override
   public int execute() throws HiveException {
     Table tbl = context.getDb().getTable(desc.getTabName());
-    if (!tbl.isPartitioned()) {
-      throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());
-    }
-
     List<String> parts;
-    if (desc.getCond() != null || desc.getOrder() != null) {
+    if(tbl!=null && tbl.isNonNative() && tbl.getStorageHandler().supportsPartitionTransform()){

Review Comment:
   Yes, unit test failed will NPE as a result i had to add the null check.
   Will update the formatting.



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getSerializationProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId, ObjectMapper mapper)
+      throws JsonProcessingException {
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString =
+        map.entrySet().stream().filter(entry -> entry.getValue() != null).map(java.lang.Object::toString)
+            .collect(Collectors.joining("/"));
+    return String.format("Spec-id=%s/%s", specId, partString);
+  }
+
+  protected static JobConf getPartJobConf(Configuration confs, Path path,
+      org.apache.hadoop.hive.ql.metadata.Table tbl) {
+    JobConf job = new JobConf(confs);
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_MAPRED_INPUT_DIR, path.toString());
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_READ_COLUMN_NAMES_CONF_STR,
+        "partition,record_count,file_count," + "spec_id");
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_ICEBERG_MR_TABLE_LOC, tbl.getPath().toString());
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_MAPRED_INPUT_DIR, path.toString());

Review Comment:
   why it's declared twice? line 253 and 257? same for HIVE_MAPRED_INPUT_DIR



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

   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=4346)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=CODE_SMELL) [8 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1239498590


##########
iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out:
##########
@@ -0,0 +1,214 @@
+PREHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@hiveT1
+POSTHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@hiveT1
+PREHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@hivet1
+POSTHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@hivet1
+POSTHOOK: Output: default@hivet1@d_part=10/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=2/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=3/e_part=4
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).c SCRIPT []
+PREHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice1
+POSTHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice1
+PREHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice1
+POSTHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice1
+PREHOOK: query: show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+d_part=10/e_part=5
+d_part=2/e_part=5
+d_part=3/e_part=4
+PREHOOK: query: show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: select * from default.ice1.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice1.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: explain show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: explain show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"hiveT1"}
+
+PREHOOK: query: explain show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: explain show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"ice1"}
+
+PREHOOK: query: explain select * from default.ice1.partitions
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain select * from default.ice1.partitions
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Select Operator [SEL_1]
+      Output:["_col0","_col1","_col2","_col3"]
+      TableScan [TS_0]
+        Output:["partition","record_count","file_count","spec_id"]
+
+PREHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice2
+POSTHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice2
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+PREHOOK: type: ALTERTABLE_SETPARTSPEC
+PREHOOK: Input: default@ice2
+POSTHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+POSTHOOK: type: ALTERTABLE_SETPARTSPEC
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5
+Spec-id=0/d_part=2/e_part=5
+Spec-id=0/d_part=3/e_part=4
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+{"d_part":null,"e_part":null,"c":2}	2	1	1
+{"d_part":null,"e_part":null,"c":3}	1	1	1
+{"d_part":null,"e_part":null,"c":4}	1	1	1
+{"d_part":null,"e_part":null,"c":5}	1	1	1
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5

Review Comment:
   Updated as discussed to :
   
   ```
   {Spec-id=0}:d_part=10/e_part=5
   {Spec-id=0}:d_part=2/e_part=5
   {Spec-id=0}:d_part=3/e_part=4
   {Spec-id=1}*:c=2
   {Spec-id=1}*:c=3
   {Spec-id=1}*:c=4
   {Spec-id=1}*:c=5
   ```



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn: Unable to show partitions for iceberg table - ", e);
+    }
+    return parts;
+  }
+
+  @Override
+  public boolean supportsPartitions() {
+    return true;
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,
+      RecordReader<WritableComparable, Writable> reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws Exception {
+
+    List<String> parts = Lists.newArrayList();
+    Writable value = reader.createValue();
+    WritableComparable key = reader.createKey();
+    boolean notEoF = true;
+    String prevRow = "";
+
+    try (FetchFormatter fetcher = new DefaultFetchFormatter()) {
+      fetcher.initialize(job, HiveTableUtil.getProps());
+      org.apache.hadoop.hive.ql.metadata.Table metaDataPartTable =
+          context.getDb().getTable(hmstbl.getDbName(), hmstbl.getTableName(), "partitions", true);
+
+      while (notEoF) {
+        reader.next(key, value);
+        Deserializer currSerDe = metaDataPartTable.getDeserializer();

Review Comment:
   why do you need to reinitialize serde in every iteration?



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId) throws JsonProcessingException {
+    ObjectMapper mapper = new ObjectMapper();
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString = "";
+    for (Map.Entry<String, String> entry : map.entrySet()) {
+      if (entry.getValue() != null) {
+        String partition = String.format("%s=%s", String.valueOf(entry.getKey()), String.valueOf(entry.getValue()));

Review Comment:
   why do you need casts to String if those are already strings `Entry<String, String>`?



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

   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=4346)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=CODE_SMELL) [8 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1204289566


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1391,4 +1403,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn:", e);

Review Comment:
   Sure, i have updated the PR.  :) 



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

   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=4346)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=CODE_SMELL) [7 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -54,12 +55,17 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d
   @Override
   public int execute() throws HiveException {
     Table tbl = context.getDb().getTable(desc.getTabName());
-    if (!tbl.isPartitioned()) {
-      throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());
+    boolean isIcebergTable = isIcebergTable(tbl);
+    if (!(tbl.isPartitioned() || isIcebergTable)) {
+        throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());

Review Comment:
   this block is to handle cases when you call `show partitions` on non-partitioned table. no need to protect if there is no partition support especially when you added the default impl for showPartitions in storage handler



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

   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=4346)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=CODE_SMELL) [14 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1223242196


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId) throws JsonProcessingException {
+    ObjectMapper mapper = new ObjectMapper();

Review Comment:
   Fixed



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

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

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


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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

   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=4346)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=CODE_SMELL) [7 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -54,12 +55,17 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d
   @Override
   public int execute() throws HiveException {
     Table tbl = context.getDb().getTable(desc.getTabName());
-    if (!tbl.isPartitioned()) {
-      throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());
+    boolean isIcebergTable = isIcebergTable(tbl);
+    if (!(tbl.isPartitioned() || isIcebergTable)) {
+        throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());

Review Comment:
   this block is to handle cases when you call `show partitions` on non-partitioned table. no need to protect if there is no partition support, especially when you added the default impl for `showPartitions` in storage handler



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

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

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

   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=4346)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=CODE_SMELL) [7 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1226764039


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getSerializationProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId, ObjectMapper mapper)
+      throws JsonProcessingException {
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString =
+        map.entrySet().stream().filter(entry -> entry.getValue() != null).map(java.lang.Object::toString)
+            .collect(Collectors.joining("/"));
+    return String.format("Spec-id=%s/%s", specId, partString);
+  }
+
+  protected static JobConf getPartJobConf(Configuration confs, Path path,
+      org.apache.hadoop.hive.ql.metadata.Table tbl) {
+    JobConf job = new JobConf(confs);
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_MAPRED_INPUT_DIR, path.toString());
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_READ_COLUMN_NAMES_CONF_STR,
+        "partition,record_count,file_count," + "spec_id");

Review Comment:
   Intellij's auto-reformatter added it. Will update.



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1223240827


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId) throws JsonProcessingException {
+    ObjectMapper mapper = new ObjectMapper();
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString = "";
+    for (Map.Entry<String, String> entry : map.entrySet()) {
+      if (entry.getValue() != null) {
+        String partition = String.format("%s=%s", String.valueOf(entry.getKey()), String.valueOf(entry.getValue()));

Review Comment:
   Fixed



##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId) throws JsonProcessingException {
+    ObjectMapper mapper = new ObjectMapper();
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString = "";
+    for (Map.Entry<String, String> entry : map.entrySet()) {

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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -3697,7 +3697,6 @@ public static enum ConfVars {
         "A list of io exception handler class names. This is used\n" +
         "to construct a list exception handlers to handle exceptions thrown\n" +
         "by record readers"),
-

Review Comment:
   revert that



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

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

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


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out:
##########
@@ -0,0 +1,214 @@
+PREHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@hiveT1
+POSTHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@hiveT1
+PREHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@hivet1
+POSTHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@hivet1
+POSTHOOK: Output: default@hivet1@d_part=10/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=2/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=3/e_part=4
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).c SCRIPT []
+PREHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice1
+POSTHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice1
+PREHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice1
+POSTHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice1
+PREHOOK: query: show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+d_part=10/e_part=5
+d_part=2/e_part=5
+d_part=3/e_part=4
+PREHOOK: query: show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: select * from default.ice1.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice1.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: explain show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: explain show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"hiveT1"}
+
+PREHOOK: query: explain show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: explain show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"ice1"}
+
+PREHOOK: query: explain select * from default.ice1.partitions
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain select * from default.ice1.partitions
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Select Operator [SEL_1]
+      Output:["_col0","_col1","_col2","_col3"]
+      TableScan [TS_0]
+        Output:["partition","record_count","file_count","spec_id"]
+
+PREHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice2
+POSTHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice2
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+PREHOOK: type: ALTERTABLE_SETPARTSPEC
+PREHOOK: Input: default@ice2
+POSTHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+POSTHOOK: type: ALTERTABLE_SETPARTSPEC
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5
+Spec-id=0/d_part=2/e_part=5
+Spec-id=0/d_part=3/e_part=4
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+{"d_part":null,"e_part":null,"c":2}	2	1	1
+{"d_part":null,"e_part":null,"c":3}	1	1	1
+{"d_part":null,"e_part":null,"c":4}	1	1	1
+{"d_part":null,"e_part":null,"c":5}	1	1	1
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5

Review Comment:
   maybe
   ````
   (spec-id=0)/d_part=10/e_part=5
   (spec-id=0/d_part=2/e_part=5
   (spec-id=0)/d_part=3/e_part=4
   (spec-id=1)*/c=2
   (spec-id=1)*/c=3
   (spec-id=1)*/c=4
   (spec-id=1)*/c=5  
   ````



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId) throws JsonProcessingException {
+    ObjectMapper mapper = new ObjectMapper();

Review Comment:
   do we really need to create a new instance of ObjectMapper every time?
   PS: in the latest version of Jackson we could use light-weight and 100% thread-safe (immutable) ObjectReader instead of ObjectMapper



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId) throws JsonProcessingException {
+    ObjectMapper mapper = new ObjectMapper();

Review Comment:
   do we really need to create a new instance of ObjectMapper every 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] deniskuzZ commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getSerializationProps() {

Review Comment:
   why protected scope?



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1239689742


##########
iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out:
##########
@@ -0,0 +1,214 @@
+PREHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@hiveT1
+POSTHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@hiveT1
+PREHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@hivet1
+POSTHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@hivet1
+POSTHOOK: Output: default@hivet1@d_part=10/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=2/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=3/e_part=4
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).c SCRIPT []
+PREHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice1
+POSTHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice1
+PREHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice1
+POSTHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice1
+PREHOOK: query: show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+d_part=10/e_part=5
+d_part=2/e_part=5
+d_part=3/e_part=4
+PREHOOK: query: show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+{Spec-id=0}*:d_part=10/e_part=5
+{Spec-id=0}*:d_part=2/e_part=5
+{Spec-id=0}*:d_part=3/e_part=4
+PREHOOK: query: select * from default.ice1.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice1.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: explain show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: explain show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"hiveT1"}
+
+PREHOOK: query: explain show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: explain show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"ice1"}
+
+PREHOOK: query: explain select * from default.ice1.partitions
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain select * from default.ice1.partitions
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Select Operator [SEL_1]
+      Output:["_col0","_col1","_col2","_col3"]
+      TableScan [TS_0]
+        Output:["partition","record_count","file_count","spec_id"]
+
+PREHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice2
+POSTHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice2
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+{Spec-id=0}*:d_part=10/e_part=5
+{Spec-id=0}*:d_part=2/e_part=5
+{Spec-id=0}*:d_part=3/e_part=4
+PREHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+PREHOOK: type: ALTERTABLE_SETPARTSPEC
+PREHOOK: Input: default@ice2
+POSTHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+POSTHOOK: type: ALTERTABLE_SETPARTSPEC
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+{Spec-id=0}:d_part=10/e_part=5

Review Comment:
   LTER TABLE ice2 SET PARTITION SPEC (c) ;
   select * from default.ice2.partitions order by `partition`;
   show partitions ice2;
   ====================
   ```
   POSTHOOK: type: SHOWPARTITIONS
   POSTHOOK: Input: default@ice2
   {Spec-id=0}:d_part=10/e_part=5
   {Spec-id=0}:d_part=2/e_part=5
   {Spec-id=0}:d_part=3/e_part=4
   ```
   ====================
   insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5);
   select * from default.ice2.partitions order by `partition`;
   show partitions ice2;
   ====================
   ```
   POSTHOOK: Input: default@ice2
   {Spec-id=0}:d_part=10/e_part=5
   {Spec-id=0}:d_part=2/e_part=5
   {Spec-id=0}:d_part=3/e_part=4
   {Spec-id=1}*:c=2
   {Spec-id=1}*:c=3
   {Spec-id=1}*:c=4
   {Spec-id=1}*:c=5
   ```
   ===================
   The partition shows up in the iceberg.partitions table only after new rows are inserted to the table after the spec has changed.
   
   Updated the PR to the following as discussed:
   ```
   spec-id=0/d_part=10/e_part=5
   spec-id=0/d_part=2/e_part=5
   spec-id=0/d_part=3/e_part=4
   current-spec-id=1/c=2
   current-spec-id=1/c=3
   current-spec-id=1/c=4
   current-spec-id=1/c=5  
   
   ```



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1237580843


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -73,6 +80,7 @@ public class HiveTableUtil {
   private static final Logger LOG = LoggerFactory.getLogger(HiveTableUtil.class);
 
   static final String TABLE_EXTENSION = ".table";
+  static final String MAPRED_INPUT_DIR = "mapred.input.dir";

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] okumin commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -54,12 +55,17 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d
   @Override
   public int execute() throws HiveException {
     Table tbl = context.getDb().getTable(desc.getTabName());
-    if (!tbl.isPartitioned()) {
-      throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());
+    boolean isIcebergTable = isIcebergTable(tbl);
+    if (!(tbl.isPartitioned() || isIcebergTable)) {
+        throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());

Review Comment:
   Can we generalize this part rather than hardcoding ICEBERG? I guess this is useful for another storage handler.



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn: Unable to show partitions for iceberg table - ", e);
+    }
+    return parts;
+  }
+
+  @Override
+  public boolean supportsPartitions() {
+    return true;
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,
+      RecordReader<WritableComparable, Writable> reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws Exception {
+
+    List<String> parts = Lists.newArrayList();
+    Writable value = reader.createValue();
+    WritableComparable key = reader.createKey();
+    boolean notEoF = true;
+    String prevRow = "";
+
+    try (FetchFormatter fetcher = new DefaultFetchFormatter()) {
+      fetcher.initialize(job, HiveTableUtil.getProps());
+      org.apache.hadoop.hive.ql.metadata.Table metaDataPartTable =
+          context.getDb().getTable(hmstbl.getDbName(), hmstbl.getTableName(), "partitions", true);
+
+      while (notEoF) {
+        reader.next(key, value);

Review Comment:
   doesn't it return EOF marker?
   ````
   opNotEOF = reader.next(key, value);
   ````
   Deserializer currSerDe = metaDataPartTable.getDeserializer();
   while (recordReader.next(key, value)) {
       String[] row = fetcher.convert(
           currSerDe.deserialize(value), currSerDe.getObjectInspector()
       ).toString().split("\t");
       
       parts.add(HiveTableUtil.getParseData(row[PART_IDX], row[SPEC_IDX]));
   }



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -116,4 +122,9 @@ private List<String> getPartitionNames(Table tbl) throws HiveException {
         desc.getOrder(), desc.getLimit());
     return partNames;
   }
+
+  private boolean nonNativeTableSupportsPartitions(Table tbl) {

Review Comment:
   no need for this



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn: Unable to show partitions for iceberg table - ", e);
+    }
+    return parts;
+  }
+
+  @Override
+  public boolean supportsPartitions() {
+    return true;
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,
+      RecordReader<WritableComparable, Writable> reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws Exception {
+
+    List<String> parts = Lists.newArrayList();
+    Writable value = reader.createValue();
+    WritableComparable key = reader.createKey();
+    boolean notEoF = true;
+    String prevRow = "";
+
+    try (FetchFormatter fetcher = new DefaultFetchFormatter()) {
+      fetcher.initialize(job, HiveTableUtil.getProps());
+      org.apache.hadoop.hive.ql.metadata.Table metaDataPartTable =
+          context.getDb().getTable(hmstbl.getDbName(), hmstbl.getTableName(), "partitions", true);
+
+      while (notEoF) {
+        reader.next(key, value);
+        Deserializer currSerDe = metaDataPartTable.getDeserializer();
+        String[] row =
+            fetcher.convert(currSerDe.deserialize(value), currSerDe.getObjectInspector()).toString().split("\t");
+        if (prevRow.equalsIgnoreCase(row[PART_IDX])) {
+          notEoF = false;
+        } else {
+          prevRow = row[0];
+          parts.add(HiveTableUtil.getParseData(row[PART_IDX], row[SPEC_IDX]));
+        }
+      }
+    }
+    Collections.sort(parts);

Review Comment:
   maybe we should use TreeSet instead?



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getSerializationProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId, ObjectMapper mapper)
+      throws JsonProcessingException {
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString =
+        map.entrySet().stream().filter(entry -> entry.getValue() != null).map(java.lang.Object::toString)
+            .collect(Collectors.joining("/"));
+    return String.format("Spec-id=%s/%s", specId, partString);
+  }
+
+  protected static JobConf getPartJobConf(Configuration confs, Path path,
+      org.apache.hadoop.hive.ql.metadata.Table tbl) {
+    JobConf job = new JobConf(confs);
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_MAPRED_INPUT_DIR, path.toString());
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_READ_COLUMN_NAMES_CONF_STR,
+        "partition,record_count,file_count," + "spec_id");
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_ICEBERG_MR_TABLE_LOC, tbl.getPath().toString());
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_MAPRED_INPUT_DIR, path.toString());

Review Comment:
   why it's declared twice? line 253 and 257?



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getSerializationProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId, ObjectMapper mapper)
+      throws JsonProcessingException {
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString =
+        map.entrySet().stream().filter(entry -> entry.getValue() != null).map(java.lang.Object::toString)
+            .collect(Collectors.joining("/"));
+    return String.format("Spec-id=%s/%s", specId, partString);
+  }
+
+  protected static JobConf getPartJobConf(Configuration confs, Path path,
+      org.apache.hadoop.hive.ql.metadata.Table tbl) {
+    JobConf job = new JobConf(confs);
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_MAPRED_INPUT_DIR, path.toString());
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_READ_COLUMN_NAMES_CONF_STR,
+        "partition,record_count,file_count," + "spec_id");

Review Comment:
   why concatenate `spec_id ` and declare directly



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1227878020


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getSerializationProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId, ObjectMapper mapper)
+      throws JsonProcessingException {
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString =
+        map.entrySet().stream().filter(entry -> entry.getValue() != null).map(java.lang.Object::toString)
+            .collect(Collectors.joining("/"));
+    return String.format("Spec-id=%s/%s", specId, partString);
+  }
+
+  protected static JobConf getPartJobConf(Configuration confs, Path path,
+      org.apache.hadoop.hive.ql.metadata.Table tbl) {
+    JobConf job = new JobConf(confs);
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_MAPRED_INPUT_DIR, path.toString());
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_READ_COLUMN_NAMES_CONF_STR,
+        "partition,record_count,file_count," + "spec_id");
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_ICEBERG_MR_TABLE_LOC, tbl.getPath().toString());
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_MAPRED_INPUT_DIR, path.toString());

Review Comment:
   fixed



##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -159,9 +169,12 @@ public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, H
   private static final String TABLE_NAME_SEPARATOR = "..";
   private static final String ICEBERG = "iceberg";
   private static final String PUFFIN = "puffin";

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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out:
##########
@@ -0,0 +1,214 @@
+PREHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@hiveT1
+POSTHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@hiveT1
+PREHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@hivet1
+POSTHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@hivet1
+POSTHOOK: Output: default@hivet1@d_part=10/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=2/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=3/e_part=4
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).c SCRIPT []
+PREHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice1
+POSTHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice1
+PREHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice1
+POSTHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice1
+PREHOOK: query: show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+d_part=10/e_part=5
+d_part=2/e_part=5
+d_part=3/e_part=4
+PREHOOK: query: show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: select * from default.ice1.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice1.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: explain show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: explain show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"hiveT1"}
+
+PREHOOK: query: explain show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: explain show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"ice1"}
+
+PREHOOK: query: explain select * from default.ice1.partitions
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain select * from default.ice1.partitions
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Select Operator [SEL_1]
+      Output:["_col0","_col1","_col2","_col3"]
+      TableScan [TS_0]
+        Output:["partition","record_count","file_count","spec_id"]
+
+PREHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice2
+POSTHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice2
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+PREHOOK: type: ALTERTABLE_SETPARTSPEC
+PREHOOK: Input: default@ice2
+POSTHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+POSTHOOK: type: ALTERTABLE_SETPARTSPEC
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5
+Spec-id=0/d_part=2/e_part=5
+Spec-id=0/d_part=3/e_part=4
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+{"d_part":null,"e_part":null,"c":2}	2	1	1
+{"d_part":null,"e_part":null,"c":3}	1	1	1
+{"d_part":null,"e_part":null,"c":4}	1	1	1
+{"d_part":null,"e_part":null,"c":5}	1	1	1
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5

Review Comment:
   maybe
   ````
   d_part=10/e_part=5
   d_part=2/e_part=5
   d_part=3/e_part=4
   *|c=2
   *|c=3
   *|c=4
   *|c=5 
   ````



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out:
##########
@@ -0,0 +1,214 @@
+PREHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@hiveT1
+POSTHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@hiveT1
+PREHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@hivet1
+POSTHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@hivet1
+POSTHOOK: Output: default@hivet1@d_part=10/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=2/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=3/e_part=4
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).c SCRIPT []
+PREHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice1
+POSTHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice1
+PREHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice1
+POSTHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice1
+PREHOOK: query: show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+d_part=10/e_part=5
+d_part=2/e_part=5
+d_part=3/e_part=4
+PREHOOK: query: show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: select * from default.ice1.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice1.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: explain show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: explain show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"hiveT1"}
+
+PREHOOK: query: explain show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: explain show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"ice1"}
+
+PREHOOK: query: explain select * from default.ice1.partitions
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain select * from default.ice1.partitions
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Select Operator [SEL_1]
+      Output:["_col0","_col1","_col2","_col3"]
+      TableScan [TS_0]
+        Output:["partition","record_count","file_count","spec_id"]
+
+PREHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice2
+POSTHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice2
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+PREHOOK: type: ALTERTABLE_SETPARTSPEC
+PREHOOK: Input: default@ice2
+POSTHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+POSTHOOK: type: ALTERTABLE_SETPARTSPEC
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5
+Spec-id=0/d_part=2/e_part=5
+Spec-id=0/d_part=3/e_part=4
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+{"d_part":null,"e_part":null,"c":2}	2	1	1
+{"d_part":null,"e_part":null,"c":3}	1	1	1
+{"d_part":null,"e_part":null,"c":4}	1	1	1
+{"d_part":null,"e_part":null,"c":5}	1	1	1
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5

Review Comment:
   maybe
   ````
   {spec-id=0}/d_part=10/e_part=5
   {spec-id=0/d_part=2/e_part=5
   {spec-id=0}/d_part=3/e_part=4
   {spec-id=1}*/c=2
   {spec-id=1}*/c=3
   {spec-id=1}*/c=4
   {spec-id=1}*/c=5  
   ````



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1235688240


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1433,4 +1445,54 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws HiveException {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get(Constants.METADATA_LOCATION));
+    JobConf job = HiveTableUtil.getPartJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        return getParts(context, job, reader, hmstbl);
+      }
+    } catch (Exception e) {
+      throw new HiveException(e, ErrorMsg.GENERIC_ERROR,
+          "show partitions for table " + hmstbl.getTableName() + ". " + ErrorMsg.TABLE_NOT_PARTITIONED +
+              " or the table is empty ");
+    }
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,

Review Comment:
   Fixed



##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1433,4 +1445,54 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws HiveException {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get(Constants.METADATA_LOCATION));
+    JobConf job = HiveTableUtil.getPartJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        return getParts(context, job, reader, hmstbl);
+      }
+    } catch (Exception e) {
+      throw new HiveException(e, ErrorMsg.GENERIC_ERROR,
+          "show partitions for table " + hmstbl.getTableName() + ". " + ErrorMsg.TABLE_NOT_PARTITIONED +
+              " or the table is empty ");
+    }
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,
+      RecordReader<WritableComparable, Writable> reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws Exception {
+
+    List<String> parts = Lists.newArrayList();
+    Writable value = reader.createValue();
+    WritableComparable key = reader.createKey();
+
+    try (FetchFormatter fetcher = new DefaultFetchFormatter()) {
+      fetcher.initialize(job, HiveTableUtil.getSerializationProps());
+      org.apache.hadoop.hive.ql.metadata.Table metaDataPartTable =
+          context.getDb().getTable(hmstbl.getDbName(), hmstbl.getTableName(), "partitions", true);
+      Deserializer currSerDe = metaDataPartTable.getDeserializer();
+      ObjectMapper mapper = new ObjectMapper();
+      Table tbl = getTable(hmstbl);
+

Review Comment:
   Fixed



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

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

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


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn: Unable to show partitions for iceberg table - ", e);

Review Comment:
   should we throw an exception here with the error message? users won't even realize something went wrong when we return an empty result



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

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

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getSerializationProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId, ObjectMapper mapper)
+      throws JsonProcessingException {
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString =
+        map.entrySet().stream().filter(entry -> entry.getValue() != null).map(java.lang.Object::toString)
+            .collect(Collectors.joining("/"));
+    return String.format("Spec-id=%s/%s", specId, partString);
+  }
+
+  protected static JobConf getPartJobConf(Configuration confs, Path path,
+      org.apache.hadoop.hive.ql.metadata.Table tbl) {
+    JobConf job = new JobConf(confs);
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_MAPRED_INPUT_DIR, path.toString());
+    HiveConf.setVar(job, HiveConf.ConfVars.HIVE_READ_COLUMN_NAMES_CONF_STR,
+        "partition,record_count,file_count," + "spec_id");

Review Comment:
   why concatenate `spec_id ` instead declare directly



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -73,6 +80,7 @@ public class HiveTableUtil {
   private static final Logger LOG = LoggerFactory.getLogger(HiveTableUtil.class);
 
   static final String TABLE_EXTENSION = ".table";
+  static final String MAPRED_INPUT_DIR = "mapred.input.dir";

Review Comment:
   obsolete



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1237581876


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -3697,7 +3697,6 @@ public static enum ConfVars {
         "A list of io exception handler class names. This is used\n" +
         "to construct a list exception handlers to handle exceptions thrown\n" +
         "by record readers"),
-

Review Comment:
   fixed



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

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

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


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out:
##########
@@ -0,0 +1,214 @@
+PREHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@hiveT1
+POSTHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@hiveT1
+PREHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@hivet1
+POSTHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@hivet1
+POSTHOOK: Output: default@hivet1@d_part=10/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=2/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=3/e_part=4
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).c SCRIPT []
+PREHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice1
+POSTHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice1
+PREHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice1
+POSTHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice1
+PREHOOK: query: show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+d_part=10/e_part=5
+d_part=2/e_part=5
+d_part=3/e_part=4
+PREHOOK: query: show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+{Spec-id=0}*:d_part=10/e_part=5
+{Spec-id=0}*:d_part=2/e_part=5
+{Spec-id=0}*:d_part=3/e_part=4
+PREHOOK: query: select * from default.ice1.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice1.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: explain show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: explain show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"hiveT1"}
+
+PREHOOK: query: explain show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: explain show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"ice1"}
+
+PREHOOK: query: explain select * from default.ice1.partitions
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain select * from default.ice1.partitions
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Select Operator [SEL_1]
+      Output:["_col0","_col1","_col2","_col3"]
+      TableScan [TS_0]
+        Output:["partition","record_count","file_count","spec_id"]
+
+PREHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice2
+POSTHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice2
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+{Spec-id=0}*:d_part=10/e_part=5
+{Spec-id=0}*:d_part=2/e_part=5
+{Spec-id=0}*:d_part=3/e_part=4
+PREHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+PREHOOK: type: ALTERTABLE_SETPARTSPEC
+PREHOOK: Input: default@ice2
+POSTHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+POSTHOOK: type: ALTERTABLE_SETPARTSPEC
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+{Spec-id=0}:d_part=10/e_part=5

Review Comment:
   why there is no current spec marker?



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1237578069


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +233,30 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  static Properties getSerializationProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  static String getParseData(String parseData, String specId, ObjectMapper mapper, Integer currentSpecId)
+      throws JsonProcessingException {
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString =
+        map.entrySet().stream().filter(entry -> entry.getValue() != null).map(java.lang.Object::toString)

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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1223240509


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn: Unable to show partitions for iceberg table - ", e);
+    }
+    return parts;
+  }
+
+  @Override
+  public boolean supportsPartitions() {
+    return true;
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,
+      RecordReader<WritableComparable, Writable> reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws Exception {
+
+    List<String> parts = Lists.newArrayList();
+    Writable value = reader.createValue();
+    WritableComparable key = reader.createKey();
+    boolean notEoF = true;
+    String prevRow = "";
+
+    try (FetchFormatter fetcher = new DefaultFetchFormatter()) {
+      fetcher.initialize(job, HiveTableUtil.getProps());
+      org.apache.hadoop.hive.ql.metadata.Table metaDataPartTable =
+          context.getDb().getTable(hmstbl.getDbName(), hmstbl.getTableName(), "partitions", true);
+
+      while (notEoF) {
+        reader.next(key, value);

Review Comment:
   Nope, it doesn't return the EoF marker. 



##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getProps() {

Review Comment:
   sure



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1223241187


##########
ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java:
##########
@@ -619,4 +621,14 @@ default void prepareAlterTableEnvironmentContext(AbstractAlterTableDesc alterTab
   default Boolean hasAppendsOnly(org.apache.hadoop.hive.ql.metadata.Table hmsTable, SnapshotContext since) {
     return null;
   }
+
+  default List<String> showPartitions(DDLOperationContext context,
+      org.apache.hadoop.hive.ql.metadata.Table tbl) throws UnsupportedOperationException {
+    throw new UnsupportedOperationException("Storage handler does not support show partitions command");
+  }
+
+  default boolean supportsPartitions(){

Review Comment:
   Fixed



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

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

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


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -3697,7 +3700,9 @@ public static enum ConfVars {
         "A list of io exception handler class names. This is used\n" +
         "to construct a list exception handlers to handle exceptions thrown\n" +
         "by record readers"),
-
+    HIVE_MAPRED_INPUT_DIR("mapred.input.dir", "", "Path to input dir."),

Review Comment:
   it's in constants now, why wasn't it removed from here?



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1225915121


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId) throws JsonProcessingException {
+    ObjectMapper mapper = new ObjectMapper();
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString = "";
+    for (Map.Entry<String, String> entry : map.entrySet()) {
+      if (entry.getValue() != null) {
+        String partition = String.format("%s=%s", String.valueOf(entry.getKey()), String.valueOf(entry.getValue()));
+        partString = partString.length() > 0 ? partString + "/" + partition : partition;
+      }
+    }
+    return String.format("Spec-id=%s/%s", specId, partString);
+  }
+
+  protected static JobConf getJobConf(Configuration confs, Path path, org.apache.hadoop.hive.ql.metadata.Table tbl) {
+    JobConf job = new JobConf(confs);
+    job.set("mapred.input.dir", path.toString());

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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1227878757


##########
ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java:
##########
@@ -638,4 +640,10 @@ default void prepareAlterTableEnvironmentContext(AbstractAlterTableDesc alterTab
   default Boolean hasAppendsOnly(org.apache.hadoop.hive.ql.metadata.Table hmsTable, SnapshotContext since) {
     return null;
   }
+
+  default List<String> showPartitions(DDLOperationContext context,

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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java:
##########
@@ -638,4 +640,18 @@ default void prepareAlterTableEnvironmentContext(AbstractAlterTableDesc alterTab
   default Boolean hasAppendsOnly(org.apache.hadoop.hive.ql.metadata.Table hmsTable, SnapshotContext since) {
     return null;
   }
+
+  /**
+   * Checks if storage handler supports Show Partitions and returns a list of partitions
+   * @param context
+   * @param tbl
+   * @return List of partitions
+   * @throws UnsupportedOperationException
+   * @throws HiveException
+   */
+  default List<String> showPartitions(DDLOperationContext context,
+      org.apache.hadoop.hive.ql.metadata.Table tbl) throws UnsupportedOperationException, HiveException {

Review Comment:
   why do you throw HiveException here?



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out:
##########
@@ -0,0 +1,214 @@
+PREHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@hiveT1
+POSTHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@hiveT1
+PREHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@hivet1
+POSTHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@hivet1
+POSTHOOK: Output: default@hivet1@d_part=10/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=2/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=3/e_part=4
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).c SCRIPT []
+PREHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice1
+POSTHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice1
+PREHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice1
+POSTHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice1
+PREHOOK: query: show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+d_part=10/e_part=5
+d_part=2/e_part=5
+d_part=3/e_part=4
+PREHOOK: query: show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: select * from default.ice1.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice1.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: explain show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: explain show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"hiveT1"}
+
+PREHOOK: query: explain show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: explain show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"ice1"}
+
+PREHOOK: query: explain select * from default.ice1.partitions
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain select * from default.ice1.partitions
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Select Operator [SEL_1]
+      Output:["_col0","_col1","_col2","_col3"]
+      TableScan [TS_0]
+        Output:["partition","record_count","file_count","spec_id"]
+
+PREHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice2
+POSTHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice2
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+PREHOOK: type: ALTERTABLE_SETPARTSPEC
+PREHOOK: Input: default@ice2
+POSTHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+POSTHOOK: type: ALTERTABLE_SETPARTSPEC
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5
+Spec-id=0/d_part=2/e_part=5
+Spec-id=0/d_part=3/e_part=4
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+{"d_part":null,"e_part":null,"c":2}	2	1	1
+{"d_part":null,"e_part":null,"c":3}	1	1	1
+{"d_part":null,"e_part":null,"c":4}	1	1	1
+{"d_part":null,"e_part":null,"c":5}	1	1	1
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5

Review Comment:
   maybe
   ````
   d_part=10/e_part=5
   d_part=2/e_part=5
   d_part=3/e_part=4
   */c=2
   */c=3
   */c=4
   */c=5 
   ````



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out:
##########
@@ -0,0 +1,214 @@
+PREHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@hiveT1
+POSTHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@hiveT1
+PREHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@hivet1
+POSTHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@hivet1
+POSTHOOK: Output: default@hivet1@d_part=10/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=2/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=3/e_part=4
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).c SCRIPT []
+PREHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice1
+POSTHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice1
+PREHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice1
+POSTHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice1
+PREHOOK: query: show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+d_part=10/e_part=5
+d_part=2/e_part=5
+d_part=3/e_part=4
+PREHOOK: query: show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: select * from default.ice1.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice1.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: explain show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: explain show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"hiveT1"}
+
+PREHOOK: query: explain show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: explain show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"ice1"}
+
+PREHOOK: query: explain select * from default.ice1.partitions
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain select * from default.ice1.partitions
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Select Operator [SEL_1]
+      Output:["_col0","_col1","_col2","_col3"]
+      TableScan [TS_0]
+        Output:["partition","record_count","file_count","spec_id"]
+
+PREHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice2
+POSTHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice2
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+PREHOOK: type: ALTERTABLE_SETPARTSPEC
+PREHOOK: Input: default@ice2
+POSTHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+POSTHOOK: type: ALTERTABLE_SETPARTSPEC
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5
+Spec-id=0/d_part=2/e_part=5
+Spec-id=0/d_part=3/e_part=4
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+{"d_part":null,"e_part":null,"c":2}	2	1	1
+{"d_part":null,"e_part":null,"c":3}	1	1	1
+{"d_part":null,"e_part":null,"c":4}	1	1	1
+{"d_part":null,"e_part":null,"c":5}	1	1	1
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5

Review Comment:
   maybe
   ````
   {0}/d_part=10/e_part=5
   {0/d_part=2/e_part=5
   {0}/d_part=3/e_part=4
   {1}*/c=2
   {1}*/c=3
   {1}*/c=4
   {1}*/c=5 
   ````



-- 
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] okumin commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -54,12 +55,17 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d
   @Override
   public int execute() throws HiveException {
     Table tbl = context.getDb().getTable(desc.getTabName());
-    if (!tbl.isPartitioned()) {
-      throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());
+    boolean isIcebergTable = isIcebergTable(tbl);
+    if (!(tbl.isPartitioned() || isIcebergTable)) {
+        throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());

Review Comment:
   Haha. We've been developing a tailored storage with some similar concepts to Iceberg for 10 years. We really like recent changes for Iceberg because it would give us a chance to move our hacks to standard APIs of storage handlers. Let me check newly added APIs and propose better generalization.



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn: Unable to show partitions for iceberg table - ", e);

Review Comment:
   if there are no partitions you should throw `new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED`



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1235695433


##########
iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out:
##########
@@ -0,0 +1,214 @@
+PREHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@hiveT1
+POSTHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@hiveT1
+PREHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@hivet1
+POSTHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@hivet1
+POSTHOOK: Output: default@hivet1@d_part=10/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=2/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=3/e_part=4
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).c SCRIPT []
+PREHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice1
+POSTHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice1
+PREHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice1
+POSTHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice1
+PREHOOK: query: show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+d_part=10/e_part=5
+d_part=2/e_part=5
+d_part=3/e_part=4
+PREHOOK: query: show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: select * from default.ice1.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice1.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: explain show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: explain show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"hiveT1"}
+
+PREHOOK: query: explain show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: explain show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"ice1"}
+
+PREHOOK: query: explain select * from default.ice1.partitions
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain select * from default.ice1.partitions
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Select Operator [SEL_1]
+      Output:["_col0","_col1","_col2","_col3"]
+      TableScan [TS_0]
+        Output:["partition","record_count","file_count","spec_id"]
+
+PREHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice2
+POSTHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice2
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+PREHOOK: type: ALTERTABLE_SETPARTSPEC
+PREHOOK: Input: default@ice2
+POSTHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+POSTHOOK: type: ALTERTABLE_SETPARTSPEC
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5
+Spec-id=0/d_part=2/e_part=5
+Spec-id=0/d_part=3/e_part=4
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+{"d_part":null,"e_part":null,"c":2}	2	1	1
+{"d_part":null,"e_part":null,"c":3}	1	1	1
+{"d_part":null,"e_part":null,"c":4}	1	1	1
+{"d_part":null,"e_part":null,"c":5}	1	1	1
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5

Review Comment:
   I think it will get confusing after some time when the number of partitions-evolution increase. I think it would be good to retain spec id with "*" to show the current spec id. 



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1204290594


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -54,12 +55,17 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d
   @Override
   public int execute() throws HiveException {
     Table tbl = context.getDb().getTable(desc.getTabName());
-    if (!tbl.isPartitioned()) {
-      throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());
+    boolean isIcebergTable = isIcebergTable(tbl);
+    if (!(tbl.isPartitioned() || isIcebergTable)) {
+        throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());

Review Comment:
   > Can we generalize this part rather than hardcoding ICEBERG? I guess this is useful for another storage handler.
   
   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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();

Review Comment:
   no need to initialize the partitions



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java:
##########
@@ -619,4 +621,14 @@ default void prepareAlterTableEnvironmentContext(AbstractAlterTableDesc alterTab
   default Boolean hasAppendsOnly(org.apache.hadoop.hive.ql.metadata.Table hmsTable, SnapshotContext since) {
     return null;
   }
+
+  default List<String> showPartitions(DDLOperationContext context,
+      org.apache.hadoop.hive.ql.metadata.Table tbl) throws UnsupportedOperationException {
+    throw new UnsupportedOperationException("Storage handler does not support show partitions command");
+  }
+
+  default boolean supportsPartitions(){

Review Comment:
   why do we need this? it's ok to call `default` showPartitions that might return `UnsupportedOperationException`



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

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

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

   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=4346)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=CODE_SMELL) [7 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1226760029


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -54,12 +54,12 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d
   @Override
   public int execute() throws HiveException {
     Table tbl = context.getDb().getTable(desc.getTabName());
-    if (!tbl.isPartitioned()) {

Review Comment:
   tbl.isPartitioned() evaluates to false for iceberg tables



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -3697,7 +3700,9 @@ public static enum ConfVars {
         "A list of io exception handler class names. This is used\n" +
         "to construct a list exception handlers to handle exceptions thrown\n" +
         "by record readers"),
-
+    HIVE_MAPRED_INPUT_DIR("mapred.input.dir", "", "Path to input dir."),

Review Comment:
   move it to HiveTableUtils constants



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +233,30 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  static Properties getSerializationProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  static String getParseData(String parseData, String specId, ObjectMapper mapper, Integer currentSpecId)
+      throws JsonProcessingException {
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString =
+        map.entrySet().stream().filter(entry -> entry.getValue() != null).map(java.lang.Object::toString)
+            .collect(Collectors.joining("/"));
+    String currentSpecMarker = currentSpecId.toString().equals(specId) ? "*" : "";
+    return String.format("Spec-id=%s%s/%s", specId, currentSpecMarker, partString);
+  }
+
+  static JobConf getPartJobConf(Configuration confs, Path path, org.apache.hadoop.hive.ql.metadata.Table tbl) {
+    JobConf job = new JobConf(confs);
+    job.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "partition,record_count,file_count,spec_id");

Review Comment:
   move to constants PARTITION_SCHEMA



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1225915102


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();

Review Comment:
   Fixed



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

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

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


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java:
##########
@@ -638,4 +640,10 @@ default void prepareAlterTableEnvironmentContext(AbstractAlterTableDesc alterTab
   default Boolean hasAppendsOnly(org.apache.hadoop.hive.ql.metadata.Table hmsTable, SnapshotContext since) {
     return null;
   }
+
+  default List<String> showPartitions(DDLOperationContext context,

Review Comment:
   please add a javadoc, this is an interface



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out:
##########
@@ -0,0 +1,214 @@
+PREHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@hiveT1
+POSTHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@hiveT1
+PREHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@hivet1
+POSTHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@hivet1
+POSTHOOK: Output: default@hivet1@d_part=10/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=2/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=3/e_part=4
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).c SCRIPT []
+PREHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice1
+POSTHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice1
+PREHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice1
+POSTHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice1
+PREHOOK: query: show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+d_part=10/e_part=5
+d_part=2/e_part=5
+d_part=3/e_part=4
+PREHOOK: query: show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: select * from default.ice1.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice1.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: explain show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: explain show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"hiveT1"}
+
+PREHOOK: query: explain show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: explain show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"ice1"}
+
+PREHOOK: query: explain select * from default.ice1.partitions
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain select * from default.ice1.partitions
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Select Operator [SEL_1]
+      Output:["_col0","_col1","_col2","_col3"]
+      TableScan [TS_0]
+        Output:["partition","record_count","file_count","spec_id"]
+
+PREHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice2
+POSTHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice2
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+PREHOOK: type: ALTERTABLE_SETPARTSPEC
+PREHOOK: Input: default@ice2
+POSTHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+POSTHOOK: type: ALTERTABLE_SETPARTSPEC
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5
+Spec-id=0/d_part=2/e_part=5
+Spec-id=0/d_part=3/e_part=4
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+{"d_part":null,"e_part":null,"c":2}	2	1	1
+{"d_part":null,"e_part":null,"c":3}	1	1	1
+{"d_part":null,"e_part":null,"c":4}	1	1	1
+{"d_part":null,"e_part":null,"c":5}	1	1	1
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5

Review Comment:
   maybe
   ````
   /d_part=10/e_part=5
   /d_part=2/e_part=5
   /d_part=3/e_part=4
   */c=2
   */c=3
   */c=4
   */c=5 
   ````



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1230995088


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -54,12 +54,12 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d
   @Override
   public int execute() throws HiveException {
     Table tbl = context.getDb().getTable(desc.getTabName());
-    if (!tbl.isPartitioned()) {

Review Comment:
   https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java#L612
   
   As discussed,  List<FieldSchema> partKeys = tTable.getPartitionKeys();  returns null for iceberg.
   
   I will raise a follow up jira to address this.



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

   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=4346)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=CODE_SMELL) [7 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

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

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


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -2207,6 +2207,9 @@ public static enum ConfVars {
         "Whether to use codec pool in ORC. Disable if there are bugs with codec reuse."),
     HIVE_ICEBERG_STATS_SOURCE("hive.iceberg.stats.source", "iceberg",
         "Use stats from iceberg table snapshot for query planning. This has two values metastore and iceberg"),
+    HIVE_ICEBERG_MR_TABLE_LOC("iceberg.mr.table.location", "", "Iceberg table location"),

Review Comment:
   why is this part of HiveConf, should we declare them in InputFormatConfig?



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn: Unable to show partitions for iceberg table - ", e);
+    }
+    return parts;
+  }
+
+  @Override
+  public boolean supportsPartitions() {
+    return true;
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,
+      RecordReader<WritableComparable, Writable> reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws Exception {
+
+    List<String> parts = Lists.newArrayList();
+    Writable value = reader.createValue();
+    WritableComparable key = reader.createKey();
+    boolean notEoF = true;
+    String prevRow = "";
+
+    try (FetchFormatter fetcher = new DefaultFetchFormatter()) {
+      fetcher.initialize(job, HiveTableUtil.getProps());
+      org.apache.hadoop.hive.ql.metadata.Table metaDataPartTable =
+          context.getDb().getTable(hmstbl.getDbName(), hmstbl.getTableName(), "partitions", true);
+
+      while (notEoF) {
+        reader.next(key, value);

Review Comment:
   next should return false if there is nothing remained



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -54,12 +54,12 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d
   @Override
   public int execute() throws HiveException {
     Table tbl = context.getDb().getTable(desc.getTabName());
-    if (!tbl.isPartitioned()) {
-      throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());
-    }
-
     List<String> parts;
-    if (desc.getCond() != null || desc.getOrder() != null) {
+    if(tbl!=null && tbl.isNonNative() && tbl.getStorageHandler().supportsPartitionTransform()){

Review Comment:
   formatting + can we have a null tbl here? 



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -54,12 +54,12 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d
   @Override
   public int execute() throws HiveException {
     Table tbl = context.getDb().getTable(desc.getTabName());
-    if (!tbl.isPartitioned()) {

Review Comment:
   why did we move this? Doesn't it work for iceberg tables, we should have a partition spec in HMS?



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1223240156


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn: Unable to show partitions for iceberg table - ", e);
+    }
+    return parts;
+  }
+
+  @Override
+  public boolean supportsPartitions() {
+    return true;
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,
+      RecordReader<WritableComparable, Writable> reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws Exception {
+
+    List<String> parts = Lists.newArrayList();
+    Writable value = reader.createValue();
+    WritableComparable key = reader.createKey();
+    boolean notEoF = true;
+    String prevRow = "";
+
+    try (FetchFormatter fetcher = new DefaultFetchFormatter()) {
+      fetcher.initialize(job, HiveTableUtil.getProps());
+      org.apache.hadoop.hive.ql.metadata.Table metaDataPartTable =
+          context.getDb().getTable(hmstbl.getDbName(), hmstbl.getTableName(), "partitions", true);
+
+      while (notEoF) {
+        reader.next(key, value);
+        Deserializer currSerDe = metaDataPartTable.getDeserializer();

Review Comment:
   fixed



##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn: Unable to show partitions for iceberg table - ", e);
+    }
+    return parts;
+  }
+
+  @Override
+  public boolean supportsPartitions() {
+    return true;
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,
+      RecordReader<WritableComparable, Writable> reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws Exception {
+
+    List<String> parts = Lists.newArrayList();
+    Writable value = reader.createValue();
+    WritableComparable key = reader.createKey();
+    boolean notEoF = true;
+    String prevRow = "";
+
+    try (FetchFormatter fetcher = new DefaultFetchFormatter()) {
+      fetcher.initialize(job, HiveTableUtil.getProps());
+      org.apache.hadoop.hive.ql.metadata.Table metaDataPartTable =
+          context.getDb().getTable(hmstbl.getDbName(), hmstbl.getTableName(), "partitions", true);
+
+      while (notEoF) {
+        reader.next(key, value);
+        Deserializer currSerDe = metaDataPartTable.getDeserializer();
+        String[] row =
+            fetcher.convert(currSerDe.deserialize(value), currSerDe.getObjectInspector()).toString().split("\t");
+        if (prevRow.equalsIgnoreCase(row[PART_IDX])) {
+          notEoF = false;
+        } else {
+          prevRow = row[0];
+          parts.add(HiveTableUtil.getParseData(row[PART_IDX], row[SPEC_IDX]));
+        }
+      }
+    }
+    Collections.sort(parts);

Review Comment:
   Iceberg doesn't support a tree set, the iceberg audit throws an expectation when we 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] aturoczy commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -54,12 +55,17 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d
   @Override
   public int execute() throws HiveException {
     Table tbl = context.getDb().getTable(desc.getTabName());
-    if (!tbl.isPartitioned()) {
-      throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());
+    boolean isIcebergTable = isIcebergTable(tbl);
+    if (!(tbl.isPartitioned() || isIcebergTable)) {
+        throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());

Review Comment:
   > Haha. We've been developing a tailored storage with some similar concepts to Iceberg for 10 years. We really like recent changes for Iceberg because it would give us a chance to move our hacks to standard APIs of storage handlers. Let me check newly added APIs and propose better generalization.
   
   Yes, Iceberg has a very elegant API and solutions. I like it, also the concept removes lot of hacks from the hive side as well.



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -157,9 +166,12 @@ public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, H
   private static final String TABLE_NAME_SEPARATOR = "..";
   private static final String ICEBERG = "iceberg";

Review Comment:
   we can remove this and use the Constants



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -54,12 +54,12 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d
   @Override
   public int execute() throws HiveException {
     Table tbl = context.getDb().getTable(desc.getTabName());
-    if (!tbl.isPartitioned()) {

Review Comment:
   we do have part spec in HMS, so why do we return false for Iceberg?



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out:
##########
@@ -0,0 +1,214 @@
+PREHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@hiveT1
+POSTHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@hiveT1
+PREHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@hivet1
+POSTHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@hivet1
+POSTHOOK: Output: default@hivet1@d_part=10/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=2/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=3/e_part=4
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).c SCRIPT []
+PREHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice1
+POSTHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice1
+PREHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice1
+POSTHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice1
+PREHOOK: query: show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+d_part=10/e_part=5
+d_part=2/e_part=5
+d_part=3/e_part=4
+PREHOOK: query: show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: select * from default.ice1.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice1.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: explain show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: explain show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"hiveT1"}
+
+PREHOOK: query: explain show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: explain show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"ice1"}
+
+PREHOOK: query: explain select * from default.ice1.partitions
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain select * from default.ice1.partitions
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Select Operator [SEL_1]
+      Output:["_col0","_col1","_col2","_col3"]
+      TableScan [TS_0]
+        Output:["partition","record_count","file_count","spec_id"]
+
+PREHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice2
+POSTHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice2
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0*/d_part=10/e_part=5
+Spec-id=0*/d_part=2/e_part=5
+Spec-id=0*/d_part=3/e_part=4
+PREHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+PREHOOK: type: ALTERTABLE_SETPARTSPEC
+PREHOOK: Input: default@ice2
+POSTHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+POSTHOOK: type: ALTERTABLE_SETPARTSPEC
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5
+Spec-id=0/d_part=2/e_part=5
+Spec-id=0/d_part=3/e_part=4
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+{"d_part":null,"e_part":null,"c":2}	2	1	1
+{"d_part":null,"e_part":null,"c":3}	1	1	1
+{"d_part":null,"e_part":null,"c":4}	1	1	1
+{"d_part":null,"e_part":null,"c":5}	1	1	1
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+Spec-id=0/d_part=10/e_part=5

Review Comment:
   maybe
   ````
   d_part=10/e_part=5
   d_part=2/e_part=5
   d_part=3/e_part=4
   *c=2
   *c=3
   *c=4
   *c=5 
   ````



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1433,4 +1445,54 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws HiveException {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get(Constants.METADATA_LOCATION));
+    JobConf job = HiveTableUtil.getPartJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        return getParts(context, job, reader, hmstbl);
+      }
+    } catch (Exception e) {
+      throw new HiveException(e, ErrorMsg.GENERIC_ERROR,
+          "show partitions for table " + hmstbl.getTableName() + ". " + ErrorMsg.TABLE_NOT_PARTITIONED +
+              " or the table is empty ");
+    }
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,
+      RecordReader<WritableComparable, Writable> reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws Exception {
+
+    List<String> parts = Lists.newArrayList();
+    Writable value = reader.createValue();
+    WritableComparable key = reader.createKey();
+
+    try (FetchFormatter fetcher = new DefaultFetchFormatter()) {
+      fetcher.initialize(job, HiveTableUtil.getSerializationProps());
+      org.apache.hadoop.hive.ql.metadata.Table metaDataPartTable =
+          context.getDb().getTable(hmstbl.getDbName(), hmstbl.getTableName(), "partitions", true);
+      Deserializer currSerDe = metaDataPartTable.getDeserializer();
+      ObjectMapper mapper = new ObjectMapper();
+      Table tbl = getTable(hmstbl);
+
+
+      while (reader.next(key, value)) {
+        String[] row =
+            fetcher.convert(currSerDe.deserialize(value), currSerDe.getObjectInspector()).toString().split("\t");

Review Comment:
   could we move `toString` to the next line



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1433,4 +1445,54 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws HiveException {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get(Constants.METADATA_LOCATION));
+    JobConf job = HiveTableUtil.getPartJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        return getParts(context, job, reader, hmstbl);
+      }
+    } catch (Exception e) {
+      throw new HiveException(e, ErrorMsg.GENERIC_ERROR,
+          "show partitions for table " + hmstbl.getTableName() + ". " + ErrorMsg.TABLE_NOT_PARTITIONED +
+              " or the table is empty ");
+    }
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,
+      RecordReader<WritableComparable, Writable> reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws Exception {
+
+    List<String> parts = Lists.newArrayList();
+    Writable value = reader.createValue();
+    WritableComparable key = reader.createKey();
+
+    try (FetchFormatter fetcher = new DefaultFetchFormatter()) {
+      fetcher.initialize(job, HiveTableUtil.getSerializationProps());
+      org.apache.hadoop.hive.ql.metadata.Table metaDataPartTable =
+          context.getDb().getTable(hmstbl.getDbName(), hmstbl.getTableName(), "partitions", true);
+      Deserializer currSerDe = metaDataPartTable.getDeserializer();
+      ObjectMapper mapper = new ObjectMapper();
+      Table tbl = getTable(hmstbl);
+

Review Comment:
   why so many new lines?



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1235688990


##########
ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java:
##########
@@ -638,4 +640,18 @@ default void prepareAlterTableEnvironmentContext(AbstractAlterTableDesc alterTab
   default Boolean hasAppendsOnly(org.apache.hadoop.hive.ql.metadata.Table hmsTable, SnapshotContext since) {
     return null;
   }
+
+  /**
+   * Checks if storage handler supports Show Partitions and returns a list of partitions
+   * @param context
+   * @param tbl
+   * @return List of partitions
+   * @throws UnsupportedOperationException
+   * @throws HiveException
+   */
+  default List<String> showPartitions(DDLOperationContext context,
+      org.apache.hadoop.hive.ql.metadata.Table tbl) throws UnsupportedOperationException, HiveException {

Review Comment:
   To indicate if table is not partitioned or table is empty :
   https://github.com/apache/hive/pull/4346/files#diff-93864ecf035fe51b92185015da842a56837cea89064813de39c278c6f8fed03cR1464 



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

   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=4346)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=CODE_SMELL) [7 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

   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=4346)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=CODE_SMELL) [7 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1230996923


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -2207,6 +2207,9 @@ public static enum ConfVars {
         "Whether to use codec pool in ORC. Disable if there are bugs with codec reuse."),
     HIVE_ICEBERG_STATS_SOURCE("hive.iceberg.stats.source", "iceberg",
         "Use stats from iceberg table snapshot for query planning. This has two values metastore and iceberg"),
+    HIVE_ICEBERG_MR_TABLE_LOC("iceberg.mr.table.location", "", "Iceberg table location"),

Review Comment:
   Removed from hive conf in the updated PR.



-- 
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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1230964271


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -3697,7 +3700,9 @@ public static enum ConfVars {
         "A list of io exception handler class names. This is used\n" +
         "to construct a list exception handlers to handle exceptions thrown\n" +
         "by record readers"),
-
+    HIVE_MAPRED_INPUT_DIR("mapred.input.dir", "", "Path to input dir."),

Review Comment:
   Fixed. 



##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -157,9 +166,12 @@ public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, H
   private static final String TABLE_NAME_SEPARATOR = "..";
   private static final String ICEBERG = "iceberg";

Review Comment:
   Done, moved it to Constants



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

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

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


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -3697,7 +3700,9 @@ public static enum ConfVars {
         "A list of io exception handler class names. This is used\n" +
         "to construct a list exception handlers to handle exceptions thrown\n" +
         "by record readers"),
-
+    HIVE_MAPRED_INPUT_DIR("mapred.input.dir", "", "Path to input dir."),

Review Comment:
   move it to HiveTableUtils constants, that is not a hive prop



-- 
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] aturoczy commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -54,12 +55,17 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d
   @Override
   public int execute() throws HiveException {
     Table tbl = context.getDb().getTable(desc.getTabName());
-    if (!tbl.isPartitioned()) {
-      throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());
+    boolean isIcebergTable = isIcebergTable(tbl);
+    if (!(tbl.isPartitioned() || isIcebergTable)) {
+        throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());

Review Comment:
   fair point



-- 
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] aturoczy commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1391,4 +1403,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn:", e);

Review Comment:
   After warn needs a space, also it would be nice if it would describe. Which exception would you like to catch, to catch everything is by default not the best practice. (OK just for warning could be fine, but it is more elegant to provide practical messages) 



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();
+    JobConf job = HiveTableUtil.getJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        parts = getParts(context, job, reader, hmstbl);
+      }
+      return parts;
+    } catch (Exception e) {
+      LOG.warn("Warn: Unable to show partitions for iceberg table - ", e);
+    }
+    return parts;
+  }
+
+  @Override
+  public boolean supportsPartitions() {
+    return true;
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,
+      RecordReader<WritableComparable, Writable> reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws Exception {
+
+    List<String> parts = Lists.newArrayList();
+    Writable value = reader.createValue();
+    WritableComparable key = reader.createKey();
+    boolean notEoF = true;
+    String prevRow = "";
+
+    try (FetchFormatter fetcher = new DefaultFetchFormatter()) {

Review Comment:
   not sure, but can't we use FetchTask for this?



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1408,4 +1420,64 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get("metadata_location"));
+    List<String> parts = Lists.newArrayList();

Review Comment:
   no need to initialize the partitions, you can just return result of `getParts`



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId) throws JsonProcessingException {
+    ObjectMapper mapper = new ObjectMapper();
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString = "";
+    for (Map.Entry<String, String> entry : map.entrySet()) {
+      if (entry.getValue() != null) {
+        String partition = String.format("%s=%s", String.valueOf(entry.getKey()), String.valueOf(entry.getValue()));
+        partString = partString.length() > 0 ? partString + "/" + partition : partition;
+      }
+    }
+    return String.format("Spec-id=%s/%s", specId, partString);
+  }
+
+  protected static JobConf getJobConf(Configuration confs, Path path, org.apache.hadoop.hive.ql.metadata.Table tbl) {
+    JobConf job = new JobConf(confs);
+    job.set("mapred.input.dir", path.toString());

Review Comment:
   could we use protective HiveConf.setxx() instead of job.set?



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +231,34 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  protected static Properties getProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  protected static String getParseData(String parseData, String specId) throws JsonProcessingException {
+    ObjectMapper mapper = new ObjectMapper();
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString = "";
+    for (Map.Entry<String, String> entry : map.entrySet()) {

Review Comment:
   could be simplified to
   ````
   map.entrySet().stream()
       .filter(entry -> entry.getValue() != null)
       .map( Object::toString )
       .collect( Collectors.joining( "/" ) );
   ````



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1433,4 +1445,54 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws HiveException {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get(Constants.METADATA_LOCATION));
+    JobConf job = HiveTableUtil.getPartJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        return getParts(context, job, reader, hmstbl);
+      }
+    } catch (Exception e) {
+      throw new HiveException(e, ErrorMsg.GENERIC_ERROR,
+          "show partitions for table " + hmstbl.getTableName() + ". " + ErrorMsg.TABLE_NOT_PARTITIONED +
+              " or the table is empty ");
+    }
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,

Review Comment:
   getPartitions()



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

   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=4346)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=CODE_SMELL) [51 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java:
##########
@@ -1433,4 +1445,54 @@ Boolean hasAppendsOnly(Iterable<Snapshot> snapshots, SnapshotContext since) {
 
     return null;
   }
+
+  @Override
+  public List<String> showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws HiveException {
+    Configuration confs = context.getConf();
+    Path path = new Path(hmstbl.getParameters().get(Constants.METADATA_LOCATION));
+    JobConf job = HiveTableUtil.getPartJobConf(confs, path, hmstbl);
+    Class<? extends InputFormat> formatter = hmstbl.getInputFormatClass();
+
+    try {
+      InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job);
+      InputSplit[] splits = inputFormat.getSplits(job, 1);
+      try (RecordReader<WritableComparable, Writable> reader = inputFormat.getRecordReader(splits[0], job,
+          Reporter.NULL)) {
+        return getParts(context, job, reader, hmstbl);
+      }
+    } catch (Exception e) {
+      throw new HiveException(e, ErrorMsg.GENERIC_ERROR,
+          "show partitions for table " + hmstbl.getTableName() + ". " + ErrorMsg.TABLE_NOT_PARTITIONED +
+              " or the table is empty ");
+    }
+  }
+
+  private List<String> getParts(DDLOperationContext context, Configuration job,
+      RecordReader<WritableComparable, Writable> reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl)
+      throws Exception {
+
+    List<String> parts = Lists.newArrayList();
+    Writable value = reader.createValue();
+    WritableComparable key = reader.createKey();
+
+    try (FetchFormatter fetcher = new DefaultFetchFormatter()) {
+      fetcher.initialize(job, HiveTableUtil.getSerializationProps());
+      org.apache.hadoop.hive.ql.metadata.Table metaDataPartTable =
+          context.getDb().getTable(hmstbl.getDbName(), hmstbl.getTableName(), "partitions", true);
+      Deserializer currSerDe = metaDataPartTable.getDeserializer();
+      ObjectMapper mapper = new ObjectMapper();
+      Table tbl = getTable(hmstbl);
+
+
+      while (reader.next(key, value)) {
+        String[] row =
+            fetcher.convert(currSerDe.deserialize(value), currSerDe.getObjectInspector()).toString().split("\t");

Review Comment:
   could we move `toString` to the next line
   .toString().split(Utilities.tabCode)



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java:
##########
@@ -225,4 +233,30 @@ public static boolean isCtas(Properties properties) {
     return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS));
   }
 
+  static Properties getSerializationProps() {
+    Properties props = new Properties();
+    props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode);
+    props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL");
+    return props;
+  }
+
+  static String getParseData(String parseData, String specId, ObjectMapper mapper, Integer currentSpecId)
+      throws JsonProcessingException {
+    Map<String, String> map = mapper.readValue(parseData, Map.class);
+    String partString =
+        map.entrySet().stream().filter(entry -> entry.getValue() != null).map(java.lang.Object::toString)

Review Comment:
   formatting, put filter and map on next separate lines



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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


##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java:
##########
@@ -54,12 +54,12 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d
   @Override
   public int execute() throws HiveException {
     Table tbl = context.getDb().getTable(desc.getTabName());
-    if (!tbl.isPartitioned()) {
-      throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName());
-    }
-
     List<String> parts;
-    if (desc.getCond() != null || desc.getOrder() != null) {
+    if(tbl!=null && tbl.isNonNative() && tbl.getStorageHandler().supportsPartitionTransform()){

Review Comment:
   formatting



-- 
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 #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

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

   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=4346)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4346&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=4346&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=4346&resolved=false&types=CODE_SMELL) [7 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4346&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=4346&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=4346&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] simhadri-g commented on a diff in pull request #4346: HIVE-26832: Implement SHOW PARTITIONS for Iceberg Tables

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4346:
URL: https://github.com/apache/hive/pull/4346#discussion_r1239689742


##########
iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out:
##########
@@ -0,0 +1,214 @@
+PREHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@hiveT1
+POSTHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@hiveT1
+PREHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@hivet1
+POSTHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@hivet1
+POSTHOOK: Output: default@hivet1@d_part=10/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=2/e_part=5
+POSTHOOK: Output: default@hivet1@d_part=3/e_part=4
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=10,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).c SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).a SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).b SCRIPT []
+POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).c SCRIPT []
+PREHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice1
+POSTHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice1
+PREHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice1
+POSTHOOK: query: insert into ice1 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice1
+PREHOOK: query: show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+d_part=10/e_part=5
+d_part=2/e_part=5
+d_part=3/e_part=4
+PREHOOK: query: show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+{Spec-id=0}*:d_part=10/e_part=5
+{Spec-id=0}*:d_part=2/e_part=5
+{Spec-id=0}*:d_part=3/e_part=4
+PREHOOK: query: select * from default.ice1.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice1.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: explain show partitions hiveT1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@hivet1
+POSTHOOK: query: explain show partitions hiveT1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@hivet1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"hiveT1"}
+
+PREHOOK: query: explain show partitions ice1
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice1
+POSTHOOK: query: explain show partitions ice1
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice1
+Stage-1
+  Fetch Operator
+    limit:-1
+    Stage-0
+      Show Partitions{"limit:":"-1","table:":"ice1"}
+
+PREHOOK: query: explain select * from default.ice1.partitions
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice1
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain select * from default.ice1.partitions
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Select Operator [SEL_1]
+      Output:["_col0","_col1","_col2","_col3"]
+      TableScan [TS_0]
+        Output:["partition","record_count","file_count","spec_id"]
+
+PREHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice2
+POSTHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice2
+PREHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice2
+POSTHOOK: query: insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5}	2	1	0
+{"d_part":2,"e_part":5}	1	1	0
+{"d_part":3,"e_part":4}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+{Spec-id=0}*:d_part=10/e_part=5
+{Spec-id=0}*:d_part=2/e_part=5
+{Spec-id=0}*:d_part=3/e_part=4
+PREHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+PREHOOK: type: ALTERTABLE_SETPARTSPEC
+PREHOOK: Input: default@ice2
+POSTHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c)
+POSTHOOK: type: ALTERTABLE_SETPARTSPEC
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: default@ice2
+PREHOOK: query: select * from default.ice2.partitions order by `partition`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice2
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from default.ice2.partitions order by `partition`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice2
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+{"d_part":10,"e_part":5,"c":null}	2	1	0
+{"d_part":2,"e_part":5,"c":null}	1	1	0
+{"d_part":3,"e_part":4,"c":null}	2	1	0
+PREHOOK: query: show partitions ice2
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ice2
+POSTHOOK: query: show partitions ice2
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ice2
+{Spec-id=0}:d_part=10/e_part=5

Review Comment:
   
   ALTER TABLE ice2 SET PARTITION SPEC (c) ;
   select * from default.ice2.partitions order by `partition`;
   show partitions ice2;
   
   ```
   POSTHOOK: type: SHOWPARTITIONS
   POSTHOOK: Input: default@ice2
   {Spec-id=0}:d_part=10/e_part=5
   {Spec-id=0}:d_part=2/e_part=5
   {Spec-id=0}:d_part=3/e_part=4
   ```
   
   insert into ice2 values  ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5);
   select * from default.ice2.partitions order by `partition`;
   show partitions ice2;
   
   ```
   POSTHOOK: Input: default@ice2
   {Spec-id=0}:d_part=10/e_part=5
   {Spec-id=0}:d_part=2/e_part=5
   {Spec-id=0}:d_part=3/e_part=4
   {Spec-id=1}*:c=2
   {Spec-id=1}*:c=3
   {Spec-id=1}*:c=4
   {Spec-id=1}*:c=5
   ```
   ===================
   The partition shows up in the iceberg.partitions table only after new rows are inserted to the table after the spec has changed.
   
   Updated the PR to the following as discussed:
   ```
   spec-id=0/d_part=10/e_part=5
   spec-id=0/d_part=2/e_part=5
   spec-id=0/d_part=3/e_part=4
   current-spec-id=1/c=2
   current-spec-id=1/c=3
   current-spec-id=1/c=4
   current-spec-id=1/c=5  
   
   ```



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