|
@@ -41,8 +41,13 @@ import org.apache.hadoop.service.AbstractService;
|
|
|
import org.apache.hadoop.yarn.api.records.QueueACL;
|
|
|
import org.apache.hadoop.yarn.api.records.ReservationACL;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
+import org.apache.hadoop.yarn.security.AccessType;
|
|
|
+import org.apache.hadoop.yarn.security.Permission;
|
|
|
+import org.apache.hadoop.yarn.security.PrivilegedEntity;
|
|
|
+import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
|
|
import org.apache.hadoop.yarn.util.Clock;
|
|
|
import org.apache.hadoop.yarn.util.SystemClock;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
@@ -74,6 +79,12 @@ public class AllocationFileLoaderService extends AbstractService {
|
|
|
|
|
|
public static final long THREAD_JOIN_TIMEOUT_MS = 1000;
|
|
|
|
|
|
+ private static final String ROOT = "root";
|
|
|
+ private static final AccessControlList EVERYBODY_ACL =
|
|
|
+ new AccessControlList("*");
|
|
|
+ private static final AccessControlList NOBODY_ACL =
|
|
|
+ new AccessControlList(" ");
|
|
|
+
|
|
|
private final Clock clock;
|
|
|
|
|
|
private long lastSuccessfulReload; // Last time we successfully reloaded queues
|
|
@@ -93,7 +104,9 @@ public class AllocationFileLoaderService extends AbstractService {
|
|
|
public AllocationFileLoaderService() {
|
|
|
this(SystemClock.getInstance());
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ private List<Permission> defaultPermissions;
|
|
|
+
|
|
|
public AllocationFileLoaderService(Clock clock) {
|
|
|
super(AllocationFileLoaderService.class.getName());
|
|
|
this.clock = clock;
|
|
@@ -208,6 +221,7 @@ public class AllocationFileLoaderService extends AbstractService {
|
|
|
ParserConfigurationException, SAXException,
|
|
|
AllocationConfigurationException {
|
|
|
if (allocFile == null) {
|
|
|
+ reloadListener.onReload(null);
|
|
|
return;
|
|
|
}
|
|
|
LOG.info("Loading allocation file " + allocFile);
|
|
@@ -224,9 +238,10 @@ public class AllocationFileLoaderService extends AbstractService {
|
|
|
Map<String, Long> minSharePreemptionTimeouts = new HashMap<>();
|
|
|
Map<String, Long> fairSharePreemptionTimeouts = new HashMap<>();
|
|
|
Map<String, Float> fairSharePreemptionThresholds = new HashMap<>();
|
|
|
- Map<String, Map<QueueACL, AccessControlList>> queueAcls = new HashMap<>();
|
|
|
+ Map<String, Map<AccessType, AccessControlList>> queueAcls =
|
|
|
+ new HashMap<>();
|
|
|
Map<String, Map<ReservationACL, AccessControlList>> reservationAcls =
|
|
|
- new HashMap<>();
|
|
|
+ new HashMap<>();
|
|
|
Set<String> reservableQueues = new HashSet<>();
|
|
|
Set<String> nonPreemptableQueues = new HashSet<>();
|
|
|
int userMaxAppsDefault = Integer.MAX_VALUE;
|
|
@@ -444,7 +459,7 @@ public class AllocationFileLoaderService extends AbstractService {
|
|
|
Map<String, Long> minSharePreemptionTimeouts,
|
|
|
Map<String, Long> fairSharePreemptionTimeouts,
|
|
|
Map<String, Float> fairSharePreemptionThresholds,
|
|
|
- Map<String, Map<QueueACL, AccessControlList>> queueAcls,
|
|
|
+ Map<String, Map<AccessType, AccessControlList>> queueAcls,
|
|
|
Map<String, Map<ReservationACL, AccessControlList>> resAcls,
|
|
|
Map<FSQueueType, Set<String>> configuredQueues,
|
|
|
Set<String> reservableQueues,
|
|
@@ -468,7 +483,7 @@ public class AllocationFileLoaderService extends AbstractService {
|
|
|
queueName = parentName + "." + queueName;
|
|
|
}
|
|
|
|
|
|
- Map<QueueACL, AccessControlList> acls = new HashMap<>();
|
|
|
+ Map<AccessType, AccessControlList> acls = new HashMap<>();
|
|
|
Map<ReservationACL, AccessControlList> racls = new HashMap<>();
|
|
|
NodeList fields = element.getChildNodes();
|
|
|
boolean isLeaf = true;
|
|
@@ -526,10 +541,10 @@ public class AllocationFileLoaderService extends AbstractService {
|
|
|
queuePolicies.put(queueName, policy);
|
|
|
} else if ("aclSubmitApps".equals(field.getTagName())) {
|
|
|
String text = ((Text)field.getFirstChild()).getData();
|
|
|
- acls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(text));
|
|
|
+ acls.put(AccessType.SUBMIT_APP, new AccessControlList(text));
|
|
|
} else if ("aclAdministerApps".equals(field.getTagName())) {
|
|
|
String text = ((Text)field.getFirstChild()).getData();
|
|
|
- acls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(text));
|
|
|
+ acls.put(AccessType.ADMINISTER_QUEUE, new AccessControlList(text));
|
|
|
} else if ("aclAdministerReservations".equals(field.getTagName())) {
|
|
|
String text = ((Text)field.getFirstChild()).getData();
|
|
|
racls.put(ReservationACL.ADMINISTER_RESERVATIONS,
|
|
@@ -578,6 +593,17 @@ public class AllocationFileLoaderService extends AbstractService {
|
|
|
}
|
|
|
configuredQueues.get(FSQueueType.PARENT).add(queueName);
|
|
|
}
|
|
|
+ // Set default acls if not defined
|
|
|
+ // The root queue defaults to all access
|
|
|
+ for (QueueACL acl : QueueACL.values()) {
|
|
|
+ AccessType accessType = SchedulerUtils.toAccessType(acl);
|
|
|
+ if (acls.get(accessType) == null){
|
|
|
+ AccessControlList defaultAcl = queueName.equals(ROOT) ?
|
|
|
+ EVERYBODY_ACL : NOBODY_ACL;
|
|
|
+ acls.put(accessType, defaultAcl);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
queueAcls.put(queueName, acls);
|
|
|
resAcls.put(queueName, racls);
|
|
|
if (maxQueueResources.containsKey(queueName) &&
|
|
@@ -590,8 +616,30 @@ public class AllocationFileLoaderService extends AbstractService {
|
|
|
minQueueResources.get(queueName)));
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- public interface Listener {
|
|
|
- public void onReload(AllocationConfiguration info);
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns the list of default permissions.
|
|
|
+ * The default permission for the root queue is everybody ("*")
|
|
|
+ * and the default permission for all other queues is nobody ("").
|
|
|
+ * The default permission list would be loaded before the permissions
|
|
|
+ * from allocation file.
|
|
|
+ * @return default permission list
|
|
|
+ */
|
|
|
+ protected List<Permission> getDefaultPermissions() {
|
|
|
+ if (defaultPermissions == null) {
|
|
|
+ defaultPermissions = new ArrayList<>();
|
|
|
+ Map<AccessType, AccessControlList> acls =
|
|
|
+ new HashMap<>();
|
|
|
+ for (QueueACL acl : QueueACL.values()) {
|
|
|
+ acls.put(SchedulerUtils.toAccessType(acl), EVERYBODY_ACL);
|
|
|
+ }
|
|
|
+ defaultPermissions.add(new Permission(
|
|
|
+ new PrivilegedEntity(EntityType.QUEUE, ROOT), acls));
|
|
|
+ }
|
|
|
+ return defaultPermissions;
|
|
|
+ }
|
|
|
+
|
|
|
+ interface Listener {
|
|
|
+ void onReload(AllocationConfiguration info) throws IOException;
|
|
|
}
|
|
|
}
|