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

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

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/AuditLeaderListener.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.ColumnType;
+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.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.charset.StandardCharsets;
+
+import static java.util.Objects.requireNonNull;
+
+public class AuditLeaderListener implements 
LeaderElection.LeadershipStateListener {
+  private final Configuration configuration;
+
+  private final Path tableLocation;
+
+  private final static String SERDE = 
"org.apache.hadoop.hive.serde2.JsonSerDe";
+  private final static String INPUTFORMAT = 
"org.apache.hadoop.mapred.TextInputFormat";
+  private final static String OUTPUTFORMAT = 
"org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat";
+
+  public AuditLeaderListener(TableName tableName, IHMSHandler handler) throws 
Exception {
+    requireNonNull(tableName, "tableName is null");
+    requireNonNull(handler, "handler is null");
+    this.configuration = handler.getConf();
+    try {
+      // store the leader info as json + text for human-readable
+      Table table = new TableBuilder()
+          .setCatName(tableName.getCat())
+          .setDbName(tableName.getDb())
+          .setTableName(tableName.getTable())
+          .addCol("leader", ColumnType.STRING_TYPE_NAME)
+          .addCol("name", ColumnType.STRING_TYPE_NAME)
+          .addCol("elected_time", ColumnType.BIGINT_TYPE_NAME)
+          .setSerdeLib(SERDE)
+          .setInputFormat(INPUTFORMAT)
+          .setOutputFormat(OUTPUTFORMAT)
+          .build(handler.getConf());
+      handler.create_table(table);
+    } catch (AlreadyExistsException e) {
+      // ignore
+    }
+
+    Table table = handler.getMS().getTable(tableName.getCat(),
+        tableName.getDb(), tableName.getTable());
+    this.tableLocation = new Path(table.getSd().getLocation());
+    String serde = table.getSd().getSerdeInfo().getSerializationLib();
+    String input = table.getSd().getInputFormat();
+    String output = table.getSd().getOutputFormat();
+    if (!SERDE.equals(serde) || !INPUTFORMAT.equals(input)
+        || !OUTPUTFORMAT.equals(output)) {
+      throw new RuntimeException(tableName + " should be in json + text 
format");
+    }
+
+  }
+
+  @Override
+  public void takeLeadership(LeaderElection election) throws Exception {
+    String hostName = getHostname();
+    String message = "{\"leader\": \"" + hostName + "\", \"name\": \""
+        + election.getName() + "\", \"elected_time\": " + 
System.currentTimeMillis() + "} \n";
+    Path path = new Path(tableLocation, "leader.json");
+    try {
+      FileSystem fs = Warehouse.getFs(path, configuration);
+      try (OutputStream outputStream = fs.exists(path) ?
+          fs.append(path) :
+          fs.create(path, false)) {
+        outputStream.write(message.getBytes(StandardCharsets.UTF_8));

Review Comment:
   this acts the same as putting the entry into the table,  this saves us from 
connecting HS2 and issuing an insert at HMS side.





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

    Worklog Id:     (was: 812324)
    Time Spent: 3h  (was: 2h 50m)

> 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
>  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