Aggarwal-Raghav commented on code in PR #6213:
URL: https://github.com/apache/hive/pull/6213#discussion_r2569747540


##########
ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java:
##########
@@ -0,0 +1,492 @@
+/*
+ * 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.ql.hooks;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.llap.registry.impl.LlapRegistryService;
+import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.exec.ExplainTask;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.tez.TezTask;
+import org.apache.hadoop.hive.ql.log.PerfLogger;
+import org.apache.hadoop.hive.ql.parse.ExplainConfiguration;
+import org.apache.hadoop.hive.ql.plan.ExplainWork;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hive.common.util.ShutdownHookManager;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * ATSHook sends query + plan info to Yarn App Timeline Server. To enable 
(hadoop 2.4 and up) set
+ * hive.exec.pre.hooks/hive.exec.post.hooks/hive.exec.failure.hooks to include 
this class.
+ */
+public class ATSHook implements ExecuteWithHookContext {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ATSHook.class.getName());
+  private boolean isATSEnabled = false;
+  private static final Object LOCK = new Object();
+  private static final int VERSION = 2;
+  private static ExecutorService executor;
+  private static ExecutorService senderExecutor;
+  private static TimelineClient timelineClient;
+  private enum EntityTypes { HIVE_QUERY_ID };
+  private enum EventTypes { QUERY_SUBMITTED, QUERY_COMPLETED };
+  private static final String ATS_DOMAIN_PREFIX = "hive_";
+  private static boolean defaultATSDomainCreated = false;
+  private static final String DEFAULT_ATS_DOMAIN = "hive_default_ats";
+
+  @VisibleForTesting
+  enum OtherInfoTypes {
+    QUERY, STATUS, TEZ, MAPRED, INVOKER_INFO, SESSION_ID, THREAD_NAME, VERSION,
+    CLIENT_IP_ADDRESS, HIVE_ADDRESS, HIVE_INSTANCE_TYPE, CONF, PERF, 
LLAP_APP_ID
+  };
+  private enum ExecutionMode {
+    MR, TEZ, LLAP, SPARK, NONE
+  };
+  private enum PrimaryFilterTypes {
+    user, requestuser, operationid, executionmode, tablesread, tableswritten, 
queue
+  };
+
+  private static final int WAIT_TIME = 3;
+
+  private static final String[] PERF_KEYS = new String[] {
+    PerfLogger.PARSE, PerfLogger.COMPILE, PerfLogger.ANALYZE, 
PerfLogger.OPTIMIZER,
+    PerfLogger.GET_SPLITS, PerfLogger.RUN_TASKS,
+  };
+
+  private static void setupAtsExecutor(HiveConf conf) {
+    synchronized(LOCK) {
+      if (executor == null) {
+
+        // The call to ATS appears to block indefinitely, blocking the ATS 
thread while
+        // the hook continues to submit work to the ExecutorService with each 
query.
+        // Over time the queued items can cause OOM as the HookContext seems 
to contain
+        // some items which use a lot of memory.
+        // Prevent this situation by creating executor with bounded capacity -
+        // the event will not be sent to ATS if there are too many outstanding 
work submissions.
+        int queueCapacity = conf.getIntVar(ConfVars.ATS_HOOK_QUEUE_CAPACITY);
+
+        // Executor to create the ATS events.
+        // This can use significant resources and should not be done on the 
main query thread.
+        LOG.info("Creating ATS executor queue with capacity " + queueCapacity);
+        BlockingQueue<Runnable> queue = new 
LinkedBlockingQueue<Runnable>(queueCapacity);
+        ThreadFactory threadFactory = new 
ThreadFactoryBuilder().setDaemon(true).setNameFormat("ATS Logger %d").build();
+        executor = new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS, 
queue, threadFactory);
+
+        // Create a separate thread to send the events.
+        // Keep separate from the creating events in case the send blocks.
+        BlockingQueue<Runnable> senderQueue = new 
LinkedBlockingQueue<Runnable>(queueCapacity);
+        senderExecutor = new ThreadPoolExecutor(1, 1, 0, 
TimeUnit.MILLISECONDS, senderQueue, threadFactory);
+
+        YarnConfiguration yarnConf = new YarnConfiguration();

Review Comment:
   Tez view uses yarn timeline server to pass the query and plan. Based on the 
comments in ATS, yes it's for YARN deployments.
   
   Ther was a mail thread on this 
https://www.mail-archive.com/[email protected]/msg145250.html as mentioned by 
@zhangbutao on JIRA, but there was no response. After testing on my setup, i 
confirmed tez-view uses ATSHook. But query graphical plan and query string is 
present in DAG as well. So, the dilemma is wheter to keep ATSHook reverted as 
majorty info is present in DAG or re-enable it just to populate info in `Hive 
Queries` tab



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to