|
@@ -33,6 +33,7 @@ import static org.apache.hadoop.util.Time.now;
|
|
|
import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
+import java.security.PrivilegedExceptionAction;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collection;
|
|
|
import java.util.EnumSet;
|
|
@@ -175,6 +176,7 @@ import org.apache.hadoop.ipc.RefreshResponse;
|
|
|
import org.apache.hadoop.net.Node;
|
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.security.Groups;
|
|
|
+import org.apache.hadoop.security.SecurityUtil;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.authorize.AuthorizationException;
|
|
|
import org.apache.hadoop.security.authorize.ProxyUsers;
|
|
@@ -2113,15 +2115,15 @@ public class NameNodeRpcServer implements NamenodeProtocols {
|
|
|
}
|
|
|
|
|
|
@Override // ClientProtocol
|
|
|
- public EventBatchList getEditsFromTxid(long txid) throws IOException {
|
|
|
+ public EventBatchList getEditsFromTxid(final long txid) throws IOException {
|
|
|
checkNNStartup();
|
|
|
namesystem.checkOperation(OperationCategory.READ); // only active
|
|
|
namesystem.checkSuperuserPrivilege();
|
|
|
- int maxEventsPerRPC = nn.getConf().getInt(
|
|
|
+ final int maxEventsPerRPC = nn.getConf().getInt(
|
|
|
DFSConfigKeys.DFS_NAMENODE_INOTIFY_MAX_EVENTS_PER_RPC_KEY,
|
|
|
DFSConfigKeys.DFS_NAMENODE_INOTIFY_MAX_EVENTS_PER_RPC_DEFAULT);
|
|
|
- FSEditLog log = namesystem.getFSImage().getEditLog();
|
|
|
- long syncTxid = log.getSyncTxId();
|
|
|
+ final FSEditLog log = namesystem.getFSImage().getEditLog();
|
|
|
+ final long syncTxid = log.getSyncTxId();
|
|
|
// If we haven't synced anything yet, we can only read finalized
|
|
|
// segments since we can't reliably determine which txns in in-progress
|
|
|
// segments have actually been committed (e.g. written to a quorum of JNs).
|
|
@@ -2130,8 +2132,26 @@ public class NameNodeRpcServer implements NamenodeProtocols {
|
|
|
// journals. (In-progress segments written by old writers are already
|
|
|
// discarded for us, so if we read any in-progress segments they are
|
|
|
// guaranteed to have been written by this NameNode.)
|
|
|
- boolean readInProgress = syncTxid > 0;
|
|
|
+ final boolean readInProgress = syncTxid > 0;
|
|
|
+
|
|
|
+ // doas the NN login user for the actual operations to get edits.
|
|
|
+ // Notably this is necessary when polling from the remote edits via https.
|
|
|
+ // We have validated the client is a superuser from the NN RPC, so this
|
|
|
+ // running as the login user here is safe.
|
|
|
+ EventBatchList ret = SecurityUtil.doAsLoginUser(
|
|
|
+ new PrivilegedExceptionAction<EventBatchList>() {
|
|
|
+ @Override
|
|
|
+ public EventBatchList run() throws IOException {
|
|
|
+ return getEventBatchList(syncTxid, txid, log, readInProgress,
|
|
|
+ maxEventsPerRPC);
|
|
|
+ }
|
|
|
+ });
|
|
|
+ return ret;
|
|
|
+ }
|
|
|
|
|
|
+ private EventBatchList getEventBatchList(long syncTxid, long txid,
|
|
|
+ FSEditLog log, boolean readInProgress, int maxEventsPerRPC)
|
|
|
+ throws IOException {
|
|
|
List<EventBatch> batches = Lists.newArrayList();
|
|
|
int totalEvents = 0;
|
|
|
long maxSeenTxid = -1;
|
|
@@ -2150,7 +2170,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
|
|
|
// and are using QJM -- the edit log will be closed and this exception
|
|
|
// will result
|
|
|
LOG.info("NN is transitioning from active to standby and FSEditLog " +
|
|
|
- "is closed -- could not read edits");
|
|
|
+ "is closed -- could not read edits");
|
|
|
return new EventBatchList(batches, firstSeenTxid, maxSeenTxid, syncTxid);
|
|
|
}
|
|
|
|