You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2022/05/20 06:48:35 UTC

[GitHub] [hive] veghlaci05 opened a new pull request, #3303: HIVE-26242: Compaction heartbeater improvements

veghlaci05 opened a new pull request, #3303:
URL: https://github.com/apache/hive/pull/3303

   ### What changes were proposed in this pull request?
   This PR introduces a new component (CompactionHeartbeatService) to centralize the compaction transaction heartbeating, and reduce resource usage. It consist of a Scheduled exevutor service and a MetaStoreClient pool.
   
   ### Why are the changes needed?
   The heartbeat of the compaction txns are wasting resources by having a dedicated separate thread and MetaStore client for every compaction transaction.
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   
   ### How was this patch tested?
   Manually and through unit tests.


-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908567842


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);

Review Comment:
   it's true by default, why do we need to set it explicitly?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908594062


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+          clientPool.returnObject(msc);

Review Comment:
   No, we don't want to return a previously invalidated object into the pool.



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907674601


##########
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.Future;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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 destroyed!");
+    }
+    return instance;
+  }
+
+  private final ScheduledThreadPoolExecutor 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 was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish(initialDelay);
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    heartbeatExecutor = new ScheduledThreadPoolExecutor(0);
+    heartbeatExecutor.setRemoveOnCancelPolicy(true);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();
+        msc.heartbeat(txnId, lockId);
+        clientPool.returnObject(msc);
+      } catch (NoSuchElementException nsee) {
+        LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", nsee);
+      } catch (Exception e) {
+        LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        try {
+          clientPool.invalidateObject(msc);
+        } catch (Exception ex) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      } finally {
+        synchronized (lock) {
+          running = false;
+          lock.notifyAll();
+        }
+      }
+    }
+
+    public void waitUntilFinish(long timeout) throws InterruptedException {
+      synchronized (lock) {
+        if (running) {

Review Comment:
   If it's while it makes no sense to have timeout, we can possibly wait forever.



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887728099


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Singleton service responsible for heartbeating the compaction transactions.
+ */
+class CompactionHeartbeatService {
+
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionHeartbeatService.class);
+
+  private static volatile CompactionHeartbeatService INSTANCE;

Review Comment:
   shouldn't be uppercase if it's not marked as `final`



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887856299


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(30);

Review Comment:
   why should it be a concurrent map, I don't see any usage of it in the code



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(30);

Review Comment:
   why should it be a concurrent map, I don't see any usage of it in the code?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908552785


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+          clientPool.returnObject(msc);
+        } catch (NoSuchElementException e) {
+          LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", e);
+        } catch (TException e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        } catch (Exception e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+          if (msc != null) {
+            try {
+              clientPool.invalidateObject(msc);
+            } catch (Exception ex) {
+              LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+            }
+          }
+        }
+      }, initialDelay, period, TimeUnit.MILLISECONDS);
+    }
+
+    public void stop() throws InterruptedException {
+      LOG.info("Shutting down compaction txn heartbeater instance.");
+      heartbeatExecutor.shutdownNow();

Review Comment:
   please add the null check:
   ````
   if (heartbeatExecutor != null) {
   ...
   }
   ````



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908586905


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+          clientPool.returnObject(msc);
+        } catch (NoSuchElementException e) {
+          LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", e);
+        } catch (TException e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        } catch (Exception e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+          if (msc != null) {
+            try {
+              clientPool.invalidateObject(msc);
+            } catch (Exception ex) {
+              LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+            }
+          }
+        }
+      }, initialDelay, period, TimeUnit.MILLISECONDS);
+    }
+
+    public void stop() throws InterruptedException {
+      LOG.info("Shutting down compaction txn heartbeater instance.");
+      heartbeatExecutor.shutdownNow();
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+      LOG.info("Compaction txn heartbeater instance is successfully stopped.");
+    }
+
+    private CompactionHeartbeater(long txnId, long lockId, String tableName) {

Review Comment:
   could we place constructor as the first method



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908582550


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+          clientPool.returnObject(msc);
+        } catch (NoSuchElementException e) {
+          LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", e);
+        } catch (TException e) {

Review Comment:
   Not exactly, for TException the client is not invalidated, because it is not necessary (NoSuchTxnException, etc are not client failures)



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907496006


##########
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.Future;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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 destroyed!");
+    }
+    return instance;
+  }
+
+  private final ScheduledThreadPoolExecutor 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 was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish(initialDelay);
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    heartbeatExecutor = new ScheduledThreadPoolExecutor(0);
+    heartbeatExecutor.setRemoveOnCancelPolicy(true);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();
+        msc.heartbeat(txnId, lockId);
+        clientPool.returnObject(msc);
+      } catch (NoSuchElementException nsee) {
+        LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", nsee);
+      } catch (Exception e) {
+        LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        try {
+          clientPool.invalidateObject(msc);
+        } catch (Exception ex) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      } finally {
+        synchronized (lock) {
+          running = false;
+          lock.notifyAll();
+        }
+      }
+    }
+
+    public void waitUntilFinish(long timeout) throws InterruptedException {
+      synchronized (lock) {
+        if (running) {

Review Comment:
   should be `while`



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887880115


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);

Review Comment:
   heartbeatExecutor pool is shared now and it's size is 1, how could it be used to heartbeat multiple transactions?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887896665


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(5);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private static final class IMetaStoreClientFactory extends BasePooledObjectFactory<IMetaStoreClient> {
+
+    private final HiveConf conf;
+
+    @Override
+    public IMetaStoreClient create() throws Exception {
+      return HiveMetaStoreUtils.getHiveMetastoreClient(conf);
+    }
+
+    @Override
+    public PooledObject<IMetaStoreClient> wrap(IMetaStoreClient iMetaStoreClient) {
+      return new DefaultPooledObject<>(iMetaStoreClient);
+    }
+
+    @Override
+    public void destroyObject(PooledObject<IMetaStoreClient> p) {
+      p.getObject().close();
+    }
+
+    @Override
+    public boolean validateObject(PooledObject<IMetaStoreClient> p) {
+      //Not in use currently, would be good to validate the client at borrowing/returning, but this needs support from
+      //MetaStoreClient side
+      return super.validateObject(p);
+    }
+
+    public IMetaStoreClientFactory(HiveConf conf) {
+      this.conf = Objects.requireNonNull(conf);
+    }
+
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();
+        msc.heartbeat(txnId, lockId);
+        clientPool.returnObject(msc);
+      } catch (NoSuchElementException nsee) {
+        LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", nsee);
+      } catch (Exception e) {
+        LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        try {
+          clientPool.invalidateObject(msc);
+        } catch (Exception ex) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      } finally {
+        synchronized (lock) {
+          running = false;
+          lock.notifyAll();
+        }
+      }
+    }
+
+    public void waitUntilFinish() throws InterruptedException {

Review Comment:
   Join is only in `CompletableFuture<T>`, but not in `ScheduledFuture<T>` which the `ScheduledExecutorService` returns.



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887888610


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(5);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private static final class IMetaStoreClientFactory extends BasePooledObjectFactory<IMetaStoreClient> {
+
+    private final HiveConf conf;
+
+    @Override
+    public IMetaStoreClient create() throws Exception {
+      return HiveMetaStoreUtils.getHiveMetastoreClient(conf);
+    }
+
+    @Override
+    public PooledObject<IMetaStoreClient> wrap(IMetaStoreClient iMetaStoreClient) {
+      return new DefaultPooledObject<>(iMetaStoreClient);
+    }
+
+    @Override
+    public void destroyObject(PooledObject<IMetaStoreClient> p) {
+      p.getObject().close();
+    }
+
+    @Override
+    public boolean validateObject(PooledObject<IMetaStoreClient> p) {
+      //Not in use currently, would be good to validate the client at borrowing/returning, but this needs support from
+      //MetaStoreClient side
+      return super.validateObject(p);
+    }
+
+    public IMetaStoreClientFactory(HiveConf conf) {
+      this.conf = Objects.requireNonNull(conf);
+    }
+
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();
+        msc.heartbeat(txnId, lockId);
+        clientPool.returnObject(msc);
+      } catch (NoSuchElementException nsee) {
+        LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", nsee);
+      } catch (Exception e) {
+        LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        try {
+          clientPool.invalidateObject(msc);
+        } catch (Exception ex) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      } finally {
+        synchronized (lock) {
+          running = false;
+          lock.notifyAll();
+        }
+      }
+    }
+
+    public void waitUntilFinish() throws InterruptedException {
+      synchronized (lock) {
+        if (running) {
+          lock.wait();
+        }
+      }
+    }
+
+    private CompactionHeartbeater(long txnId, long lockId, String tableName) {
+      this.tableName = Objects.requireNonNull(tableName);
+      this.txnId = txnId;
+      this.lockId = lockId;
+    }
+
+  }
+
+  private static final class TaskWrapper {

Review Comment:
   see above



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887725752


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Singleton service responsible for heartbeating the compaction transactions.
+ */
+class CompactionHeartbeatService {
+
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionHeartbeatService.class);
+
+  private static volatile CompactionHeartbeatService INSTANCE;

Review Comment:
   why not `on demand holder` idiom



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887865099


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();

Review Comment:
   The only blocking calls in the executor thread are creating the MetaStore client and calling the heartbeat method. Does the client has its own timeout functionality? If yes, I think there is no need to introduce the same logic here again. If there is no such construct in the client, then it is a good point.



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887863531


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();

Review Comment:
   why do we need to handle InterruptedException here and not in upstream code?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908547594


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+          clientPool.returnObject(msc);
+        } catch (NoSuchElementException e) {
+          LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", e);
+        } catch (TException e) {

Review Comment:
   could we join  TException & Exception, as we handle them in a same fashion



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907376912


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/IMetaStoreClientFactory.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+
+import java.util.Objects;
+
+final class IMetaStoreClientFactory extends BasePooledObjectFactory<IMetaStoreClient> {
+
+  private final HiveConf conf;
+
+  @Override
+  public IMetaStoreClient create() throws Exception {
+    return HiveMetaStoreUtils.getHiveMetastoreClient(conf);
+  }
+
+  @Override
+  public PooledObject<IMetaStoreClient> wrap(IMetaStoreClient iMetaStoreClient) {
+    return new DefaultPooledObject<>(iMetaStoreClient);
+  }
+
+  @Override
+  public void destroyObject(PooledObject<IMetaStoreClient> p) {
+    p.getObject().close();
+  }
+
+  @Override
+  public boolean validateObject(PooledObject<IMetaStoreClient> p) {

Review Comment:
   p -> msc



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908585359


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+          clientPool.returnObject(msc);
+        } catch (NoSuchElementException e) {
+          LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", e);
+        } catch (TException e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        } catch (Exception e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+          if (msc != null) {
+            try {
+              clientPool.invalidateObject(msc);
+            } catch (Exception ex) {
+              LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+            }
+          }
+        }
+      }, initialDelay, period, TimeUnit.MILLISECONDS);
+    }
+
+    public void stop() throws InterruptedException {
+      LOG.info("Shutting down compaction txn heartbeater instance.");
+      heartbeatExecutor.shutdownNow();

Review Comment:
   heartbeatExecutor is instantiated in the CompactionHeartbeatService constructor, null check is not necessary I think.



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907373970


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java:
##########
@@ -692,18 +657,7 @@ void open(CompactionInfo ci) throws TException {
             + "}, status {" + res.getState() + "}, reason {" + res.getErrorMessage() + "}");
       }
       lockId = res.getLockid();
