abstractdog commented on code in PR #5613:
URL: https://github.com/apache/hive/pull/5613#discussion_r1927056806


##########
ql/src/java/org/apache/hadoop/hive/ql/queryhistory/QueryHistoryService.java:
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.queryhistory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.DriverContext;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ServiceContext;
+import org.apache.hadoop.hive.ql.queryhistory.persist.QueryHistoryPersistor;
+import org.apache.hadoop.hive.ql.queryhistory.schema.QueryHistoryRecord;
+import org.apache.hadoop.hive.ql.queryhistory.schema.QueryHistorySchema;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hive.common.util.ReflectionUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public class QueryHistoryService {
+  private static final Logger LOG = 
LoggerFactory.getLogger(QueryHistoryService.class);
+
+  private static QueryHistoryService INSTANCE = null;
+
+  private QueryHistoryPersistor queryHistoryPersistor;
+
+  protected final Queue<QueryHistoryRecord> queryHistoryQueue = new 
LinkedBlockingQueue<>();
+  private int maxBatchSize = 0;
+  private int maxQueueSizeInMemoryBytes = 0;
+
+  private final ExecutorService persistExecutor = 
Executors.newFixedThreadPool(1,
+      new ThreadFactoryBuilder().setNameFormat("QueryHistoryService persistor 
thread").build());
+
+  private ServiceContext serviceContext;
+
+  public static QueryHistoryService start(HiveConf inputConf, ServiceContext 
serviceContext) {
+    HiveConf conf = new HiveConf(inputConf);
+    // we can only hit this codepath if the INSTANCE is already initialized 
while calling start(...)
+    // for convenient test compatibility, this call is allowed to be 
idempotent (LOG.warn instead of exception)
+    if (INSTANCE != null) {
+      LOG.warn("There is already a QueryHistoryService instance ({}), 
returning existing instance", INSTANCE);
+      return INSTANCE;
+    }
+
+    INSTANCE = createServiceInstance(serviceContext, conf);
+    return INSTANCE;
+  }
+
+  private static QueryHistoryService createServiceInstance(ServiceContext 
serviceContext, HiveConf conf) {
+    LOG.info("Starting QueryHistoryService");
+    QueryHistoryService queryHistoryService = new QueryHistoryService();
+
+    initService(serviceContext, conf, queryHistoryService);
+    return queryHistoryService;
+  }
+
+  @VisibleForTesting
+  static void initService(ServiceContext serviceContext, HiveConf conf, 
QueryHistoryService queryHistoryService) {
+    queryHistoryService.queryHistoryPersistor = createPersistor(conf);
+    queryHistoryService.queryHistoryPersistor.init(conf, new 
QueryHistorySchema());
+    queryHistoryService.maxBatchSize = HiveConf.getIntVar(conf,
+        HiveConf.ConfVars.HIVE_QUERY_HISTORY_SERVICE_PERSIST_MAX_BATCH_SIZE);
+    queryHistoryService.maxQueueSizeInMemoryBytes = HiveConf.getIntVar(conf,
+        HiveConf.ConfVars.HIVE_QUERY_HISTORY_SERVICE_PERSIST_MAX_MEMORY_BYTES);
+
+    queryHistoryService.printConfigInformation();
+    queryHistoryService.serviceContext = serviceContext;
+
+    try {
+      queryHistoryService.persistExecutor.submit(() -> {
+        // this session won't need tez, let's not mess with starting a 
TezSessionState
+        conf.setBoolVar(HiveConf.ConfVars.HIVE_CLI_TEZ_INITIALIZE_SESSION, 
false);
+
+        // localize this conf for the thread that will be running the 
persistor,
+        // as iceberg integration heavily relies on the session state conf
+        SessionState session = SessionState.start(conf);

Review Comment:
   SessionState is bound to a thread, and persistExecutor is a single-threaded 
executor by design, so this task inside it takes care of connecting the conf 
present in QueryHistoryService to be present for all the repository 
tasks,that's what I meant by "localize this conf for the thread that will be 
running the repository"



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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscr...@hive.apache.org
For additional commands, e-mail: gitbox-h...@hive.apache.org

Reply via email to