Skip to content

Commit

Permalink
HBASE-21995 Add a coprocessor to set HDFS ACL for hbase granted user
Browse files Browse the repository at this point in the history
  • Loading branch information
meiyi committed Apr 18, 2019
1 parent 428afa9 commit ec91917
Show file tree
Hide file tree
Showing 8 changed files with 1,984 additions and 11 deletions.
Expand Up @@ -750,6 +750,17 @@ default void postSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ct
final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
throws IOException {}

/**
* Called after the snapshot operation has been completed.
* @param ctx the environment to interact with the framework and master
* @param snapshot the SnapshotDescriptor for the snapshot
* @param tableDescriptor the TableDescriptor of the table to snapshot
*/
default void postCompletedSnapshotAction(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
throws IOException {
}

/**
* Called before listSnapshots request has been processed.
* @param ctx the environment to interact with the framework and master
Expand Down
Expand Up @@ -26,9 +26,6 @@
import java.io.FileNotFoundException;
import java.util.List;

import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.CanSetDropBehind;
import org.apache.hadoop.fs.CanSetReadahead;
import org.apache.hadoop.fs.CanUnbuffer;
Expand All @@ -40,6 +37,10 @@
import org.apache.hadoop.fs.Seekable;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.AccessControlException;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* The FileLink is a sort of hardlink, that allows access to a file given a set of locations.
Expand Down Expand Up @@ -297,6 +298,7 @@ public void unbuffer() {
* @throws IOException on unexpected error, or file not found.
*/
private FSDataInputStream tryOpen() throws IOException {
AccessControlException accessControlException = null;
for (Path path: fileLink.getLocations()) {
if (path.equals(currentPath)) continue;
try {
Expand All @@ -314,12 +316,24 @@ private FSDataInputStream tryOpen() throws IOException {
return(in);
} catch (FileNotFoundException e) {
// Try another file location
} catch (AccessControlException e) {
// Try another file location
accessControlException = e;
} catch (RemoteException re) {
IOException ioe = re.unwrapRemoteException(FileNotFoundException.class);
if (!(ioe instanceof FileNotFoundException)) throw re;
IOException ioe =
re.unwrapRemoteException(FileNotFoundException.class, AccessControlException.class);
if (ioe instanceof AccessControlException) {
accessControlException = (AccessControlException) ioe;
} else if (!(ioe instanceof FileNotFoundException)) {
throw re;
}
}
}
throw new FileNotFoundException("Unable to open link: " + fileLink);
if (accessControlException != null) {
throw accessControlException;
} else {
throw new FileNotFoundException("Unable to open link: " + fileLink);
}
}

@Override
Expand Down Expand Up @@ -405,14 +419,22 @@ public Path getAvailablePath(FileSystem fs) throws IOException {
* @throws IOException on unexpected error.
*/
public FileStatus getFileStatus(FileSystem fs) throws IOException {
AccessControlException accessControlException = null;
for (int i = 0; i < locations.length; ++i) {
try {
return fs.getFileStatus(locations[i]);
} catch (FileNotFoundException e) {
// Try another file location
} catch (AccessControlException e) {
// Try another file location
accessControlException = e;
}
}
throw new FileNotFoundException("Unable to open link: " + this);
if (accessControlException != null) {
throw accessControlException;
} else {
throw new FileNotFoundException("Unable to open link: " + this);
}
}

/**
Expand Down
Expand Up @@ -1049,6 +1049,16 @@ public void call(MasterObserver observer) throws IOException {
});
}

public void postCompletedSnapshotAction(final SnapshotDescription snapshot,
final TableDescriptor hTableDescriptor) throws IOException {
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
@Override
public void call(MasterObserver observer) throws IOException {
observer.postCompletedSnapshotAction(this, snapshot, hTableDescriptor);
}
});
}

public void preListSnapshot(final SnapshotDescription snapshot) throws IOException {
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
@Override
Expand Down
Expand Up @@ -59,7 +59,7 @@
import org.slf4j.LoggerFactory;

import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;

import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;

/**
Expand Down Expand Up @@ -228,6 +228,10 @@ public void process() {
status.markComplete(msg);
LOG.info(msg);
metricsSnapshot.addSnapshot(status.getCompletionTimestamp() - status.getStartTime());
if (master.getMasterCoprocessorHost() != null) {
master.getMasterCoprocessorHost()
.postCompletedSnapshotAction(ProtobufUtil.createSnapshotDesc(snapshot), this.htd);
}
} catch (Exception e) { // FindBugs: REC_CATCH_EXCEPTION
status.abort("Failed to complete snapshot " + snapshot.getName() + " on table " +
snapshotTable + " because " + e.getMessage());
Expand Down

0 comments on commit ec91917

Please sign in to comment.