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

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

                Author: ASF GitHub Bot
            Created on: 06/Jul/21 00:41
            Start Date: 06/Jul/21 00:41
    Worklog Time Spent: 10m 
      Work Description: hmangla98 commented on a change in pull request #2121:
URL: https://github.com/apache/hive/pull/2121#discussion_r664168191



##########
File path: 
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
##########
@@ -138,6 +141,226 @@ public void tearDown() throws Throwable {
     primary.run("drop database if exists " + primaryDbName + "_extra cascade");
   }
 
+  @Test
+  public void testFailoverDuringDump() throws Throwable {
+    HiveConf primaryConf = primary.getConf();
+    TxnStore txnHandler = TxnUtils.getTxnStore(primary.getConf());
+    WarehouseInstance.Tuple dumpData = null;
+    List<String> failoverConfigs = Arrays.asList("'" + 
HiveConf.ConfVars.HIVE_REPL_FAILOVER_START + "'='true'");
+    dumpData = primary.run("use " + primaryDbName)
+            .run("create table t1 (id int) clustered by(id) into 3 buckets 
stored as orc " +
+                    "tblproperties (\"transactional\"=\"true\")")
+            .run("create table t2 (rank int) partitioned by (name string) 
tblproperties(\"transactional\"=\"true\", " +
+                    "\"transactional_properties\"=\"insert_only\")")
+            .dump(primaryDbName, failoverConfigs);
+
+    //This dump is not failover ready as target db can be used for replication 
only after first incremental load.
+    FileSystem fs = new Path(dumpData.dumpLocation).getFileSystem(conf);
+    Path dumpPath = new Path(dumpData.dumpLocation, 
ReplUtils.REPL_HIVE_BASE_DIR);
+    assertFalse(fs.exists(new Path(dumpPath, 
ReplAck.FAILOVER_READY_MARKER.toString())));
+    
assertFalse(MetaStoreUtils.isDbBeingFailedOver(primary.getDatabase(primaryDbName)));
+
+    replica.load(replicatedDbName, primaryDbName)
+            .run("use " + replicatedDbName)
+            .run("show tables")
+            .verifyResults(new String[]{"t1", "t2"})
+            .run("repl status " + replicatedDbName)
+            .verifyResult(dumpData.lastReplicationId);
+
+    primary.run("use " + primaryDbName)
+            .run("insert into t1 values(1)")
+            .run("insert into t2 partition(name='Bob') values(11)")
+            .run("insert into t2 partition(name='Carl') values(10)");
+
+    /**Open transactions can be of two types:
+     Case 1) Txns that belong to different db or have not acquired HIVE LOCKS: 
These txns would be caught in
+     _failovermetadata file.
+     Case 2) Txns that belong to db under replication: These txns would be 
aborted as part of dump operation.
+     */
+    // Open 3 txns for Database which is not under replication
+    int numTxnsForSecDb = 3;
+    List<Long> txnsForSecDb = openTxns(numTxnsForSecDb, txnHandler, 
primaryConf);
+
+    // Allocate write ids for both tables of secondary db for 3 txns
+    // t1=5 and t2=5
+    Map<String, Long> tablesInSecDb = new HashMap<>();
+    tablesInSecDb.put("t1", (long) numTxnsForSecDb);
+    tablesInSecDb.put("t2", (long) numTxnsForSecDb);
+    List<Long> lockIdsForSecDb = 
allocateWriteIdsForTablesAndAquireLocks(primaryDbName + "_extra",
+            tablesInSecDb, txnHandler, txnsForSecDb, primaryConf);
+
+    //Open 2 txns for Primary Db
+    int numTxnsForPrimaryDb = 2;
+    List<Long> txnsForPrimaryDb = openTxns(numTxnsForPrimaryDb, txnHandler, 
primaryConf);
+
+    // Allocate write ids for both tables of primary db for 2 txns
+    // t1=5 and t2=5
+    Map<String, Long> tablesInPrimaryDb = new HashMap<>();
+    tablesInPrimaryDb.put("t1", (long) numTxnsForPrimaryDb + 1);
+    tablesInPrimaryDb.put("t2", (long) numTxnsForPrimaryDb + 2);
+    List<Long> lockIdsForPrimaryDb = 
allocateWriteIdsForTablesAndAquireLocks(primaryDbName,
+            tablesInPrimaryDb, txnHandler, txnsForPrimaryDb, primaryConf);
+
+    //Open 1 txn with no hive locks acquired
+    List<Long> txnsWithNoLocks = openTxns(1, txnHandler, primaryConf);
+
+    dumpData = primary.dump(primaryDbName, failoverConfigs);
+
+    fs = new Path(dumpData.dumpLocation).getFileSystem(conf);
+    dumpPath = new Path(dumpData.dumpLocation, ReplUtils.REPL_HIVE_BASE_DIR);
+    assertTrue(fs.exists(new Path(dumpPath, DUMP_ACKNOWLEDGEMENT.toString())));
+    assertTrue(fs.exists(new Path(dumpPath, 
FailoverMetaData.FAILOVER_METADATA)));
+    assertTrue(fs.exists(new Path(dumpPath, 
ReplAck.FAILOVER_READY_MARKER.toString())));
+    
assertTrue(MetaStoreUtils.isDbBeingFailedOver(primary.getDatabase(primaryDbName)));
+    FailoverMetaData failoverMD = new FailoverMetaData(dumpPath, conf);
+
+    List<Long> openTxns = failoverMD.getOpenTxns();
+    List<Long> txnsAborted = failoverMD.getAbortedTxns();
+    assertTrue(txnsAborted.size() == 2);
+    assertTrue(txnsAborted.containsAll(txnsForPrimaryDb));
+    assertTrue(openTxns.size() == 4);
+    assertTrue(openTxns.containsAll(txnsForSecDb));
+    assertTrue(openTxns.containsAll(txnsWithNoLocks));
+    assertTrue(failoverMD.getTxnsWithoutLock().equals(txnsWithNoLocks));
+
+
+    //TxnsForPrimaryDb and txnsWithNoLocks would have been aborted by dump 
operation.
+    verifyAllOpenTxnsAborted(txnsForPrimaryDb, primaryConf);
+    verifyAllOpenTxnsNotAborted(txnsForSecDb, primaryConf);
+    verifyAllOpenTxnsNotAborted(txnsWithNoLocks, primaryConf);
+    //Abort the txns
+    txnHandler.abortTxns(new AbortTxnsRequest(txnsForSecDb));
+    txnHandler.abortTxns(new AbortTxnsRequest(txnsWithNoLocks));
+    verifyAllOpenTxnsAborted(txnsForSecDb, primaryConf);
+    verifyAllOpenTxnsAborted(txnsWithNoLocks, primaryConf);
+    releaseLocks(txnHandler, lockIdsForSecDb);
+
+    replica.load(replicatedDbName, primaryDbName)

Review comment:
       This patch is for dump operation only. (As of now) Load will perform the 
same way no matter what the value of this conf is. 




-- 
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-unsubscr...@hive.apache.org

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


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

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

> Handle failover case during Repl Dump
> -------------------------------------
>
>                 Key: HIVE-24918
>                 URL: https://issues.apache.org/jira/browse/HIVE-24918
>             Project: Hive
>          Issue Type: New Feature
>            Reporter: Haymant Mangla
>            Assignee: Haymant Mangla
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 2h 10m
>  Remaining Estimate: 0h
>
> To handle:
>  a) Whenever user wants to go ahead with failover, during the next or 
> subsequent repl dump operation upon confirming that there are no pending open 
> transaction events, It should create a _failover_ready marker file in the 
> dump dir. This marker file would contain scheduled query name
> that has generated this dump.
> b) Skip next repl dump instances once we have the marker file placed.



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

Reply via email to