-
Notifications
You must be signed in to change notification settings - Fork 4.7k
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
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||
---|---|---|---|---|
|
@@ -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; | ||||
|
@@ -56,6 +57,7 @@ public class Initiator extends MetaStoreCompactorThread { | |||
private ExecutorService compactionExecutor; | ||||
|
||||
private boolean metricsEnabled; | ||||
private boolean shouldUseMutex = true; | ||||
|
||||
@Override | ||||
public void run() { | ||||
|
@@ -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 | ||||
|
@@ -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())) { | ||||
startedAt = System.currentTimeMillis(); | ||||
prevStart = handle.getLastUpdateTime(); | ||||
|
||||
|
@@ -174,16 +174,13 @@ public void run() { | |||
|
||||
// Check for timed out remote workers. | ||||
recoverFailedCompactions(true); | ||||
handle.releaseLocks(startedAt); | ||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @Indhumathi27, Sorry for the late reply! There was a problem hiding this comment. Choose a reason for hiding this commentThe 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
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If exception happens before updating the current There was a problem hiding this comment. Choose a reason for hiding this commentThe 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); | ||||
|
@@ -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 { | ||||
|
@@ -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 |
---|---|---|
@@ -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 | ||
} | ||
} | ||
|
||
} |
There was a problem hiding this comment.
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 ?
There was a problem hiding this comment.
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()
andreleaseLocks(Long timestamp)
ofTxnStore.MutexAPI.LockHandle
in this class, so not use theAutoCloseable
.