-      heartbeatExecutor = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryBuilder()
-                .setPriority(Thread.MIN_PRIORITY)
-                .setDaemon(true)
-                .setNameFormat("CompactionTxnHeartbeater-" + txnId)
-                .build());
-      long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
-      heartbeatExecutor.scheduleAtFixedRate(
-              new CompactionHeartbeater(this, TxnUtils.getFullTableName(ci.dbname, ci.tableName), conf),
-              txnTimeout / 4,
-              txnTimeout / 2,
-              TimeUnit.MILLISECONDS
-      );
+      CompactionHeartbeatService.getInstance(conf).startHeartbeat(txnId, lockId, ci.tableName);

Review Comment:
   should we pass `TxnUtils.getFullTableName(ci.dbname, ci.tableName)`



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java:
##########
@@ -692,18 +657,7 @@ void open(CompactionInfo ci) throws TException {
             + "}, status {" + res.getState() + "}, reason {" + res.getErrorMessage() + "}");
       }
       lockId = res.getLockid();
-      heartbeatExecutor = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryBuilder()
-                .setPriority(Thread.MIN_PRIORITY)
-                .setDaemon(true)
-                .setNameFormat("CompactionTxnHeartbeater-" + txnId)
-                .build());
-      long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
-      heartbeatExecutor.scheduleAtFixedRate(
-              new CompactionHeartbeater(this, TxnUtils.getFullTableName(ci.dbname, ci.tableName), conf),
-              txnTimeout / 4,
-              txnTimeout / 2,
-              TimeUnit.MILLISECONDS
-      );
+      CompactionHeartbeatService.getInstance(conf).startHeartbeat(txnId, lockId, ci.tableName);

