You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2021/02/08 08:39:18 UTC

[GitHub] [hive] kishendas opened a new pull request #1956: Send tableId in get_partitions_by_names HMS API

kishendas opened a new pull request #1956:
URL: https://github.com/apache/hive/pull/1956


   ### What changes were proposed in this pull request?
   send tableId to get_partitions_by_names HMS API
   
   ### Why are the changes needed?
   tableId and validWriteIdList are compared in the HMS cache in order to decide whether request should be served from the cache or database. 
   
   ### Does this PR introduce _any_ user-facing change?
   no
   
   ### How was this patch tested?
   unit tests
   


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

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



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


[GitHub] [hive] kishendas commented on a change in pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
kishendas commented on a change in pull request #1956:
URL: https://github.com/apache/hive/pull/1956#discussion_r577841886



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -3195,17 +3196,24 @@ public void alterPartitions(String dbName, String tableName,
   public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(String dbName, String tableName,
       List<String> partitionNames) throws HiveException {
     try {
-      return getMSC().getPartitionsByNames(dbName, tableName, partitionNames);
+      GetPartitionsByNamesRequest req = new GetPartitionsByNamesRequest();
+      req.setDb_name(dbName);
+      req.setTbl_name(tableName);
+      req.setNames(partitionNames);
+      return getPartitionsByNames(req, null);
     } catch (Exception e) {
       LOG.error("Failed getPartitionsByNames", e);
       throw new HiveException(e);
     }
   }
 
-    public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(GetPartitionsByNamesRequest req)
+    public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(GetPartitionsByNamesRequest req,
+      Table table)
         throws HiveException {
     try {
-      Table table = getTable(req.getDb_name(), req.getTbl_name());
+      if( table == null ) {
+        table = getTable(req.getDb_name(), req.getTbl_name());

Review comment:
       TableId is still required for the remote cache to decide whether it can serve the data from cache or has to refresh it from 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.

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



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


[GitHub] [hive] yongzhi commented on a change in pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
yongzhi commented on a change in pull request #1956:
URL: https://github.com/apache/hive/pull/1956#discussion_r574002879



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
##########
@@ -2223,6 +2224,29 @@ public PartitionsResponse getPartitionsRequest(PartitionsRequest req)
     return deepCopyPartitions(FilterUtils.filterPartitionsIfEnabled(isClientFilterEnabled, filterHook, parts));
   }
 
+  @Override
+  public GetPartitionsByNamesResult getPartitionsByNames(GetPartitionsByNamesRequest req)
+          throws NoSuchObjectException, MetaException, TException {
+    checkDbAndTableFilters(req.getCatName(), req.getDb_name(), req.getTbl_name());
+    req.setDb_name(prependCatalogToDbName(req.getCatName(), req.getDb_name(), conf));
+    if (req.getValidWriteIdList() == null) {
+      req.setValidWriteIdList(getValidWriteIdList(prependCatalogToDbName(req.getCatName(), req.getDb_name(),
+              conf), req.getTbl_name()));
+    }
+    if (req.getId() <= 0) {
+      req.setId(getTable(prependCatalogToDbName(req.getCatName(), req.getDb_name(), conf), req.getTbl_name()).getId());

Review comment:
       getTable put here will add one more calls to HMS if cache is not used. So it should be at cachedHMS client.




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

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



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


[GitHub] [hive] kishendas commented on a change in pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
kishendas commented on a change in pull request #1956:
URL: https://github.com/apache/hive/pull/1956#discussion_r579461372



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
##########
@@ -1497,8 +1497,8 @@ GetPartitionsPsWithAuthResponse listPartitionsWithAuthInfoRequest(GetPartitionsP
    * @throws MetaException error accessing the RDBMS.
    * @throws TException thrift transport error
    */
-  List<Partition> getPartitionsByNames(String db_name, String tbl_name,
-      List<String> part_names) throws NoSuchObjectException, MetaException, TException;
+  List<Partition> getPartitionsByNames(String db_name, String tbl_name, List<String> part_names,

Review comment:
       I am not changing HMS APIs 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.

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



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


[GitHub] [hive] yongzhi commented on a change in pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
yongzhi commented on a change in pull request #1956:
URL: https://github.com/apache/hive/pull/1956#discussion_r577839999



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -3195,17 +3196,24 @@ public void alterPartitions(String dbName, String tableName,
   public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(String dbName, String tableName,
       List<String> partitionNames) throws HiveException {
     try {
-      return getMSC().getPartitionsByNames(dbName, tableName, partitionNames);
+      GetPartitionsByNamesRequest req = new GetPartitionsByNamesRequest();
+      req.setDb_name(dbName);
+      req.setTbl_name(tableName);
+      req.setNames(partitionNames);
+      return getPartitionsByNames(req, null);
     } catch (Exception e) {
       LOG.error("Failed getPartitionsByNames", e);
       throw new HiveException(e);
     }
   }
 
-    public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(GetPartitionsByNamesRequest req)
+    public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(GetPartitionsByNamesRequest req,
+      Table table)
         throws HiveException {
     try {
-      Table table = getTable(req.getDb_name(), req.getTbl_name());
+      if( table == null ) {
+        table = getTable(req.getDb_name(), req.getTbl_name());

Review comment:
       As if you do not use cached client, you need not table id, it will be different for the client direct use HMS apis (no need to call getTable). 




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

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



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


[GitHub] [hive] kishendas commented on a change in pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
kishendas commented on a change in pull request #1956:
URL: https://github.com/apache/hive/pull/1956#discussion_r573970499



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
##########
@@ -2223,6 +2224,29 @@ public PartitionsResponse getPartitionsRequest(PartitionsRequest req)
     return deepCopyPartitions(FilterUtils.filterPartitionsIfEnabled(isClientFilterEnabled, filterHook, parts));
   }
 
+  @Override
+  public GetPartitionsByNamesResult getPartitionsByNames(GetPartitionsByNamesRequest req)
+          throws NoSuchObjectException, MetaException, TException {
+    checkDbAndTableFilters(req.getCatName(), req.getDb_name(), req.getTbl_name());
+    req.setDb_name(prependCatalogToDbName(req.getCatName(), req.getDb_name(), conf));
+    if (req.getValidWriteIdList() == null) {
+      req.setValidWriteIdList(getValidWriteIdList(prependCatalogToDbName(req.getCatName(), req.getDb_name(),
+              conf), req.getTbl_name()));
+    }
+    if (req.getId() <= 0) {
+      req.setId(getTable(prependCatalogToDbName(req.getCatName(), req.getDb_name(), conf), req.getTbl_name()).getId());

Review comment:
       That's right. 




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

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



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


[GitHub] [hive] kishendas commented on a change in pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
kishendas commented on a change in pull request #1956:
URL: https://github.com/apache/hive/pull/1956#discussion_r577206307



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
##########
@@ -2223,6 +2224,29 @@ public PartitionsResponse getPartitionsRequest(PartitionsRequest req)
     return deepCopyPartitions(FilterUtils.filterPartitionsIfEnabled(isClientFilterEnabled, filterHook, parts));
   }
 
+  @Override
+  public GetPartitionsByNamesResult getPartitionsByNames(GetPartitionsByNamesRequest req)
+          throws NoSuchObjectException, MetaException, TException {
+    checkDbAndTableFilters(req.getCatName(), req.getDb_name(), req.getTbl_name());
+    req.setDb_name(prependCatalogToDbName(req.getCatName(), req.getDb_name(), conf));
+    if (req.getValidWriteIdList() == null) {
+      req.setValidWriteIdList(getValidWriteIdList(prependCatalogToDbName(req.getCatName(), req.getDb_name(),
+              conf), req.getTbl_name()));
+    }
+    if (req.getId() <= 0) {
+      req.setId(getTable(prependCatalogToDbName(req.getCatName(), req.getDb_name(), conf), req.getTbl_name()).getId());

Review comment:
       Please check now. I have removed this logic from HiveMetaStoreClient.java . 




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

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



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


[GitHub] [hive] yongzhi commented on a change in pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
yongzhi commented on a change in pull request #1956:
URL: https://github.com/apache/hive/pull/1956#discussion_r577305917



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -3195,17 +3196,24 @@ public void alterPartitions(String dbName, String tableName,
   public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(String dbName, String tableName,
       List<String> partitionNames) throws HiveException {
     try {
-      return getMSC().getPartitionsByNames(dbName, tableName, partitionNames);
+      GetPartitionsByNamesRequest req = new GetPartitionsByNamesRequest();
+      req.setDb_name(dbName);
+      req.setTbl_name(tableName);
+      req.setNames(partitionNames);
+      return getPartitionsByNames(req, null);
     } catch (Exception e) {
       LOG.error("Failed getPartitionsByNames", e);
       throw new HiveException(e);
     }
   }
 
-    public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(GetPartitionsByNamesRequest req)
+    public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(GetPartitionsByNamesRequest req,
+      Table table)
         throws HiveException {
     try {
-      Table table = getTable(req.getDb_name(), req.getTbl_name());
+      if( table == null ) {
+        table = getTable(req.getDb_name(), req.getTbl_name());

Review comment:
       This can cause a round trip to HMS to get the table too for none cached client. I think you can add the logic here: https://github.com/apache/hive/blob/master/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientWithLocalCache.java#L460




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

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



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


[GitHub] [hive] yongzhi commented on a change in pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
yongzhi commented on a change in pull request #1956:
URL: https://github.com/apache/hive/pull/1956#discussion_r579460370



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
##########
@@ -1497,8 +1497,8 @@ GetPartitionsPsWithAuthResponse listPartitionsWithAuthInfoRequest(GetPartitionsP
    * @throws MetaException error accessing the RDBMS.
    * @throws TException thrift transport error
    */
-  List<Partition> getPartitionsByNames(String db_name, String tbl_name,
-      List<String> part_names) throws NoSuchObjectException, MetaException, TException;
+  List<Partition> getPartitionsByNames(String db_name, String tbl_name, List<String> part_names,

Review comment:
       How about old version clients?




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

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



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


[GitHub] [hive] yongzhi commented on a change in pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
yongzhi commented on a change in pull request #1956:
URL: https://github.com/apache/hive/pull/1956#discussion_r573198158



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
##########
@@ -2223,6 +2224,29 @@ public PartitionsResponse getPartitionsRequest(PartitionsRequest req)
     return deepCopyPartitions(FilterUtils.filterPartitionsIfEnabled(isClientFilterEnabled, filterHook, parts));
   }
 
+  @Override
+  public GetPartitionsByNamesResult getPartitionsByNames(GetPartitionsByNamesRequest req)
+          throws NoSuchObjectException, MetaException, TException {
+    checkDbAndTableFilters(req.getCatName(), req.getDb_name(), req.getTbl_name());
+    req.setDb_name(prependCatalogToDbName(req.getCatName(), req.getDb_name(), conf));
+    if (req.getValidWriteIdList() == null) {
+      req.setValidWriteIdList(getValidWriteIdList(prependCatalogToDbName(req.getCatName(), req.getDb_name(),
+              conf), req.getTbl_name()));
+    }
+    if (req.getId() <= 0) {
+      req.setId(getTable(prependCatalogToDbName(req.getCatName(), req.getDb_name(), conf), req.getTbl_name()).getId());

Review comment:
       The table ID here will be only used by the cached HMS client, right? If it is used by HiveMetastoreClient, will we throw an error if the ID from getTable different from the one is passed-in?




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

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



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


[GitHub] [hive] kishendas commented on a change in pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
kishendas commented on a change in pull request #1956:
URL: https://github.com/apache/hive/pull/1956#discussion_r577420383



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -3195,17 +3196,24 @@ public void alterPartitions(String dbName, String tableName,
   public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(String dbName, String tableName,
       List<String> partitionNames) throws HiveException {
     try {
-      return getMSC().getPartitionsByNames(dbName, tableName, partitionNames);
+      GetPartitionsByNamesRequest req = new GetPartitionsByNamesRequest();
+      req.setDb_name(dbName);
+      req.setTbl_name(tableName);
+      req.setNames(partitionNames);
+      return getPartitionsByNames(req, null);
     } catch (Exception e) {
       LOG.error("Failed getPartitionsByNames", e);
       throw new HiveException(e);
     }
   }
 
-    public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(GetPartitionsByNamesRequest req)
+    public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(GetPartitionsByNamesRequest req,
+      Table table)
         throws HiveException {
     try {
-      Table table = getTable(req.getDb_name(), req.getTbl_name());
+      if( table == null ) {
+        table = getTable(req.getDb_name(), req.getTbl_name());

Review comment:
       Table Id is not cached right now in the local cache, so it would be the same. 




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

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



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


[GitHub] [hive] yongzhi merged pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
yongzhi merged pull request #1956:
URL: https://github.com/apache/hive/pull/1956


   


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

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



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


[GitHub] [hive] kishendas commented on a change in pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
kishendas commented on a change in pull request #1956:
URL: https://github.com/apache/hive/pull/1956#discussion_r579713503



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
##########
@@ -1497,8 +1497,8 @@ GetPartitionsPsWithAuthResponse listPartitionsWithAuthInfoRequest(GetPartitionsP
    * @throws MetaException error accessing the RDBMS.
    * @throws TException thrift transport error
    */
-  List<Partition> getPartitionsByNames(String db_name, String tbl_name,
-      List<String> part_names) throws NoSuchObjectException, MetaException, TException;
+  List<Partition> getPartitionsByNames(String db_name, String tbl_name, List<String> part_names,

Review comment:
       @yongzhi Please check the latest. I have retained all the existing IMetaStoreClient APIs and introduced new ones. Also, got a green build. 




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

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



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


[GitHub] [hive] kishendas commented on a change in pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
kishendas commented on a change in pull request #1956:
URL: https://github.com/apache/hive/pull/1956#discussion_r577886973



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -3195,17 +3196,24 @@ public void alterPartitions(String dbName, String tableName,
   public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(String dbName, String tableName,
       List<String> partitionNames) throws HiveException {
     try {
-      return getMSC().getPartitionsByNames(dbName, tableName, partitionNames);
+      GetPartitionsByNamesRequest req = new GetPartitionsByNamesRequest();
+      req.setDb_name(dbName);
+      req.setTbl_name(tableName);
+      req.setNames(partitionNames);
+      return getPartitionsByNames(req, null);
     } catch (Exception e) {
       LOG.error("Failed getPartitionsByNames", e);
       throw new HiveException(e);
     }
   }
 
-    public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(GetPartitionsByNamesRequest req)
+    public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(GetPartitionsByNamesRequest req,
+      Table table)
         throws HiveException {
     try {
-      Table table = getTable(req.getDb_name(), req.getTbl_name());
+      if( table == null ) {
+        table = getTable(req.getDb_name(), req.getTbl_name());

Review comment:
       Ok, removed the extra getTable call now. 




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

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



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


[GitHub] [hive] kishendas commented on a change in pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
kishendas commented on a change in pull request #1956:
URL: https://github.com/apache/hive/pull/1956#discussion_r579428630



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
##########
@@ -1497,8 +1497,8 @@ GetPartitionsPsWithAuthResponse listPartitionsWithAuthInfoRequest(GetPartitionsP
    * @throws MetaException error accessing the RDBMS.
    * @throws TException thrift transport error
    */
-  List<Partition> getPartitionsByNames(String db_name, String tbl_name,
-      List<String> part_names) throws NoSuchObjectException, MetaException, TException;
+  List<Partition> getPartitionsByNames(String db_name, String tbl_name, List<String> part_names,

Review comment:
       No, we want everyone to send validWriteIdList and tableId going forward. So, creating new interfaces doesn't work. 




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

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



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


[GitHub] [hive] yongzhi commented on a change in pull request #1956: HIVE-24743: Send tableId in get_partitions_by_names HMS API

Posted by GitBox <gi...@apache.org>.
yongzhi commented on a change in pull request #1956:
URL: https://github.com/apache/hive/pull/1956#discussion_r579426352



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
##########
@@ -1497,8 +1497,8 @@ GetPartitionsPsWithAuthResponse listPartitionsWithAuthInfoRequest(GetPartitionsP
    * @throws MetaException error accessing the RDBMS.
    * @throws TException thrift transport error
    */
-  List<Partition> getPartitionsByNames(String db_name, String tbl_name,
-      List<String> part_names) throws NoSuchObjectException, MetaException, TException;
+  List<Partition> getPartitionsByNames(String db_name, String tbl_name, List<String> part_names,

Review comment:
       Just change an interface may cause a backward compatible issue, maybe just add a new one and keeps old ones?




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

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



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