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

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

                Author: ASF GitHub Bot
            Created on: 25/Jan/22 10:47
            Start Date: 25/Jan/22 10:47
    Worklog Time Spent: 10m 
      Work Description: pvary commented on a change in pull request #2967:
URL: https://github.com/apache/hive/pull/2967#discussion_r791585817



##########
File path: 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandlerContext.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+
+/**
+ * When one hms client connects in, we create a handler context for it.
+ * We store session information here.
+ */
+public final class HMSHandlerContext {
+
+  private static final ThreadLocal<HMSHandlerContext> context = new 
ThreadLocal<>();
+
+  private static final AtomicInteger nextSerialNum = new AtomicInteger();
+
+  private RawStore rawStore;
+
+  private TxnStore txnStore;
+
+  // Thread local HMSHandler used during shutdown to notify meta listeners
+  private HMSHandler hmsHandler;
+
+  // Thread local configuration is needed as many threads could make changes
+  // to the conf using the connection hook
+  private Configuration configuration;
+
+  // Thread local Map to keep track of modified meta conf keys
+  private Map<String, String> modifiedConfig = new HashMap<>();
+
+  private Integer threadId = nextSerialNum.incrementAndGet();
+  // This will only be set if the metastore is being accessed from a metastore 
Thrift server,
+  // not if it is from the CLI. Also, only if the TTransport being used to 
connect is an
+  // instance of TSocket. This is also not set when kerberos is used.
+  private String ipAddress;
+
+  private Map<String, com.codahale.metrics.Timer.Context> timerContexts = new 
HashMap<>();
+
+  private HMSHandlerContext() {
+
+  }
+
+  public static Optional<RawStore> getRawStore() {
+    HMSHandlerContext ctx = context.get();
+    return ctx != null ? ctx.getLocalRawStore() : Optional.empty();
+  }
+
+  public static Optional<HMSHandler> getHMSHandler() {
+    HMSHandlerContext ctx = context.get();
+    return ctx != null ? ctx.getLocalHmsHandler() : Optional.empty();
+  }
+
+  public static Optional<String> getIpAddress() {
+    HMSHandlerContext ctx = context.get();
+    return ctx != null ? ctx.getRemoteIpAddress() : Optional.empty();
+  }
+
+  public static Optional<Configuration> getConfiguration() {
+    HMSHandlerContext ctx = context.get();
+    return ctx != null ? ctx.getLocalConfiguration() : Optional.empty();
+  }
+
+  public static TxnStore getTxnStore(Configuration conf) {
+    return getContext().getLocalTxnStore().orElseGet(() -> {
+      TxnStore txnStore = TxnUtils.getTxnStore(conf);
+      setTxnStore(txnStore);
+      return txnStore;
+    });
+  }
+
+  public static Map<String, String> getModifiedConfig() {
+    return getContext().modifiedConfig;
+  }
+
+  public static Integer getThreadId() {
+    return getContext().threadId;
+  }
+
+  public static Map<String, com.codahale.metrics.Timer.Context> 
getTimerContexts() {
+    return getContext().timerContexts;
+  }
+
+  private static HMSHandlerContext getContext() {
+    HMSHandlerContext ctx = context.get();
+    if (ctx == null) {
+      context.set(ctx = new HMSHandlerContext());
+    }
+    return ctx;
+  }
+
+  public static void setRawStore(RawStore rawStore) {
+    getContext().rawStore = rawStore;
+  }
+
+  public static void setTxnStore(TxnStore txnStore) {
+    getContext().txnStore = txnStore;
+  }
+
+  public static void setHMSHandler(HMSHandler hmsHandler) {
+    getContext().hmsHandler = hmsHandler;
+  }
+
+  public static void setConfiguration(Configuration conf) {
+    getContext().configuration = conf;
+  }
+
+  public static void setIpAddress(String ipAddress) {
+    getContext().ipAddress = ipAddress;
+  }
+
+  public static void clear(CleanupHook cleanupHook) {
+    HMSHandlerContext ctx = context.get();
+    context.remove();
+    if (ctx != null && cleanupHook != null) {

Review comment:
       Feels a bit like overengineering to me.
   Why not just do this the `cleanupHook` stuff outside, and clear the values 
here?




-- 
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: 714375)
    Time Spent: 50m  (was: 40m)

> Group HMSHandler's thread locals into a single context
> ------------------------------------------------------
>
>                 Key: HIVE-25892
>                 URL: https://issues.apache.org/jira/browse/HIVE-25892
>             Project: Hive
>          Issue Type: Improvement
>          Components: Standalone Metastore
>            Reporter: Zhihua Deng
>            Priority: Minor
>              Labels: pull-request-available
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> There are more than six ThreadLocal variables in HMSHandler, we can group 
> them together into a single context to improve the management of variables 
> and the code readability.
>  



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to