Review Comment:
   should we pass `TxnUtils.getFullTableName(ci.dbname, ci.tableName)`?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887886357


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();

Review Comment:
   what if timeout set high?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887904369


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(5);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private static final class IMetaStoreClientFactory extends BasePooledObjectFactory<IMetaStoreClient> {
+
+    private final HiveConf conf;
+
+    @Override
+    public IMetaStoreClient create() throws Exception {
+      return HiveMetaStoreUtils.getHiveMetastoreClient(conf);
+    }
+
+    @Override
+    public PooledObject<IMetaStoreClient> wrap(IMetaStoreClient iMetaStoreClient) {
+      return new DefaultPooledObject<>(iMetaStoreClient);
+    }
+
+    @Override
+    public void destroyObject(PooledObject<IMetaStoreClient> p) {
+      p.getObject().close();
+    }
+
+    @Override
+    public boolean validateObject(PooledObject<IMetaStoreClient> p) {
+      //Not in use currently, would be good to validate the client at borrowing/returning, but this needs support from
+      //MetaStoreClient side
+      return super.validateObject(p);
+    }
+
+    public IMetaStoreClientFactory(HiveConf conf) {
+      this.conf = Objects.requireNonNull(conf);
+    }
+
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();

Review Comment:
   This is the catch block responsible for it. Can we do anything else here?
   
   `} catch (NoSuchElementException nsee) {
      LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", nsee);
   }`



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887876080


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(5);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);

Review Comment:
   Yes, `GenericObjectPool` is [thread safe](https://commons.apache.org/proper/commons-pool/apidocs/org/apache/commons/pool2/impl/GenericObjectPool.html)



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887868543


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(5);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private static final class IMetaStoreClientFactory extends BasePooledObjectFactory<IMetaStoreClient> {
+
+    private final HiveConf conf;
+
+    @Override
+    public IMetaStoreClient create() throws Exception {
+      return HiveMetaStoreUtils.getHiveMetastoreClient(conf);
+    }
+
+    @Override
+    public PooledObject<IMetaStoreClient> wrap(IMetaStoreClient iMetaStoreClient) {
+      return new DefaultPooledObject<>(iMetaStoreClient);
+    }
+
+    @Override
+    public void destroyObject(PooledObject<IMetaStoreClient> p) {
+      p.getObject().close();
+    }
+
+    @Override
+    public boolean validateObject(PooledObject<IMetaStoreClient> p) {
+      //Not in use currently, would be good to validate the client at borrowing/returning, but this needs support from
+      //MetaStoreClient side
+      return super.validateObject(p);
+    }
+
+    public IMetaStoreClientFactory(HiveConf conf) {
+      this.conf = Objects.requireNonNull(conf);
+    }
+
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();
+        msc.heartbeat(txnId, lockId);
+        clientPool.returnObject(msc);
+      } catch (NoSuchElementException nsee) {
+        LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", nsee);
+      } catch (Exception e) {
+        LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        try {
+          clientPool.invalidateObject(msc);
+        } catch (Exception ex) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      } finally {
+        synchronized (lock) {
+          running = false;
+          lock.notifyAll();
+        }
+      }
+    }
+
+    public void waitUntilFinish() throws InterruptedException {

Review Comment:
   can't we simply use `join` for this?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887884610


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(30);

Review Comment:
   and? 



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887889651


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);

Review Comment:
   not keep, but the execution is single-threaded



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] klcopp commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
klcopp commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r905820030


##########
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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish(initialDelay);
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(
+        MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS));
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(5);

Review Comment:
   This is the max number of HMS clients in the pool right? Shouldn't we set this to == COMPACTOR_WORKER_THREADS since that is the number of concurrent compactions that could be running?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907674601


