[ 
https://issues.apache.org/jira/browse/HIVE-25154?focusedWorklogId=604938&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-604938
 ]

ASF GitHub Bot logged work on HIVE-25154:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 02/Jun/21 04:20
            Start Date: 02/Jun/21 04:20
    Worklog Time Spent: 10m 
      Work Description: pkumarsinha commented on a change in pull request #2311:
URL: https://github.com/apache/hive/pull/2311#discussion_r643399198



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java
##########
@@ -625,6 +639,16 @@ public boolean runOneWorkerIteration(
     }
     String cmd = null;
     try {
+      TableName tb = req.tableName;
+      String dbName = 
MetaStoreUtils.prependCatalogToDbName(tb.getCat(),tb.getDb(), conf);
+      if (dbsBeingFailedOver.contains(dbName)
+              || 
MetaStoreUtils.isDbBeingFailedOver(rs.getDatabase(tb.getCat(), tb.getDb()))) {
+        if (!dbsBeingFailedOver.contains(dbName)) {

Review comment:
       How will this condition be true ?

##########
File path: 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PartitionManagementTask.java
##########
@@ -222,17 +237,31 @@ private void setupMsckPathInvalidation() {
     private Configuration conf;
     private String qualifiedTableName;
     private CountDownLatch countDownLatch;
+    private Set<String> dbsBeingFailedOver;
+    private IMetaStoreClient msc;
 
-    MsckThread(MsckInfo msckInfo, Configuration conf, String 
qualifiedTableName, CountDownLatch countDownLatch) {
+    MsckThread(MsckInfo msckInfo, Configuration conf, String 
qualifiedTableName,
+               CountDownLatch countDownLatch, Set<String> dbsBeingFailedOver, 
IMetaStoreClient msc) {
       this.msckInfo = msckInfo;
       this.conf = conf;
       this.qualifiedTableName = qualifiedTableName;
       this.countDownLatch = countDownLatch;
+      this.dbsBeingFailedOver = dbsBeingFailedOver;
+      this.msc = msc;
     }
 
     @Override
     public void run() {
       try {
+        String dbName = 
MetaStoreUtils.prependCatalogToDbName(msckInfo.getCatalogName(), 
msckInfo.getDbName(), conf);
+        if (dbsBeingFailedOver.contains(dbName) ||
+                
MetaStoreUtils.isDbBeingFailedOver(msc.getDatabase(msckInfo.getCatalogName(), 
msckInfo.getDbName()))) {
+          if (!dbsBeingFailedOver.contains(dbName)) {

Review comment:
       This isn't thread-safe




-- 
This is an automated message from the 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:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 604938)
    Time Spent: 2h 10m  (was: 2h)

> Disable StatsUpdaterThread and PartitionManagementTask for db that is being 
> failoved over.
> ------------------------------------------------------------------------------------------
>
>                 Key: HIVE-25154
>                 URL: https://issues.apache.org/jira/browse/HIVE-25154
>             Project: Hive
>          Issue Type: Improvement
>            Reporter: Haymant Mangla
>            Assignee: Haymant Mangla
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 2h 10m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to