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

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

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaseLeaderElection.java:
##########
@@ -0,0 +1,473 @@
+/*
+ * 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.leader;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.api.CheckLockRequest;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.HeartbeatRequest;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.SecurityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Hive Lock based leader election.
+ * If wins, the current instance becomes the leader,
+ * and a heartbeat daemon will be started to renew the lock before timeout.
+ * If loses, a non-leader watcher will also be started to check the
+ * lock periodically to see if he can grab the lock in order to be the leader.
+ * The change of Leadership can be received by registering the
+ * listeners through {@link LeaderElection#addStateListener}.
+ */
+public class LeaseLeaderElection implements LeaderElection<TableName> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(LeaseLeaderElection.class);
+
+  private static final AtomicLong ID = new AtomicLong();
+
+  // Result of election
+  private volatile boolean isLeader;
+
+  private TxnStore store;
+
+  // Initial sleep time for locking the table at retrying.
+  private long nextSleep = 50;
+
+  // A daemon used for renewing the lock before timeout,
+  // this happens when the current instance wins the election.
+  private LeaseWatcher heartbeater;
+
+  // For non-leader instances to check the lock periodically to
+  // see if there is a chance to take over the leadership.
+  // At any time, only one of heartbeater and nonLeaderWatcher is alive.
+  private LeaseWatcher nonLeaderWatcher;
+
+  // Current lock id
+  private volatile long lockId = -1;
+
+  // Leadership change listeners
+  private List<LeadershipStateListener> listeners = new ArrayList<>();
+
+  // Property for testing only
+  public static final String METASTORE_RENEW_LEASE = 
"metastore.renew.leader.lease";
+
+  private String name;
+
+  private void doWork(LockResponse resp, Configuration conf,
+      TableName tableName) throws LeaderException {
+    lockId = resp.getLockid();
+    assert resp.getState() == LockState.ACQUIRED || resp.getState() == 
LockState.WAITING;
+    shutdownWatcher();
+
+    switch (resp.getState()) {
+    case ACQUIRED:
+      boolean renewLease = conf.getBoolean(METASTORE_RENEW_LEASE, true);
+      heartbeater = renewLease ?
+          new Heartbeater(conf, tableName) : new 
ReleaseAndRequireWatcher(conf, tableName);
+      heartbeater.perform();
+      if (!isLeader) {
+        isLeader = true;
+        notifyListener();
+      }
+      break;
+    case WAITING:
+      nonLeaderWatcher = new NonLeaderWatcher(conf, tableName);
+      nonLeaderWatcher.perform();
+      if (isLeader) {
+        isLeader = false;
+        notifyListener();
+      }
+      break;
+    default:
+      throw new IllegalStateException("Unexpected lock state: " + 
resp.getState());
+    }
+  }
+
+  private void notifyListener() {
+    listeners.forEach(listener -> {
+      try {
+        if (isLeader) {
+          listener.takeLeadership(this);
+        } else {
+          listener.lossLeadership(this);

Review Comment:
   checked in: 
   
https://github.com/apache/hive/blob/bcbdd0c77f73ce7f46342cc30d6c565027fac311/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaseLeaderElection.java#L114-L120





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

    Worklog Id:     (was: 812336)
    Time Spent: 3h 50m  (was: 3h 40m)

> 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: 3h 50m
>  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