##########
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.Future;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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 destroyed!");
+    }
+    return instance;
+  }
+
+  private final ScheduledThreadPoolExecutor 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 was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish(initialDelay);
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    heartbeatExecutor = new ScheduledThreadPoolExecutor(0);
+    heartbeatExecutor.setRemoveOnCancelPolicy(true);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();
+        msc.heartbeat(txnId, lockId);
+        clientPool.returnObject(msc);
+      } catch (NoSuchElementException nsee) {
+        LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", nsee);
+      } catch (Exception e) {
+        LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        try {
+          clientPool.invalidateObject(msc);
+        } catch (Exception ex) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      } finally {
+        synchronized (lock) {
+          running = false;
+          lock.notifyAll();
+        }
+      }
+    }
+
+    public void waitUntilFinish(long timeout) throws InterruptedException {
+      synchronized (lock) {
+        if (running) {

Review Comment:
   If it's while it makes no sense to have timeout, we can possibly wait forever. I think we should not block application shutdown/commit/rollback at this point, if the timeout is not enough for the heartbeater thread to finish, we should interrupt  it anyway.



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908575155


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+          clientPool.returnObject(msc);

Review Comment:
   shouldn't it be in finally block?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908596948


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+          clientPool.returnObject(msc);
+        } catch (NoSuchElementException e) {
+          LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", e);

Review Comment:
   This heartbeat failed, but it has no effect on subsequent heartbeats. Since it will be retried because of the scheduling, I did not want to make it more complex by introducing some retry logic 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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908593432


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);

Review Comment:
   private final



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908575155


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+          clientPool.returnObject(msc);

Review Comment:
   shouldn't it be in finally block? you are not returning msc back to the pool in case of exception



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907474234


##########
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.Future;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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 destroyed!");
+    }
+    return instance;
+  }
+
+  private final ScheduledThreadPoolExecutor 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 was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish(initialDelay);
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    heartbeatExecutor = new ScheduledThreadPoolExecutor(0);
+    heartbeatExecutor.setRemoveOnCancelPolicy(true);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {

Review Comment:
   why do we need to synchronize accesst o primitive volatile variable?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908547594


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+          clientPool.returnObject(msc);
+        } catch (NoSuchElementException e) {
+          LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", e);
+        } catch (TException e) {

Review Comment:
   should we join  TException & Exception? is it ok just to log an error and do nothing?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907381578


##########
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!");

Review Comment:
   maybe "CompactionHeartbeatService is already destroyed" ?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907377882


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/IMetaStoreClientFactory.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+
+import java.util.Objects;
+

Review Comment:
   javadoc is missing



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r909292463


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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(() -> instance.shutdown(), SHUTDOWN_HOOK_PRIORITY);
+        }
+      }
+    }
+    if (instance.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   */
+  void shutdown() {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setMaxWaitMillis(2000);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    private final Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+    private CompactionHeartbeater(long txnId, long lockId, String tableName) {
+      heartbeatExecutor = new ScheduledThreadPoolExecutor(1);
+      this.tableName = Objects.requireNonNull(tableName);
+      this.txnId = txnId;
+      this.lockId = lockId;
+    }
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+        } catch (NoSuchElementException e) {
+          LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", e);
+          // This heartbeat attempt failed, and there is no client to return to the pool.
+          return;
+        } catch (TException e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+          // Heartbeat failed, but the client is not broken, we can return it to the pool.
+        } catch (Exception e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+          // Unknown error, invalidate the client, maybe it is broken.
+          if (msc != null) {
+            try {
+              clientPool.invalidateObject(msc);
+            } catch (Exception ex) {
+              LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+            }
+          }
+          return;
+        }
+        try {
+          if (msc != null) {
+            clientPool.returnObject(msc);
+          }
+        } catch (Exception e) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      }, initialDelay, period, TimeUnit.MILLISECONDS);
+    }
+
+    public void stop() {
+      LOG.info("Shutting down compaction txn heartbeater instance.");
+      heartbeatExecutor.shutdownNow();
+      try {
+        if (!heartbeatExecutor.awaitTermination(5, TimeUnit.SECONDS)) {
+          LOG.warn("Heartbeating for transaction {} did not stop in 5 seconds, do not wait any longer.", this);
+          return;
+        }
+      } catch (InterruptedException ex) {
+        //Caller thread was interrupted while waiting for heartbeater to terminate.
+        //Nothing to do, just restore the interrupted state.
+        Thread.currentThread().interrupt();

Review Comment:
   how would it behave in case shutdownhook is invoked? would it interrupt the shutdownhook thread?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r909287780


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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(() -> instance.shutdown(), SHUTDOWN_HOOK_PRIORITY);
+        }
+      }
+    }
+    if (instance.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   */
+  void shutdown() {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setMaxWaitMillis(2000);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    private final Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+    private CompactionHeartbeater(long txnId, long lockId, String tableName) {
+      heartbeatExecutor = new ScheduledThreadPoolExecutor(1);
+      this.tableName = Objects.requireNonNull(tableName);
+      this.txnId = txnId;
+      this.lockId = lockId;
+    }
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+        } catch (NoSuchElementException e) {
+          LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", e);
+          // This heartbeat attempt failed, and there is no client to return to the pool.
+          return;
+        } catch (TException e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+          // Heartbeat failed, but the client is not broken, we can return it to the pool.
+        } catch (Exception e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+          // Unknown error, invalidate the client, maybe it is broken.
+          if (msc != null) {
+            try {
+              clientPool.invalidateObject(msc);
+            } catch (Exception ex) {
+              LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+            }
+          }
+          return;
+        }
+        try {
+          if (msc != null) {
+            clientPool.returnObject(msc);
+          }
+        } catch (Exception e) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      }, initialDelay, period, TimeUnit.MILLISECONDS);
+    }
+
+    public void stop() {
+      LOG.info("Shutting down compaction txn heartbeater instance.");
+      heartbeatExecutor.shutdownNow();
+      try {
+        if (!heartbeatExecutor.awaitTermination(5, TimeUnit.SECONDS)) {
+          LOG.warn("Heartbeating for transaction {} did not stop in 5 seconds, do not wait any longer.", this);

Review Comment:
   you should pass txnId not 'this' as it's not CompactionTxn anymore



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887882124


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);

Review Comment:
   corePoolSize – the number of threads to keep in the pool, even if they are idle
   
   According to the javadoc it's not the maximum number of threads, however it's a good point: The maximum is not set



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887884610


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(30);

Review Comment:
   and? you are not manipulating the iterator



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887849154


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {

Review Comment:
   should we use ShutdownHookManager.addShutdownHook(runnable, SHUTDOWN_HOOK_PRIORITY);



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on PR #3303:
URL: https://github.com/apache/hive/pull/3303#issuecomment-1167729672

   > I think we should do something like this:
   > 
   > ```
   > ExecutorService executorService = CompactorUtil.createExecutorWithThreadFactory(numberOfWorkers, "CompactionHeartbeat-executor-thread-%d");
   >     
   > Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(() -> executorService.execute(heartbeater));
   > tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
   > ```
   
   What is the benefit of this construct?


-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907377344


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/IMetaStoreClientFactory.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+
+import java.util.Objects;
+
+final class IMetaStoreClientFactory extends BasePooledObjectFactory<IMetaStoreClient> {
+
+  private final HiveConf conf;
+
+  @Override
+  public IMetaStoreClient create() throws Exception {
+    return HiveMetaStoreUtils.getHiveMetastoreClient(conf);
+  }
+
+  @Override
+  public PooledObject<IMetaStoreClient> wrap(IMetaStoreClient iMetaStoreClient) {

Review Comment:
   iMetaStoreClient -> msc



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on PR #3303:
URL: https://github.com/apache/hive/pull/3303#issuecomment-1167566882

   I think we should do something like this:
   ````
       ExecutorService executorService = CompactorUtil.createExecutorWithThreadFactory(numberOfWorkers, "CompactionHeartbeat-executor-thread-%d");
       Ref<CompletableFuture<Void>> submittedTask = Ref.from(null);
       
       Future<?> scheduledTask = heartbeatExecutor.scheduleAtFixedRate(() -> {
         submittedTask.value = CompletableFuture.runAsync(heartbeater, executorService);
       }, initialDelay, period, TimeUnit.MILLISECONDS);
       tasks.put(txnId, new TaskWrapper(scheduledTask, submittedTask.value));
     }
   
   void stopHeartbeat(long txnId) throws InterruptedException {
       ....
       wrapper.scheduledTask.cancel(false);
       try {
         wrapper.submittedTask.get(initialDelay, 100);
       } finally {
         tasks.remove(txnId);
       }
       ....
     }
   ````


-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908551205


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+          clientPool.returnObject(msc);
+        } catch (NoSuchElementException e) {
+          LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", e);
+        } catch (TException e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        } catch (Exception e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+          if (msc != null) {
+            try {
+              clientPool.invalidateObject(msc);
+            } catch (Exception ex) {
+              LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+            }
+          }
+        }
+      }, initialDelay, period, TimeUnit.MILLISECONDS);
+    }
+
+    public void stop() throws InterruptedException {
+      LOG.info("Shutting down compaction txn heartbeater instance.");
+      heartbeatExecutor.shutdownNow();
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);

