[
https://issues.apache.org/jira/browse/HIVE-26509?focusedWorklogId=812338&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-812338
]
ASF GitHub Bot logged work on HIVE-26509:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 27/Sep/22 03:13
Start Date: 27/Sep/22 03:13
Worklog Time Spent: 10m
Work Description: dengzhhu653 commented on code in PR #3567:
URL: https://github.com/apache/hive/pull/3567#discussion_r980689980
##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaderElectionContext.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import
org.apache.hadoop.hive.metastore.leader.LeaderElection.LeadershipStateListener;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.util.Objects.requireNonNull;
+
+public class LeaderElectionContext {
+
+ /**
+ * Types of tasks that exactly have one instance in a given warehouse.
+ * For those tasks which belong to the same type, they will be running in
the same leader.
+ */
+ public enum TTYPE {
+ HOUSEKEEPING(new TableName(Warehouse.DEFAULT_CATALOG_NAME, "sys",
+ "metastore_housekeeping_leader"), "housekeeping"),
+ WORKER(new TableName(Warehouse.DEFAULT_CATALOG_NAME, "sys",
+ "metastore_worker_leader"), "compactor_worker"),
+ ALWAYS_TASKS(new TableName(Warehouse.DEFAULT_CATALOG_NAME, "sys",
+ "metastore_always_tasks_leader"), "always_tasks");
+ // Mutex of TTYPE, which can be a nonexistent table
+ private final TableName mutex;
+ // Name of TTYPE
+ private final String name;
+
+ TTYPE(TableName tableName, String name) {
+ this.mutex = tableName;
+ this.name = name;
+ }
+ public TableName getTableName() {
+ return mutex;
+ }
+ public String getName() {
+ return name;
+ }
+ }
+
+ private final Configuration conf;
+ private final String servHost;
+ // Whether the context should be started as a daemon
+ private final boolean startAsDaemon;
+ // Audit the event of election
+ private AuditLeaderListener auditLeaderListener;
+ // State change listeners group by type
+ private final Map<TTYPE, List<LeadershipStateListener>> listeners;
+ // Collection of leader candidates
+ private final List<LeaderElection> leaderElections = new ArrayList<>();
+ // Property for testing, a single leader will be created
+ public final static String LEADER_IN_TEST =
"metastore.leader.election.in.test";
+
+ private LeaderElectionContext(String servHost, Configuration conf,
+ Map<TTYPE, List<LeadershipStateListener>> listeners,
+ boolean startAsDaemon, IHMSHandler handler) throws Exception {
+ requireNonNull(conf, "conf is null");
+ requireNonNull(listeners, "listeners is null");
+ this.servHost = servHost;
+ this.conf = new Configuration(conf);
+ this.startAsDaemon = startAsDaemon;
+ String tableName = MetastoreConf.getVar(conf,
+ MetastoreConf.ConfVars.METASTORE_HOUSEKEEPING_LEADER_AUDITTABLE);
+ if (StringUtils.isNotEmpty(tableName)) {
+ TableName table = TableName.fromString(tableName,
MetaStoreUtils.getDefaultCatalog(conf),
+ Warehouse.DEFAULT_DATABASE_NAME);
Review Comment:
The `default` database takes effect when the configured `tableName` is not
specified the database
Issue Time Tracking
-------------------
Worklog Id: (was: 812338)
Time Spent: 4h 10m (was: 4h)
> 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: 4h 10m
> 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)