[
https://issues.apache.org/jira/browse/HIVE-26242?focusedWorklogId=785091&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-785091
]
ASF GitHub Bot logged work on HIVE-26242:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 27/Jun/22 13:21
Start Date: 27/Jun/22 13:21
Worklog Time Spent: 10m
Work Description: deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907382982
##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.txn.compactor;
+
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.impl.GenericObjectPool;
+import org.apache.commons.pool2.impl.GenericObjectPoolConfig;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hive.common.util.ShutdownHookManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static
org.apache.hive.common.util.HiveStringUtils.SHUTDOWN_HOOK_PRIORITY;
+
+/**
+ * Singleton service responsible for heartbeating the compaction transactions.
+ */
+class CompactionHeartbeatService {
+
+
+ private static final Logger LOG =
LoggerFactory.getLogger(CompactionHeartbeatService.class);
+
+ private static volatile CompactionHeartbeatService instance;
+
+ /**
+ * Return the singleton instance of this class.
+ * @param conf The {@link HiveConf} used to create the service. Used only
during the firsst call
+ * @return Returns the singleton {@link CompactionHeartbeatService}
+ * @throws IllegalStateException Thrown when the service has already been
shut down.
+ */
+ static CompactionHeartbeatService getInstance(HiveConf conf) {
+ if (instance == null) {
+ synchronized (CompactionHeartbeatService.class) {
+ if (instance == null) {
+ LOG.debug("Initializing compaction txn heartbeater service.");
+ instance = new CompactionHeartbeatService(conf);
+ ShutdownHookManager.addShutdownHook(() -> {
+ try {
+ instance.shutdown();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }, SHUTDOWN_HOOK_PRIORITY);
+ }
+ }
+ }
+ if (instance.heartbeatExecutor.isShutdown()) {
+ throw new IllegalStateException("The CompactionHeartbeatService is
already shut down!");
+ }
+ return instance;
+ }
+
+ private final ScheduledExecutorService heartbeatExecutor;
+ private final ObjectPool<IMetaStoreClient> clientPool;
+ private final long initialDelay;
+ private final long period;
+ private final HashMap<Long, TaskWrapper> tasks = new HashMap<>(30);
+
+ /**
+ * Starts the heartbeat for the given transaction
+ * @param txnId The id of the compaction txn
+ * @param lockId The id of the lock associated with the txn
+ * @param tableName Required for logging only
+ * @throws IllegalStateException Thrown when the heartbeat for the given txn
has already been started.
+ */
+ void startHeartbeat(long txnId, long lockId, String tableName) {
+ if (tasks.containsKey(txnId)) {
+ throw new IllegalStateException("Heartbeat already started for TXN " +
txnId);
Review Comment:
"was" already started
Issue Time Tracking
-------------------
Worklog Id: (was: 785091)
Time Spent: 6h 10m (was: 6h)
> Compaction heartbeater improvements
> -----------------------------------
>
> Key: HIVE-26242
> URL: https://issues.apache.org/jira/browse/HIVE-26242
> Project: Hive
> Issue Type: Improvement
> Reporter: László Végh
> Assignee: László Végh
> Priority: Major
> Labels: pull-request-available
> Time Spent: 6h 10m
> Remaining Estimate: 0h
>
> The Compaction heartbeater should be improved the following ways:
> * The metastore clients should be reused between heartbeats and closed only
> at the end, when the transaction ends
> * Instead of having a dedicated heartbeater thread for each Compaction
> transaction, there should be shared a heartbeater executor where the
> heartbeat tasks can be scheduled/submitted.
--
This message was sent by Atlassian Jira
(v8.20.7#820007)