Review Comment:
   ````
             if (!heartbeatExecutor.awaitTermination(5, TimeUnit.SECONDS)) {
               LOG.warn("Heartbeating for transaction {} did not stop in 5 seconds, do not wait any longer.", this);
             }
   ````



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907474234


##########
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.Future;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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 destroyed!");
+    }
+    return instance;
+  }
+
+  private final ScheduledThreadPoolExecutor 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 was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish(initialDelay);
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    heartbeatExecutor = new ScheduledThreadPoolExecutor(0);
+    heartbeatExecutor.setRemoveOnCancelPolicy(true);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {

Review Comment:
   why do we need to synchronize access to the primitive volatile variable?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908551205


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+          clientPool.returnObject(msc);
+        } catch (NoSuchElementException e) {
+          LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", e);
+        } catch (TException e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        } catch (Exception e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+          if (msc != null) {
+            try {
+              clientPool.invalidateObject(msc);
+            } catch (Exception ex) {
+              LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+            }
+          }
+        }
+      }, initialDelay, period, TimeUnit.MILLISECONDS);
+    }
+
+    public void stop() throws InterruptedException {
+      LOG.info("Shutting down compaction txn heartbeater instance.");
+      heartbeatExecutor.shutdownNow();
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);

Review Comment:
   ````
   try {
             if (!heartbeatExecutor.awaitTermination(5, TimeUnit.SECONDS)) {
               LOG.warn("Heartbeating for transaction {} did not stop in 5 seconds, do not wait any longer.", this);
             }
           } catch (InterruptedException ex) {
             //Caller thread was interrupted while waiting for heartbeater to terminate.
             //Nothing to do, just restore the interrupted state.
             Thread.currentThread().interrupt();
           }
   ````



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r905928478


##########
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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish(initialDelay);
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(
+        MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS));
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(5);

Review Comment:
   You are right, the maxtotal could be equal to that value.



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887860283


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();

Review Comment:
   should we gracefully wait for a certain time and then issue a force terminate ?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887866285


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(30);

Review Comment:
   The map is used in the `startHeartbeat()` and `stopHeartBeat()` methods which can be called concurrently if there a re multiple worker threads configured.



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887870152


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(5);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private static final class IMetaStoreClientFactory extends BasePooledObjectFactory<IMetaStoreClient> {
+
+    private final HiveConf conf;
+
+    @Override
+    public IMetaStoreClient create() throws Exception {
+      return HiveMetaStoreUtils.getHiveMetastoreClient(conf);
+    }
+
+    @Override
+    public PooledObject<IMetaStoreClient> wrap(IMetaStoreClient iMetaStoreClient) {
+      return new DefaultPooledObject<>(iMetaStoreClient);
+    }
+
+    @Override
+    public void destroyObject(PooledObject<IMetaStoreClient> p) {
+      p.getObject().close();
+    }
+
+    @Override
+    public boolean validateObject(PooledObject<IMetaStoreClient> p) {
+      //Not in use currently, would be good to validate the client at borrowing/returning, but this needs support from
+      //MetaStoreClient side
+      return super.validateObject(p);
+    }
+
+    public IMetaStoreClientFactory(HiveConf conf) {
+      this.conf = Objects.requireNonNull(conf);
+    }
+
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();
+        msc.heartbeat(txnId, lockId);
+        clientPool.returnObject(msc);
+      } catch (NoSuchElementException nsee) {
+        LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", nsee);
+      } catch (Exception e) {
+        LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        try {
+          clientPool.invalidateObject(msc);
+        } catch (Exception ex) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      } finally {
+        synchronized (lock) {
+          running = false;
+          lock.notifyAll();
+        }
+      }
+    }
+
+    public void waitUntilFinish() throws InterruptedException {
+      synchronized (lock) {
+        if (running) {
+          lock.wait();
+        }
+      }
+    }
+
+    private CompactionHeartbeater(long txnId, long lockId, String tableName) {
+      this.tableName = Objects.requireNonNull(tableName);
+      this.txnId = txnId;
+      this.lockId = lockId;
+    }
+
+  }
+
+  private static final class TaskWrapper {

Review Comment:
   why do we need this?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887873535


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(5);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private static final class IMetaStoreClientFactory extends BasePooledObjectFactory<IMetaStoreClient> {

Review Comment:
   should be an independent class



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887887873


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(5);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private static final class IMetaStoreClientFactory extends BasePooledObjectFactory<IMetaStoreClient> {
+
+    private final HiveConf conf;
+
+    @Override
+    public IMetaStoreClient create() throws Exception {
+      return HiveMetaStoreUtils.getHiveMetastoreClient(conf);
+    }
+
+    @Override
+    public PooledObject<IMetaStoreClient> wrap(IMetaStoreClient iMetaStoreClient) {
+      return new DefaultPooledObject<>(iMetaStoreClient);
+    }
+
+    @Override
+    public void destroyObject(PooledObject<IMetaStoreClient> p) {
+      p.getObject().close();
+    }
+
+    @Override
+    public boolean validateObject(PooledObject<IMetaStoreClient> p) {
+      //Not in use currently, would be good to validate the client at borrowing/returning, but this needs support from
+      //MetaStoreClient side
+      return super.validateObject(p);
+    }
+
+    public IMetaStoreClientFactory(HiveConf conf) {
+      this.conf = Objects.requireNonNull(conf);
+    }
+
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();
+        msc.heartbeat(txnId, lockId);
+        clientPool.returnObject(msc);
+      } catch (NoSuchElementException nsee) {
+        LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", nsee);
+      } catch (Exception e) {
+        LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        try {
+          clientPool.invalidateObject(msc);
+        } catch (Exception ex) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      } finally {
+        synchronized (lock) {
+          running = false;
+          lock.notifyAll();
+        }
+      }
+    }
+
+    public void waitUntilFinish() throws InterruptedException {

Review Comment:
   Future has `join` or `get`



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907421759


##########
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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish(initialDelay);
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    heartbeatExecutor = Executors.newScheduledThreadPool(numberOfWorkers);

Review Comment:
   maximumPoolSize=Integer.MAX_VALUE, corePoolSize the number of threads to keep in the pool, even if they are idle. 
   Prefer a constructor with zero core threads for a shared pool to avoid blocking JVM exit



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908566146


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);

Review Comment:
   it's false by default, why do we need to set it explicitly?



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);

Review Comment:
   it's false by default, why do we need to set it explicitly?



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);

Review Comment:
   it's false by default, why do we need to set it explicitly?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908563081


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);

Review Comment:
   should it be configurable? what would happen if exceeded, compaction fails?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908585343


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+

