|
@@ -19,11 +19,16 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies;
|
|
|
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
+import static org.junit.Assert.fail;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.Collections;
|
|
|
import java.util.Comparator;
|
|
|
+import java.util.List;
|
|
|
import java.util.Map;
|
|
|
+import java.util.TreeSet;
|
|
|
|
|
|
import org.apache.curator.shaded.com.google.common.base.Joiner;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -443,4 +448,76 @@ public class TestDominantResourceFairnessPolicy {
|
|
|
conf.set(YarnConfiguration.RESOURCE_TYPES, Joiner.on(',').join(resources));
|
|
|
ResourceUtils.resetResourceTypes(conf);
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testModWhileSorting(){
|
|
|
+ final List<FakeSchedulable> schedulableList = new ArrayList<>();
|
|
|
+ for (int i=0; i<10000; i++) {
|
|
|
+ schedulableList.add(
|
|
|
+ (FakeSchedulable)createSchedulable((i%10)*100, (i%3)*2));
|
|
|
+ }
|
|
|
+ Comparator DRFComparator = createComparator(100000, 50000);
|
|
|
+
|
|
|
+ // To simulate unallocated resource changes
|
|
|
+ Thread modThread = modificationThread(schedulableList);
|
|
|
+ modThread.start();
|
|
|
+
|
|
|
+ // This should fail: make sure that we do test correctly
|
|
|
+ // TimSort which is used does not handle the concurrent modification of
|
|
|
+ // objects it is sorting.
|
|
|
+ try {
|
|
|
+ Collections.sort(schedulableList, DRFComparator);
|
|
|
+ fail("Sorting should have failed and did not");
|
|
|
+ } catch (IllegalArgumentException iae) {
|
|
|
+ assertEquals(iae.getMessage(), "Comparison method violates its general contract!");
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ modThread.join();
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ fail("ModThread join failed: " + ie.getMessage());
|
|
|
+ }
|
|
|
+
|
|
|
+ // clean up and try again using TreeSet which should work
|
|
|
+ schedulableList.clear();
|
|
|
+ for (int i=0; i<10000; i++) {
|
|
|
+ schedulableList.add(
|
|
|
+ (FakeSchedulable)createSchedulable((i%10)*100, (i%3)*2));
|
|
|
+ }
|
|
|
+ TreeSet<Schedulable> sortedSchedulable = new TreeSet<>(DRFComparator);
|
|
|
+ modThread = modificationThread(schedulableList);
|
|
|
+ modThread.start();
|
|
|
+ sortedSchedulable.addAll(schedulableList);
|
|
|
+ try {
|
|
|
+ modThread.join();
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ fail("ModThread join failed: " + ie.getMessage());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Thread to simulate concurrent schedulable changes while sorting
|
|
|
+ */
|
|
|
+ private Thread modificationThread(final List<FakeSchedulable> schedulableList) {
|
|
|
+ Thread modThread = new Thread() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ // This sleep is needed to make sure the sort has started before the
|
|
|
+ // modifications start and finish
|
|
|
+ Thread.sleep(500);
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ fail("Modification thread interrupted while asleep " +
|
|
|
+ ie.getMessage());
|
|
|
+ }
|
|
|
+ Resource newUsage = Resources.createResource(0, 0);
|
|
|
+ for (int j = 0; j < 1000; j++) {
|
|
|
+ FakeSchedulable sched = schedulableList.get(j * 10);
|
|
|
+ newUsage.setMemorySize(20000);
|
|
|
+ newUsage.setVirtualCores(j % 10);
|
|
|
+ sched.setResourceUsage(newUsage);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ };
|
|
|
+ return modThread;
|
|
|
+ }
|
|
|
}
|