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

Use new LeaseRecoverable interface added in HADOOP-18671 #5241

Open
wants to merge 3 commits into
base: main
Choose a base branch
from
Open
Changes from 1 commit
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 @@ -25,12 +25,11 @@
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.server.fs.VolumeManager;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonPathCapabilities;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.LeaseRecoverable;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RawLocalFileSystem;
import org.apache.hadoop.fs.viewfs.ViewFileSystem;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -41,6 +40,7 @@ public class HadoopLogCloser implements LogCloser {
@Override
public long close(AccumuloConfiguration conf, Configuration hadoopConf, VolumeManager fs,
Path source) throws IOException {

FileSystem ns = fs.getFileSystemByPath(source);

// if path points to a viewfs path, then resolve to underlying filesystem
Expand All @@ -52,10 +52,10 @@ public long close(AccumuloConfiguration conf, Configuration hadoopConf, VolumeMa
}
}

if (ns instanceof DistributedFileSystem) {
DistributedFileSystem dfs = (DistributedFileSystem) ns;
if (ns.hasPathCapability(source, CommonPathCapabilities.LEASE_RECOVERABLE)) {
try {
if (!dfs.recoverLease(source)) {
LeaseRecoverable lr = (LeaseRecoverable) ns;
if (!lr.recoverLease(source)) {
log.info("Waiting for file to be closed {}", source);
return conf.getTimeInMillis(Property.MANAGER_LEASE_RECOVERY_WAITING_PERIOD);
}
Expand All @@ -67,8 +67,6 @@ public long close(AccumuloConfiguration conf, Configuration hadoopConf, VolumeMa
ns.append(source).close();
log.info("Recovered lease on {} using append", source);
}
} else if (ns instanceof LocalFileSystem || ns instanceof RawLocalFileSystem) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Do you happen to know if the local filessystem impls will have the CommonPathCapabilities.LEASE_RECOVERABLE capability?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It looks like only DistributedFileSystem and ViewDistributedFileSystem implement the LeaseRecoverable interface.

// ignore
} else {
throw new IllegalStateException(
"Don't know how to recover a lease for " + ns.getClass().getName());
Expand Down
Loading