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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
