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

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

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



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java
##########
@@ -625,6 +633,11 @@ public boolean runOneWorkerIteration(
     }
     String cmd = null;
     try {
+      TableName tb = req.tableName;

Review comment:
       If the very first table belongs to DbBeingFailover, it will break the 
logic for "doWait"

##########
File path: 
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
##########
@@ -229,6 +231,15 @@ public static boolean isExternalTable(Table table) {
     return isExternal(params);
   }
 
+  public static boolean isDbBeingFailedOver(Database db) {
+    assert (db != null);
+    Map<String, String> dbParameters = db.getParameters();
+    if ((dbParameters != null) && 
(dbParameters.containsKey(ReplConst.REPL_FAILOVER_ENABLED))) {
+      return 
ReplConst.TRUE.equals(dbParameters.get(ReplConst.REPL_FAILOVER_ENABLED));
+    }
+    return false;

Review comment:
       Does this single line suffice?
   return dbParameters != null && 
ReplConst.TRUE.equals(dbParameters.get(ReplConst.REPL_FAILOVER_ENABLED));

##########
File path: 
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
##########
@@ -229,6 +231,15 @@ public static boolean isExternalTable(Table table) {
     return isExternal(params);
   }
 
+  public static boolean isDbBeingFailedOver(Database db) {
+    assert (db != null);
+    Map<String, String> dbParameters = db.getParameters();
+    if ((dbParameters != null) && 
(dbParameters.containsKey(ReplConst.REPL_FAILOVER_ENABLED))) {
+      return 
ReplConst.TRUE.equals(dbParameters.get(ReplConst.REPL_FAILOVER_ENABLED));
+    }
+    return false;

Review comment:
       also, ReplConst.TRUE.equals : do we need to handle case sensitiveness? 

##########
File path: 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PartitionManagementTask.java
##########
@@ -133,11 +135,21 @@ public void run() {
         LOG.info("Looking for tables using catalog: {} dbPattern: {} 
tablePattern: {} found: {}", catalogName,
           dbPattern, tablePattern, foundTableMetas.size());
 
+        Map<String, Boolean> databasesToSkip = new HashMap<>();
+
         for (TableMeta tableMeta : foundTableMetas) {
           try {
+            String dbName = 
MetaStoreUtils.prependCatalogToDbName(tableMeta.getCatName(), 
tableMeta.getDbName(), conf);
+            if (!databasesToSkip.containsKey(dbName)) {
+              Database db = msc.getDatabase(tableMeta.getCatName(), 
tableMeta.getDbName());
+              databasesToSkip.put(dbName, isTargetOfReplication(db) || 
MetaStoreUtils.isDbBeingFailedOver(db));
+            }
+            if (databasesToSkip.get(dbName)) {
+              LOG.info("Skipping table : {}", tableMeta.getTableName());

Review comment:
       use debug.

##########
File path: 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PartitionManagementTask.java
##########
@@ -133,11 +135,21 @@ public void run() {
         LOG.info("Looking for tables using catalog: {} dbPattern: {} 
tablePattern: {} found: {}", catalogName,
           dbPattern, tablePattern, foundTableMetas.size());
 
+        Map<String, Boolean> databasesToSkip = new HashMap<>();
+
         for (TableMeta tableMeta : foundTableMetas) {
           try {
+            String dbName = 
MetaStoreUtils.prependCatalogToDbName(tableMeta.getCatName(), 
tableMeta.getDbName(), conf);
+            if (!databasesToSkip.containsKey(dbName)) {
+              Database db = msc.getDatabase(tableMeta.getCatName(), 
tableMeta.getDbName());
+              databasesToSkip.put(dbName, isTargetOfReplication(db) || 
MetaStoreUtils.isDbBeingFailedOver(db));

Review comment:
       Add a INFO level log for DB that with why it is getting skipped...

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java
##########
@@ -625,6 +633,11 @@ public boolean runOneWorkerIteration(
     }
     String cmd = null;
     try {
+      TableName tb = req.tableName;

Review comment:
       add a test where db being failed over is picked up first and later the 
other db is picked up 

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java
##########
@@ -210,27 +213,34 @@ private void stopWorkers() {
     }
   }
 
-  private List<AnalyzeWork> processOneTable(TableName fullTableName)
+  private List<AnalyzeWork> processOneTable(TableName fullTableName, 
Map<String, Boolean> dbsToSkip)
       throws MetaException, NoSuchTxnException, NoSuchObjectException {
     if (isAnalyzeTableInProgress(fullTableName)) return null;
     String cat = fullTableName.getCat(), db = fullTableName.getDb(), tbl = 
fullTableName.getTable();
+    String dbName = MetaStoreUtils.prependCatalogToDbName(cat,db, conf);
+    if (!dbsToSkip.containsKey(dbName)) {
+      Database database = rs.getDatabase(cat, db);
+      boolean skipDb = false;
+      if (MetaStoreUtils.isDbBeingFailedOver(database)) {
+        skipDb = true;
+        LOG.info("Skipping all the tables which belong to database: {} as it 
is being failed over", db);
+      } else if (ReplUtils.isTargetOfReplication(database)) {

Review comment:
       There is lot of code duplication between this and PartitionManagement. 
Can we make not achieve by having a single copy?
   Also,  why do we have two methods for isTargetOfReplication(), can we have 
just one?




-- 
This is an automated message from the 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:
us...@infra.apache.org


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

    Worklog Id:     (was: 606478)
    Time Spent: 4h 10m  (was: 4h)

> 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: 4h 10m
>  Remaining Estimate: 0h
>




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

Reply via email to