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

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

                Author: ASF GitHub Bot
            Created on: 27/Sep/22 02:58
            Start Date: 27/Sep/22 02:58
    Worklog Time Spent: 10m 
      Work Description: dengzhhu653 commented on code in PR #3567:
URL: https://github.com/apache/hive/pull/3567#discussion_r980684231


##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreLeaseNonLeader.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.leader.LeaderElection;
+import org.apache.hadoop.hive.metastore.leader.LeaderElectionContext;
+import org.apache.hadoop.hive.metastore.leader.LeaseLeaderElection;
+import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertTrue;
+
+public class TestMetastoreLeaseNonLeader {
+
+  LeaderElection election;
+
+  TestMetastoreHousekeepingLeader hms;
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = MetastoreConf.newMetastoreConf();
+    TestTxnDbUtil.setConfValues(conf);
+    TestTxnDbUtil.prepDb(conf);
+    election = new LeaseLeaderElection();
+    MetastoreConf.setVar(conf, 
MetastoreConf.ConfVars.METASTORE_HOUSEKEEPING_LEADER_ELECTION, "lock");
+    TableName tableName = (TableName) 
LeaderElectionContext.getLeaderMutex(conf,
+        LeaderElectionContext.TTYPE.HOUSEKEEPING, null);
+    election.tryBeLeader(conf, tableName);
+    assertTrue("The elector should hold the lease now", election.isLeader());
+    // start the non-leader hms now
+    hms = new TestMetastoreHousekeepingLeader();
+    MetastoreConf.setTimeVar(hms.conf, 
MetastoreConf.ConfVars.LOCK_SLEEP_BETWEEN_RETRIES, 1, TimeUnit.SECONDS);
+    hms.conf.setBoolean(LeaderElectionContext.LEADER_IN_TEST, true);
+    hms.internalSetup("", false);

Review Comment:
   this is verified in:
   
https://github.com/apache/hive/blob/bcbdd0c77f73ce7f46342cc30d6c565027fac311/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreLeaseNonLeader.java#L61-L66





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

    Worklog Id:     (was: 812329)
    Time Spent: 3.5h  (was: 3h 20m)

> Introduce dynamic leader election in HMS
> ----------------------------------------
>
>                 Key: HIVE-26509
>                 URL: https://issues.apache.org/jira/browse/HIVE-26509
>             Project: Hive
>          Issue Type: New Feature
>          Components: Standalone Metastore
>            Reporter: Zhihua Deng
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 3.5h
>  Remaining Estimate: 0h
>
> From HIVE-21841 we have a leader HMS selected by configuring 
> metastore.housekeeping.leader.hostname on startup. This approach saves us 
> from running duplicated HMS's housekeeping tasks cluster-wide. 
> In this jira, we introduce another dynamic leader election: adopt hive lock 
> to implement the leader election. Once a HMS owns the lock, then it becomes 
> the leader, carries out the housekeeping tasks, and sends heartbeats to renew 
> the lock before timeout. If the leader fails to reclaim the lock, then stops 
> the already started tasks if it has, the electing event is audited. We can 
> achieve a more dynamic leader when the original goes down or in the public 
> cloud without well configured property, and reduce the leader’s burdens by 
> running these tasks among different leaders.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to