Review Comment:
   nit,space



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907376716


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/IMetaStoreClientFactory.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+
+import java.util.Objects;
+
+final class IMetaStoreClientFactory extends BasePooledObjectFactory<IMetaStoreClient> {
+
+  private final HiveConf conf;
+
+  @Override
+  public IMetaStoreClient create() throws Exception {
+    return HiveMetaStoreUtils.getHiveMetastoreClient(conf);
+  }
+
+  @Override
+  public PooledObject<IMetaStoreClient> wrap(IMetaStoreClient iMetaStoreClient) {
+    return new DefaultPooledObject<>(iMetaStoreClient);
+  }
+
+  @Override
+  public void destroyObject(PooledObject<IMetaStoreClient> p) {

Review Comment:
   p -> msc



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908142123


##########
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.Future;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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 destroyed!");
+    }
+    return instance;
+  }
+
+  private final ScheduledThreadPoolExecutor 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 was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish(initialDelay);
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    heartbeatExecutor = new ScheduledThreadPoolExecutor(0);
+    heartbeatExecutor.setRemoveOnCancelPolicy(true);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();
+        msc.heartbeat(txnId, lockId);
+        clientPool.returnObject(msc);
+      } catch (NoSuchElementException nsee) {
+        LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", nsee);
+      } catch (Exception e) {
+        LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        try {
+          clientPool.invalidateObject(msc);
+        } catch (Exception ex) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      } finally {
+        synchronized (lock) {
+          running = false;
+          lock.notifyAll();
+        }
+      }
+    }
+
+    public void waitUntilFinish(long timeout) throws InterruptedException {
+      synchronized (lock) {
+        if (running) {

Review Comment:
   if doesn't guard against spurious wakeups



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on PR #3303:
URL: https://github.com/apache/hive/pull/3303#issuecomment-1168345079

   > > I think we should do something like this:
   > > ```
   > > ExecutorService executorService = CompactorUtil.createExecutorWithThreadFactory(numberOfWorkers, "CompactionHeartbeat-executor-thread-%d");
   > >     
   > > Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(() -> executorService.execute(heartbeater));
   > > tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
   > > ```
   > 
   > What is the benefit of this construct?
   
   none, I was playing with it trying to extract CompletableFuture, so we can just call 'join' or 'get' instead on manual hacks in runnable


-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907449407


##########
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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);

Review Comment:
   Nice catch!



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907674601


##########
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.Future;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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 destroyed!");
+    }
+    return instance;
+  }
+
+  private final ScheduledThreadPoolExecutor 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 was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish(initialDelay);
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    heartbeatExecutor = new ScheduledThreadPoolExecutor(0);
+    heartbeatExecutor.setRemoveOnCancelPolicy(true);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();
+        msc.heartbeat(txnId, lockId);
+        clientPool.returnObject(msc);
+      } catch (NoSuchElementException nsee) {
+        LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", nsee);
+      } catch (Exception e) {
+        LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        try {
+          clientPool.invalidateObject(msc);
+        } catch (Exception ex) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      } finally {
+        synchronized (lock) {
+          running = false;
+          lock.notifyAll();
+        }
+      }
+    }
+
+    public void waitUntilFinish(long timeout) throws InterruptedException {
+      synchronized (lock) {
+        if (running) {

Review Comment:
   If it's while it makes no sense to have timeout, we can possibly wait forever. I think we should not block application shutdown at this point, if the timeout is not enough for the heartbeater thread to finish, we interrupt should it anyway.



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907674601


##########
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.Future;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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 destroyed!");
+    }
+    return instance;
+  }
+
+  private final ScheduledThreadPoolExecutor 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 was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish(initialDelay);
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    heartbeatExecutor = new ScheduledThreadPoolExecutor(0);
+    heartbeatExecutor.setRemoveOnCancelPolicy(true);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();
+        msc.heartbeat(txnId, lockId);
+        clientPool.returnObject(msc);
+      } catch (NoSuchElementException nsee) {
+        LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", nsee);
+      } catch (Exception e) {
+        LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        try {
+          clientPool.invalidateObject(msc);
+        } catch (Exception ex) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      } finally {
+        synchronized (lock) {
+          running = false;
+          lock.notifyAll();
+        }
+      }
+    }
+
+    public void waitUntilFinish(long timeout) throws InterruptedException {
+      synchronized (lock) {
+        if (running) {

Review Comment:
   If it's while it makes no sense to have timeout, we can possibly wait forever. I think we should not block application shutdown at this point, if the timeout is not enough for the heartbeater thread to finish, we should interrupt  it anyway.



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908577413


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+          clientPool.returnObject(msc);
+        } catch (NoSuchElementException e) {
+          LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", e);

Review Comment:
   is it ok, what happens next?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908595888


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();

Review Comment:
   what if one of the task.stop() throws InterruptedException what happens with the rest?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on PR #3303:
URL: https://github.com/apache/hive/pull/3303#issuecomment-1167366223

   getLockId() method is not used anymore


-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887872810


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(5);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private static final class IMetaStoreClientFactory extends BasePooledObjectFactory<IMetaStoreClient> {
+
+    private final HiveConf conf;
+
+    @Override
+    public IMetaStoreClient create() throws Exception {
+      return HiveMetaStoreUtils.getHiveMetastoreClient(conf);
+    }
+
+    @Override
+    public PooledObject<IMetaStoreClient> wrap(IMetaStoreClient iMetaStoreClient) {
+      return new DefaultPooledObject<>(iMetaStoreClient);
+    }
+
+    @Override
+    public void destroyObject(PooledObject<IMetaStoreClient> p) {
+      p.getObject().close();
+    }
+
+    @Override
+    public boolean validateObject(PooledObject<IMetaStoreClient> p) {
+      //Not in use currently, would be good to validate the client at borrowing/returning, but this needs support from
+      //MetaStoreClient side
+      return super.validateObject(p);
+    }
+
+    public IMetaStoreClientFactory(HiveConf conf) {
+      this.conf = Objects.requireNonNull(conf);
+    }
+
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();

Review Comment:
   what happens if the pool is exausted?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908566560


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);

Review Comment:
   it's false by default, why do we need to set it explicitly?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r909292463


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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(() -> instance.shutdown(), SHUTDOWN_HOOK_PRIORITY);
+        }
+      }
+    }
+    if (instance.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   */
+  void shutdown() {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setMaxWaitMillis(2000);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    private final Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+    private CompactionHeartbeater(long txnId, long lockId, String tableName) {
+      heartbeatExecutor = new ScheduledThreadPoolExecutor(1);
+      this.tableName = Objects.requireNonNull(tableName);
+      this.txnId = txnId;
+      this.lockId = lockId;
+    }
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+        } catch (NoSuchElementException e) {
+          LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", e);
+          // This heartbeat attempt failed, and there is no client to return to the pool.
+          return;
+        } catch (TException e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+          // Heartbeat failed, but the client is not broken, we can return it to the pool.
+        } catch (Exception e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+          // Unknown error, invalidate the client, maybe it is broken.
+          if (msc != null) {
+            try {
+              clientPool.invalidateObject(msc);
+            } catch (Exception ex) {
+              LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+            }
+          }
+          return;
+        }
+        try {
+          if (msc != null) {
+            clientPool.returnObject(msc);
+          }
+        } catch (Exception e) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      }, initialDelay, period, TimeUnit.MILLISECONDS);
+    }
+
+    public void stop() {
+      LOG.info("Shutting down compaction txn heartbeater instance.");
+      heartbeatExecutor.shutdownNow();
+      try {
+        if (!heartbeatExecutor.awaitTermination(5, TimeUnit.SECONDS)) {
+          LOG.warn("Heartbeating for transaction {} did not stop in 5 seconds, do not wait any longer.", this);
+          return;
+        }
+      } catch (InterruptedException ex) {
+        //Caller thread was interrupted while waiting for heartbeater to terminate.
+        //Nothing to do, just restore the interrupted state.
+        Thread.currentThread().interrupt();

Review Comment:
   how would it behave in case shutdownhook is invoked? would it interrupt the shutdownhook thread?
   should we catch and restore Interrupted status only in stopHeartbeat and ignore in shutdown?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
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



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r907429293


##########
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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);

Review Comment:
   Cancel does not remove the task from a scheduler. All it ensures is that isDone() method always return true. This may lead to memory leaks if you keep adding such tasks. You might end up with big pool of threads that cannot be garbage collected as scheduler still has a reference.
   
   You may want to use setRemoveOnCancelPolicy(true) in ScheduledThreadPoolExecutor class , default is set to false.



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ merged pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ merged PR #3303:
URL: https://github.com/apache/hive/pull/3303


-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908595888


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();

Review Comment:
   what if one of the task.stop() throws InterruptedException what happens with the rest?
   msc pool for sure won't be released



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908552785


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private final class CompactionHeartbeater {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final ScheduledThreadPoolExecutor heartbeatExecutor;
+
+
+    void start() {
+      heartbeatExecutor.scheduleAtFixedRate(() -> {
+        IMetaStoreClient msc = null;
+        try {
+          LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+          // Create a metastore client for each thread since it is not thread safe
+          msc = clientPool.borrowObject();
+          msc.heartbeat(txnId, lockId);
+          clientPool.returnObject(msc);
+        } catch (NoSuchElementException e) {
+          LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", e);
+        } catch (TException e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        } catch (Exception e) {
+          LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+          if (msc != null) {
+            try {
+              clientPool.invalidateObject(msc);
+            } catch (Exception ex) {
+              LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+            }
+          }
+        }
+      }, initialDelay, period, TimeUnit.MILLISECONDS);
+    }
+
+    public void stop() throws InterruptedException {
+      LOG.info("Shutting down compaction txn heartbeater instance.");
+      heartbeatExecutor.shutdownNow();

Review Comment:
   please add the null check:
   ````
   if (heartbeatExecutor != null) {
   ...
   }
   ````



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r908591572


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+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 destroyed.
+   */
+  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.shuttingDown) {
+      throw new IllegalStateException("The CompactionHeartbeatService is already destroyed!");
+    }
+    return instance;
+  }
+
+  private final ObjectPool<IMetaStoreClient> clientPool;
+  private volatile boolean shuttingDown = false;
+  private final long initialDelay;
+  private final long period;
+  private final ConcurrentHashMap<Long, CompactionHeartbeater> tasks = new ConcurrentHashMap<>(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 (shuttingDown) {
+      throw new IllegalStateException("Service is shutting down, starting new heartbeats are not possible!");
+    }
+    if (tasks.containsKey(txnId)) {
+      throw new IllegalStateException("Heartbeat was already started for TXN " + txnId);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    heartbeater.start();
+    tasks.put(txnId, heartbeater);
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) throws InterruptedException {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = tasks.get(txnId);
+    if (heartbeater == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    try {
+      heartbeater.stop();
+    } finally {
+      tasks.remove(txnId);
+    }
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    shuttingDown = true;
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    for (CompactionHeartbeater heartbeater : tasks.values()) {
+      heartbeater.stop();
+    }
+    tasks.clear();
+    clientPool.close();
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    int numberOfWorkers = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(numberOfWorkers);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);

Review Comment:
   It could be, but this is applies to waiting for an available object during borrowing it from the pool. If we exceed a NoSuchElementException will thrown, and only that single heartbeat will fail, no effect on the next one.



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887875032


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(5);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private static final class IMetaStoreClientFactory extends BasePooledObjectFactory<IMetaStoreClient> {
+
+    private final HiveConf conf;
+
+    @Override
+    public IMetaStoreClient create() throws Exception {
+      return HiveMetaStoreUtils.getHiveMetastoreClient(conf);
+    }
+
+    @Override
+    public PooledObject<IMetaStoreClient> wrap(IMetaStoreClient iMetaStoreClient) {
+      return new DefaultPooledObject<>(iMetaStoreClient);
+    }
+
+    @Override
+    public void destroyObject(PooledObject<IMetaStoreClient> p) {
+      p.getObject().close();
+    }
+
+    @Override
+    public boolean validateObject(PooledObject<IMetaStoreClient> p) {
+      //Not in use currently, would be good to validate the client at borrowing/returning, but this needs support from
+      //MetaStoreClient side
+      return super.validateObject(p);
+    }
+
+    public IMetaStoreClientFactory(HiveConf conf) {
+      this.conf = Objects.requireNonNull(conf);
+    }
+
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();
+        msc.heartbeat(txnId, lockId);
+        clientPool.returnObject(msc);
+      } catch (NoSuchElementException nsee) {
+        LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", nsee);
+      } catch (Exception e) {
+        LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        try {
+          clientPool.invalidateObject(msc);
+        } catch (Exception ex) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      } finally {
+        synchronized (lock) {
+          running = false;
+          lock.notifyAll();
+        }
+      }
+    }
+
+    public void waitUntilFinish() throws InterruptedException {
+      synchronized (lock) {
+        if (running) {
+          lock.wait();
+        }
+      }
+    }
+
+    private CompactionHeartbeater(long txnId, long lockId, String tableName) {
+      this.tableName = Objects.requireNonNull(tableName);
+      this.txnId = txnId;
+      this.lockId = lockId;
+    }
+
+  }
+
+  private static final class TaskWrapper {

Review Comment:
   It ties CompactionHeartbeater and the returned future together. If you check the `stopHeartbeat()` method, the future is required for cancelling the task in the executor, and the heartbeater instance is required to call the `waitUntilFinish()` method, which will wait for finishing the hearbeat if that's currently executing. This is required to avoid heartbeats after txn abort or commit.



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887874363


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(5);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);

Review Comment:
   is it thread-safe, say 2 threads call borrow and just 1 object is available?



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887870652


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();

Review Comment:
   It can be handled in `CompactionTxn.close()` method as well. I thought it is more straightforward to handle it  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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #3303: HIVE-26242: Compaction heartbeater improvements

Posted by GitBox <gi...@apache.org>.
veghlaci05 commented on code in PR #3303:
URL: https://github.com/apache/hive/pull/3303#discussion_r887879180


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.BasePooledObjectFactory;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+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.HiveMetaStoreUtils;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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);
+          Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+              INSTANCE.shutdown();
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+            }
+          }));
+        }
+      }
+    }
+    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 ConcurrentHashMap<Long, TaskWrapper> tasks = new ConcurrentHashMap<>(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);
+    }
+    LOG.info("Submitting heartbeat task for TXN {}", txnId);
+    CompactionHeartbeater heartbeater = new CompactionHeartbeater(txnId, lockId, tableName);
+    Future<?> submittedTask = heartbeatExecutor.scheduleAtFixedRate(heartbeater, initialDelay, period, TimeUnit.MILLISECONDS);
+    tasks.put(txnId, new TaskWrapper(heartbeater, submittedTask));
+  }
+
+  /**
+   * Stops the heartbeat for the given transaction
+   * @param txnId The id of the compaction txn
+   * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the
+   * given txnId.
+   */
+  void stopHeartbeat(long txnId) {
+    LOG.info("Stopping heartbeat task for TXN {}", txnId);
+    TaskWrapper wrapper = tasks.get(txnId);
+    if (wrapper == null) {
+      throw new IllegalStateException("No registered heartbeat found for TXN " + txnId);
+    }
+    wrapper.future.cancel(false);
+    try {
+      wrapper.heartbeater.waitUntilFinish();
+    } catch (InterruptedException e) {
+      //Restore interrupted state, but let the compaction txn commit/abort
+      Thread.currentThread().interrupt();
+    }
+    tasks.remove(txnId);
+  }
+
+  /**
+   * Shuts down the service, by closing its underlying resources. Be aware that after shutdown this service is no
+   * longer usable, there is no way to re-initialize it.
+   * @throws InterruptedException
+   */
+  void shutdown() throws InterruptedException {
+    LOG.info("Shutting down compaction txn heartbeater service.");
+    heartbeatExecutor.shutdownNow();
+    try {
+      heartbeatExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } finally {
+      tasks.clear();
+      clientPool.close();
+    }
+    LOG.info("Compaction txn heartbeater service is successfully stopped.");
+  }
+
+  private CompactionHeartbeatService(HiveConf conf) {
+    heartbeatExecutor = Executors.newScheduledThreadPool(1);
+    GenericObjectPoolConfig<IMetaStoreClient> config = new GenericObjectPoolConfig<>();
+    config.setMinIdle(1);
+    config.setMaxIdle(2);
+    config.setMaxTotal(5);
+    config.setBlockWhenExhausted(true);
+    config.setMaxWaitMillis(2000);
+    config.setTestOnBorrow(false);
+    config.setTestOnCreate(false);
+    config.setTestOnReturn(false);
+    config.setTestWhileIdle(false);
+    clientPool = new GenericObjectPool<>(new IMetaStoreClientFactory(conf), config);
+    long txnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    initialDelay = txnTimeout / 4;
+    period = txnTimeout / 2;
+  }
+
+  private static final class IMetaStoreClientFactory extends BasePooledObjectFactory<IMetaStoreClient> {
+
+    private final HiveConf conf;
+
+    @Override
+    public IMetaStoreClient create() throws Exception {
+      return HiveMetaStoreUtils.getHiveMetastoreClient(conf);
+    }
+
+    @Override
+    public PooledObject<IMetaStoreClient> wrap(IMetaStoreClient iMetaStoreClient) {
+      return new DefaultPooledObject<>(iMetaStoreClient);
+    }
+
+    @Override
+    public void destroyObject(PooledObject<IMetaStoreClient> p) {
+      p.getObject().close();
+    }
+
+    @Override
+    public boolean validateObject(PooledObject<IMetaStoreClient> p) {
+      //Not in use currently, would be good to validate the client at borrowing/returning, but this needs support from
+      //MetaStoreClient side
+      return super.validateObject(p);
+    }
+
+    public IMetaStoreClientFactory(HiveConf conf) {
+      this.conf = Objects.requireNonNull(conf);
+    }
+
+  }
+
+  private final class CompactionHeartbeater implements Runnable {
+    final private Logger LOG = LoggerFactory.getLogger(CompactionHeartbeater.class);
+    private final long txnId;
+    private final long lockId;
+    private final String tableName;
+    private final Object lock = new Object();
+    private volatile boolean running = false;
+
+
+    @Override
+    public void run() {
+      IMetaStoreClient msc = null;
+      try {
+        synchronized (lock) {
+          running = true;
+        }
+        LOG.debug("Heartbeating compaction transaction id {} for table: {}", txnId, tableName);
+        // Create a metastore client for each thread since it is not thread safe
+        msc = clientPool.borrowObject();
+        msc.heartbeat(txnId, lockId);
+        clientPool.returnObject(msc);
+      } catch (NoSuchElementException nsee) {
+        LOG.error("Compaction transaction heartbeater pool exhausted, unable to heartbeat", nsee);
+      } catch (Exception e) {
+        LOG.error("Error while heartbeating compaction transaction id {} for table: {}", txnId, tableName, e);
+        try {
+          clientPool.invalidateObject(msc);
+        } catch (Exception ex) {
+          LOG.error("Error while invalidating a broken MetaStoreClient instance", e);
+        }
+      } finally {
+        synchronized (lock) {
+          running = false;
+          lock.notifyAll();
+        }
+      }
+    }
+
+    public void waitUntilFinish() throws InterruptedException {

Review Comment:
   The executor service hides the executing thread, you have no direct reference for that. You only have the runnable what you pass in, and the Future you get. Therefore it is not possible to join on it.



-- 
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-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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