Skip to content

Commit ec91917

Browse files
author
meiyi
committed
HBASE-21995 Add a coprocessor to set HDFS ACL for hbase granted user
1 parent 428afa9 commit ec91917

File tree

8 files changed

+1984
-11
lines changed

8 files changed

+1984
-11
lines changed

hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -750,6 +750,17 @@ default void postSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ct
750750
final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
751751
throws IOException {}
752752

753+
/**
754+
* Called after the snapshot operation has been completed.
755+
* @param ctx the environment to interact with the framework and master
756+
* @param snapshot the SnapshotDescriptor for the snapshot
757+
* @param tableDescriptor the TableDescriptor of the table to snapshot
758+
*/
759+
default void postCompletedSnapshotAction(final ObserverContext<MasterCoprocessorEnvironment> ctx,
760+
final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
761+
throws IOException {
762+
}
763+
753764
/**
754765
* Called before listSnapshots request has been processed.
755766
* @param ctx the environment to interact with the framework and master

hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java

Lines changed: 29 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -26,9 +26,6 @@
2626
import java.io.FileNotFoundException;
2727
import java.util.List;
2828

29-
import org.apache.yetus.audience.InterfaceAudience;
30-
import org.slf4j.Logger;
31-
import org.slf4j.LoggerFactory;
3229
import org.apache.hadoop.fs.CanSetDropBehind;
3330
import org.apache.hadoop.fs.CanSetReadahead;
3431
import org.apache.hadoop.fs.CanUnbuffer;
@@ -40,6 +37,10 @@
4037
import org.apache.hadoop.fs.Seekable;
4138
import org.apache.hadoop.hbase.util.FSUtils;
4239
import org.apache.hadoop.ipc.RemoteException;
40+
import org.apache.hadoop.security.AccessControlException;
41+
import org.apache.yetus.audience.InterfaceAudience;
42+
import org.slf4j.Logger;
43+
import org.slf4j.LoggerFactory;
4344

4445
/**
4546
* The FileLink is a sort of hardlink, that allows access to a file given a set of locations.
@@ -297,6 +298,7 @@ public void unbuffer() {
297298
* @throws IOException on unexpected error, or file not found.
298299
*/
299300
private FSDataInputStream tryOpen() throws IOException {
301+
AccessControlException accessControlException = null;
300302
for (Path path: fileLink.getLocations()) {
301303
if (path.equals(currentPath)) continue;
302304
try {
@@ -314,12 +316,24 @@ private FSDataInputStream tryOpen() throws IOException {
314316
return(in);
315317
} catch (FileNotFoundException e) {
316318
// Try another file location
319+
} catch (AccessControlException e) {
320+
// Try another file location
321+
accessControlException = e;
317322
} catch (RemoteException re) {
318-
IOException ioe = re.unwrapRemoteException(FileNotFoundException.class);
319-
if (!(ioe instanceof FileNotFoundException)) throw re;
323+
IOException ioe =
324+
re.unwrapRemoteException(FileNotFoundException.class, AccessControlException.class);
325+
if (ioe instanceof AccessControlException) {
326+
accessControlException = (AccessControlException) ioe;
327+
} else if (!(ioe instanceof FileNotFoundException)) {
328+
throw re;
329+
}
320330
}
321331
}
322-
throw new FileNotFoundException("Unable to open link: " + fileLink);
332+
if (accessControlException != null) {
333+
throw accessControlException;
334+
} else {
335+
throw new FileNotFoundException("Unable to open link: " + fileLink);
336+
}
323337
}
324338

325339
@Override
@@ -405,14 +419,22 @@ public Path getAvailablePath(FileSystem fs) throws IOException {
405419
* @throws IOException on unexpected error.
406420
*/
407421
public FileStatus getFileStatus(FileSystem fs) throws IOException {
422+
AccessControlException accessControlException = null;
408423
for (int i = 0; i < locations.length; ++i) {
409424
try {
410425
return fs.getFileStatus(locations[i]);
411426
} catch (FileNotFoundException e) {
412427
// Try another file location
428+
} catch (AccessControlException e) {
429+
// Try another file location
430+
accessControlException = e;
413431
}
414432
}
415-
throw new FileNotFoundException("Unable to open link: " + this);
433+
if (accessControlException != null) {
434+
throw accessControlException;
435+
} else {
436+
throw new FileNotFoundException("Unable to open link: " + this);
437+
}
416438
}
417439

418440
/**

hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1049,6 +1049,16 @@ public void call(MasterObserver observer) throws IOException {
10491049
});
10501050
}
10511051

1052+
public void postCompletedSnapshotAction(final SnapshotDescription snapshot,
1053+
final TableDescriptor hTableDescriptor) throws IOException {
1054+
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
1055+
@Override
1056+
public void call(MasterObserver observer) throws IOException {
1057+
observer.postCompletedSnapshotAction(this, snapshot, hTableDescriptor);
1058+
}
1059+
});
1060+
}
1061+
10521062
public void preListSnapshot(final SnapshotDescription snapshot) throws IOException {
10531063
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
10541064
@Override

hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,7 @@
5959
import org.slf4j.LoggerFactory;
6060

6161
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
62-
62+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
6363
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
6464

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

0 commit comments

Comments
 (0)