Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HIVE-28669: Deadlock found when TxnStoreMutex trying to acquireLock #5585

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
15 changes: 9 additions & 6 deletions ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.hadoop.hive.metastore.ReplChangeManager;
import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
import org.apache.hadoop.hive.metastore.txn.TxnDummyMutex;
import org.apache.hadoop.hive.metastore.txn.TxnStore;
import org.apache.hadoop.hive.ql.txn.compactor.handler.TaskHandler;
import org.apache.hadoop.hive.ql.txn.compactor.handler.TaskHandlerFactory;
Expand All @@ -46,6 +47,7 @@ public class Cleaner extends MetaStoreCompactorThread {
static final private String CLASS_NAME = Cleaner.class.getName();
static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
private boolean metricsEnabled = false;
private boolean shouldUseMutex = true;

private ExecutorService cleanerExecutor;
private List<TaskHandler> cleanupHandlers;
Expand All @@ -70,14 +72,13 @@ public void run() {
LOG.info("Starting Cleaner thread");
try {
do {
TxnStore.MutexAPI.LockHandle handle = null;
TxnStore.MutexAPI mutex = shouldUseMutex ? txnHandler.getMutexAPI() : new TxnDummyMutex();
metadataCache.invalidate();
long startedAt = -1;

// Make sure nothing escapes this run method and kills the metastore at large,
// so wrap it in a big catch Throwable statement.
try {
handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.Cleaner.name());
try (AutoCloseable closeable = mutex.acquireLock(TxnStore.MUTEX_KEY.Cleaner.name())) {
startedAt = System.currentTimeMillis();

if (metricsEnabled) {
Expand Down Expand Up @@ -120,9 +121,6 @@ public void run() {
LOG.error("Caught an exception in the main loop of compactor cleaner, {}",
StringUtils.stringifyException(t));
} finally {
if (handle != null) {
handle.releaseLocks();
}
if (metricsEnabled) {
updateCycleDurationMetric(MetricsConstants.COMPACTION_CLEANER_CYCLE_DURATION, startedAt);
}
Expand Down Expand Up @@ -170,4 +168,9 @@ public void run() {
updateCycleDurationMetric(metric, startedAt);
}
}

@Override
public void shouldUseMutex(boolean enableMutex) {
this.shouldUseMutex = enableMutex;
}
}
20 changes: 10 additions & 10 deletions ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
import org.apache.hadoop.hive.metastore.txn.TxnDummyMutex;
import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
import org.apache.hadoop.hive.metastore.txn.TxnStore;
Expand Down Expand Up @@ -56,6 +57,7 @@ public class Initiator extends MetaStoreCompactorThread {
private ExecutorService compactionExecutor;

private boolean metricsEnabled;
private boolean shouldUseMutex = true;

@Override
public void run() {
Expand All @@ -70,6 +72,7 @@ public void run() {
long abortedTimeThreshold = HiveConf
.getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
TimeUnit.MILLISECONDS);
TxnStore.MutexAPI mutex = shouldUseMutex ? txnHandler.getMutexAPI() : new TxnDummyMutex();

// Make sure we run through the loop once before checking to stop as this makes testing
// much easier. The stop value is only for testing anyway and not used when called from
Expand All @@ -78,13 +81,10 @@ public void run() {
PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
long startedAt = -1;
long prevStart;
TxnStore.MutexAPI.LockHandle handle = null;
boolean exceptionally = false;

// Wrap the inner parts of the loop in a catch throwable so that any errors in the loop
// don't doom the entire thread.
try {
handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.Initiator.name());
try (TxnStore.MutexAPI.LockHandle handle = mutex.acquireLock(TxnStore.MUTEX_KEY.Initiator.name())) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

looks like, we have to use AutoCloseable here as well ?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We need to call getLastUpdateTime() and releaseLocks(Long timestamp) of TxnStore.MutexAPI.LockHandle in this class, so not use the AutoCloseable.

startedAt = System.currentTimeMillis();
prevStart = handle.getLastUpdateTime();

Expand Down Expand Up @@ -174,16 +174,13 @@ public void run() {

// Check for timed out remote workers.
recoverFailedCompactions(true);
handle.releaseLocks(startedAt);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Releasing the locks will not happen if any code runs into a exception before this line, since it is inside try block

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@Indhumathi27, Sorry for the late reply!
handle.releaseLocks(startedAt); this is for marking the starting point of previous run, if there is no exception, the Initiator will use the startedAt as the start to find the potential compactions:
https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java#L89
https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java#L117

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@dengzhhu653 Thanks for the reply.

Just a question over here. I understand, when there is no exception, we call handle.releaseLocks(startedAt); to update AUX table with startedAt, which will be the starting point of previous run. But in case, a unexpected exception occurred (say at line

), then how it will be handled ?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If exception happens before updating the current startedAt, this AUX table with startedAt will not get updated, next run the Initiator will use the former one(startAt) or -1 if there is no such record to search through db to find the potential compactions, and it will skip the queued or running candidates.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

got it.. thanks

} catch (InterruptedException e) {
// do not ignore interruption requests
return;
} catch (Throwable t) {
LOG.error("Initiator loop caught unexpected exception this time through the loop", t);
exceptionally = true;
} finally {
if (handle != null) {
if (!exceptionally) handle.releaseLocks(startedAt); else handle.releaseLocks();
}
if (metricsEnabled) {
perfLogger.perfLogEnd(CLASS_NAME, MetricsConstants.COMPACTION_INITIATOR_CYCLE);
updateCycleDurationMetric(MetricsConstants.COMPACTION_INITIATOR_CYCLE_DURATION, startedAt);
Expand Down Expand Up @@ -215,8 +212,6 @@ private Database resolveDatabase(CompactionInfo ci) throws MetaException, NoSuch
return CompactorUtil.resolveDatabase(conf, ci.dbname);
}



@VisibleForTesting
protected String resolveUserToRunAs(Map<String, String> cache, Table t, Partition p)
throws IOException, InterruptedException {
Expand Down Expand Up @@ -428,4 +423,9 @@ public void run() {
}
}
}

@Override
public void shouldUseMutex(boolean enableMutex) {
this.shouldUseMutex = enableMutex;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -43,4 +43,12 @@ public interface MetastoreTaskThread extends Configurable, Runnable {
default long initialDelay(TimeUnit unit) {
return runFrequency(unit);
}

/**
* Should use mutex support to allow only one copy of this task running across the warehouse.
* @param enableMutex true for enabling the mutex, false otherwise
*/
default void shouldUseMutex(boolean enableMutex) {
// no-op
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.hadoop.hive.common.ValidTxnList;
import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
import org.apache.hadoop.hive.metastore.txn.TxnDummyMutex;
import org.apache.hadoop.hive.metastore.txn.TxnStore;
import org.apache.hadoop.hive.metastore.txn.TxnUtils;
import org.slf4j.Logger;
Expand All @@ -37,6 +38,7 @@ public class MaterializationsRebuildLockCleanerTask implements MetastoreTaskThre

private Configuration conf;
private TxnStore txnHandler;
private boolean shouldUseMutex = true;

@Override
public long runFrequency(TimeUnit unit) {
Expand All @@ -60,9 +62,8 @@ public void run() {
LOG.debug("Cleaning up materialization rebuild locks");
}

TxnStore.MutexAPI.LockHandle handle = null;
try {
handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.MaterializationRebuild.name());
TxnStore.MutexAPI mutex = shouldUseMutex ? txnHandler.getMutexAPI() : new TxnDummyMutex();
try (AutoCloseable closeable = mutex.acquireLock(TxnStore.MUTEX_KEY.MaterializationRebuild.name())) {
ValidTxnList validTxnList = TxnCommonUtils.createValidReadTxnList(txnHandler.getOpenTxns(), 0);
long removedCnt = txnHandler.cleanupMaterializationRebuildLocks(validTxnList,
MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS));
Expand All @@ -73,10 +74,11 @@ public void run() {
}
} catch (Throwable t) {
LOG.error("Unexpected error in thread: {}, message: {}", Thread.currentThread().getName(), t.getMessage(), t);
} finally {
if (handle != null) {
handle.releaseLocks();
}
}
}

@Override
public void shouldUseMutex(boolean enableMutex) {
this.shouldUseMutex = enableMutex;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,4 +42,12 @@ public interface MetaStoreThread extends Configurable {
* been called.
*/
void start();

/**
* Should use mutex support to allow only one copy of this task running across the warehouse.
* @param enableMutex true for enabling the mutex, false otherwise
*/
default void shouldUseMutex(boolean enableMutex) {
// no-op
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -138,6 +138,7 @@ public void takeLeadership(LeaderElection election) throws Exception {
AtomicBoolean flag = new AtomicBoolean();
thread.setConf(configuration);
thread.init(flag);
thread.shouldUseMutex(election.hasMultipleLeaders());
metastoreThreadsMap.put(thread, flag);
HiveMetaStore.LOG.info("Starting metastore thread of type " + thread.getClass().getName());
thread.start();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,7 @@ public void takeLeadership(LeaderElection election) throws Exception {
List<MetastoreTaskThread> alwaysTasks = new ArrayList<>(getAlwaysTasks());
for (MetastoreTaskThread task : alwaysTasks) {
task.setConf(configuration);
task.shouldUseMutex(election.hasMultipleLeaders());
long freq = task.runFrequency(TimeUnit.MILLISECONDS);
// For backwards compatibility, since some threads used to be hard coded but only run if
// frequency was > 0
Expand All @@ -111,6 +112,7 @@ public void takeLeadership(LeaderElection election) throws Exception {
List<MetastoreTaskThread> remoteOnlyTasks = new ArrayList<>(getRemoteOnlyTasks());
for (MetastoreTaskThread task : remoteOnlyTasks) {
task.setConf(configuration);
task.shouldUseMutex(election.hasMultipleLeaders());
long freq = task.runFrequency(TimeUnit.MILLISECONDS);
runningTasks.add(task);
metastoreTaskThreadPool.getPool().scheduleAtFixedRate(task, freq, freq, TimeUnit.MILLISECONDS);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,11 @@
* @param <T> the type of mutex
*/
public interface LeaderElection<T> extends Closeable {
// We might have different versions of HMS, or even the same version but with different
// leader election methods running inside the warehouse, so it's hard to know how many HMS instances
// that elected as the leader. Relying on this property to tell us, default is true, means it has multiple
// HMS instances acting as the leader.
public static final String HAVE_MULTIPLE_LEADERS = "hive.metastore.have.multiple.leaders";

/**
* Place where election happens
Expand Down Expand Up @@ -65,6 +70,10 @@ public void tryBeLeader(Configuration conf, T mutex)
*/
public String getName();

default boolean hasMultipleLeaders() {
return true;
}

public interface LeadershipStateListener {

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@ public class LeaseLeaderElection implements LeaderElection<TableName> {
private String name;
private String userName;
private String hostName;
private boolean multipleLeaders;

public LeaseLeaderElection() throws IOException {
userName = SecurityUtils.getUser();
Expand Down Expand Up @@ -152,7 +153,7 @@ private void notifyListener() {
public void tryBeLeader(Configuration conf, TableName table) throws LeaderException {
requireNonNull(conf, "conf is null");
requireNonNull(table, "table is null");

this.multipleLeaders = conf.getBoolean(HAVE_MULTIPLE_LEADERS, true);
if (store == null) {
store = TxnUtils.getTxnStore(conf);
}
Expand Down Expand Up @@ -471,4 +472,9 @@ public void setName(String name) {
public String getName() {
return name;
}

@Override
public boolean hasMultipleLeaders() {
return this.multipleLeaders;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,13 +39,14 @@ public class StaticLeaderElection implements LeaderElection<String> {
private volatile boolean isLeader;
private String name;
private List<LeadershipStateListener> listeners = new ArrayList<>();
private boolean multipleLeaders;

@Override
public void tryBeLeader(Configuration conf, String hostName)
throws LeaderException {
String leaderHost = MetastoreConf.getVar(conf,
MetastoreConf.ConfVars.METASTORE_HOUSEKEEPING_LEADER_HOSTNAME);

this.multipleLeaders = conf.getBoolean(HAVE_MULTIPLE_LEADERS, true);
// For the sake of backward compatibility, when the current HMS becomes the leader when no
// leader is specified.
if (leaderHost == null || leaderHost.isEmpty()) {
Expand Down Expand Up @@ -103,4 +104,8 @@ public void close() {
}
}

@Override
public boolean hasMultipleLeaders() {
return this.multipleLeaders;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,7 @@ public void takeLeadership(LeaderElection election) throws Exception {
thread.setConf(configuration);
stop = new AtomicBoolean(false);
thread.init(stop);
thread.shouldUseMutex(election.hasMultipleLeaders());
HiveMetaStore.LOG.info("Starting metastore thread of type " + thread.getClass().getName());
thread.start();
} catch (Exception e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,7 +110,7 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl
TransactionContext context = null;
try {
jdbcResource.bindDataSource(transactional);
context = jdbcResource.getTransactionManager().getNewTransaction(transactional.propagation().value());
context = jdbcResource.getTransactionManager().getNewTransaction(transactional);
Object result = toCall.execute();
LOG.debug("Successfull method invocation within transactional context: {}, going to commit.", callerId);
if (context.isRollbackOnly()) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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.metastore.txn;

import org.apache.hadoop.hive.metastore.api.MetaException;

/**
* An empty implementation of TxnStore.MutexAPI
*/
public class TxnDummyMutex implements TxnStore.MutexAPI {

@Override
public LockHandle acquireLock(String key) throws MetaException {
return new DummyHandle();
}

@Override
public void acquireLock(String key, LockHandle handle) throws MetaException {
// no-op
}

private static class DummyHandle implements LockHandle {

private long lastUpdateTime = 0L;

@Override
public void releaseLocks() {
// no-op
}

@Override
public Long getLastUpdateTime() {
return lastUpdateTime;
}

@Override
public void releaseLocks(Long timestamp) {
this.lastUpdateTime = timestamp;
}

@Override
public void close() {
// no-op
}
}

}
Loading
Loading