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

HDDS-11900. obs granular lock for existing request #7647

Draft
wants to merge 2 commits into
base: master
Choose a base branch
from
Draft
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
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,7 @@
import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult;
import org.apache.hadoop.ozone.om.helpers.SnapshotDiffJob;
import org.apache.hadoop.ozone.om.lock.OMLockDetails;
import org.apache.hadoop.ozone.om.lock.OmLockOpr;
import org.apache.hadoop.ozone.om.ratis_snapshot.OmRatisSnapshotProvider;
import org.apache.hadoop.ozone.om.ha.OMHAMetrics;
import org.apache.hadoop.ozone.om.helpers.KeyInfoWithVolumeContext;
Expand Down Expand Up @@ -472,6 +473,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
private final OzoneLockProvider ozoneLockProvider;
private final OMPerformanceMetrics perfMetrics;
private final BucketUtilizationMetrics bucketUtilizationMetrics;
private final OmLockOpr omLockOpr;

private boolean fsSnapshotEnabled;

Expand Down Expand Up @@ -625,6 +627,7 @@ private OzoneManager(OzoneConfiguration conf, StartupOption startupOption)
configuration);
this.ozoneLockProvider = new OzoneLockProvider(getKeyPathLockEnabled(),
getEnableFileSystemPaths());
this.omLockOpr = new OmLockOpr(threadPrefix);

// For testing purpose only, not hit scm from om as Hadoop UGI can't login
// two principals in the same JVM.
Expand Down Expand Up @@ -1752,6 +1755,8 @@ public void start() throws IOException {
bootstrap(omNodeDetails);
}

omLockOpr.start();

omState = State.RUNNING;
auditMap.put("NewOmState", omState.name());
SYSTEMAUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMSystemAction.STARTUP, auditMap));
Expand Down Expand Up @@ -1829,6 +1834,7 @@ public void restart() throws IOException {
}
startJVMPauseMonitor();
setStartTime();
omLockOpr.start();
omState = State.RUNNING;
auditMap.put("NewOmState", omState.name());
SYSTEMAUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMSystemAction.STARTUP, auditMap));
Expand Down Expand Up @@ -2294,6 +2300,7 @@ public boolean stop() {
}
keyManager.stop();
stopSecretManager();
omLockOpr.stop();

if (scmTopologyClient != null) {
scmTopologyClient.stop();
Expand Down Expand Up @@ -4529,6 +4536,10 @@ public OzoneLockProvider getOzoneLockProvider() {
return this.ozoneLockProvider;
}

public OmLockOpr getOmLockOpr() {
return this.omLockOpr;
}

public ReplicationConfig getDefaultReplicationConfig() {
if (this.defaultReplicationConfig != null) {
return this.defaultReplicationConfig;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,145 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.ozone.om.lock;

import com.google.common.util.concurrent.Striped;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* key locking.
*/
public class KeyLock {
private static final Logger LOG = LoggerFactory.getLogger(KeyLock.class);
private static final long LOCK_TIMEOUT_DEFAULT = 10 * 60 * 1000;
private final Striped<ReadWriteLock> fileStripedLock;
private final long lockTimeout;

public KeyLock(int stripLockSize) {
this(stripLockSize, LOCK_TIMEOUT_DEFAULT);
}

public KeyLock(int stripLockSize, long timeout) {
fileStripedLock = Striped.readWriteLock(stripLockSize);
lockTimeout = timeout;
}

public List<Lock> lock(List<String> keyList) throws IOException {
List<Lock> locks = new ArrayList<>();
boolean isSuccess = false;
try {
Iterable<ReadWriteLock> readWriteLocks = fileStripedLock.bulkGet(keyList);
for (ReadWriteLock rwLock : readWriteLocks) {
Lock lockObj = rwLock.writeLock();
boolean b = lockObj.tryLock(lockTimeout, TimeUnit.MILLISECONDS);
if (!b) {
LOG.error("Key write lock is failed for {} after wait of {}ms", this, lockTimeout);
throw new OMException("Unable to get write lock after " + lockTimeout + "ms"
+ ", read lock info: " + rwLock.readLock(),
OMException.ResultCodes.TIMEOUT);
}
locks.add(lockObj);
}
isSuccess = true;
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new OMException("Unable to get write lock as interrupted", OMException.ResultCodes.INTERNAL_ERROR);
} finally {
if (!isSuccess) {
Collections.reverse(locks);
locks.forEach(Lock::unlock);
locks.clear();
}
}
return locks;
}

public Lock lock(String key) throws IOException {
LOG.debug("Key {} is locked for instance {} {}", key, this, fileStripedLock.get(key));
try {
Lock lockObj = fileStripedLock.get(key).writeLock();
boolean b = lockObj.tryLock(lockTimeout, TimeUnit.MILLISECONDS);
if (!b) {
LOG.error("Key {} lock is failed for {} after wait of {}ms", key, this, lockTimeout);
throw new OMException("Unable to get write lock for " + key + " after " + lockTimeout + "ms"
+ ", read lock info: " + fileStripedLock.get(key).readLock(),
OMException.ResultCodes.TIMEOUT);
}
return lockObj;
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new OMException("Unable to get read lock for " + key + " is interrupted",
OMException.ResultCodes.INTERNAL_ERROR);
}
}

public List<Lock> readLock(List<String> keyList) throws OMException {
List<Lock> locks = new ArrayList<>();
boolean isSuccess = false;
try {
Iterable<ReadWriteLock> readWriteLocks = fileStripedLock.bulkGet(keyList);
for (ReadWriteLock rwLock : readWriteLocks) {
Lock lockObj = rwLock.readLock();
boolean b = lockObj.tryLock(lockTimeout, TimeUnit.MILLISECONDS);
if (!b) {
LOG.error("Key read lock is failed for {} after wait of {}ms", this, lockTimeout);
throw new OMException("Unable to get read lock after " + lockTimeout + "ms"
+ ", write lock info: " + rwLock.writeLock(),
OMException.ResultCodes.TIMEOUT);
}
locks.add(lockObj);
}
isSuccess = true;
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new OMException("Unable to get read lock as interrupted", OMException.ResultCodes.INTERNAL_ERROR);
} finally {
if (!isSuccess) {
Collections.reverse(locks);
locks.forEach(Lock::unlock);
locks.clear();
}
}
return locks;
}

public Lock readLock(String key) throws OMException {
try {
LOG.debug("Key {} is read locked for instance {} {}", key, this, fileStripedLock.get(key));
Lock lockObj = fileStripedLock.get(key).readLock();
boolean b = lockObj.tryLock(lockTimeout, TimeUnit.MILLISECONDS);
if (!b) {
throw new OMException("Unable to get read lock for " + key + " after " + lockTimeout + "ms",
OMException.ResultCodes.TIMEOUT);
}
return lockObj;
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new OMException("Unable to get read lock for " + key + " is interrupted",
OMException.ResultCodes.INTERNAL_ERROR);
}
}
}
Loading
Loading