1 /**
2  * Licensed to the Apache Software Foundation (ASF) under one
3  * or more contributor license agreements.  See the NOTICE file
4  * distributed with this work for additional information
5  * regarding copyright ownership.  The ASF licenses this file
6  * to you under the Apache License, Version 2.0 (the
7  * "License"); you may not use this file except in compliance
8  * with the License.  You may obtain a copy of the License at
9  *
10  *     http://www.apache.org/licenses/LICENSE-2.0
11  *
12  * Unless required by applicable law or agreed to in writing, software
13  * distributed under the License is distributed on an "AS IS" BASIS,
14  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15  * See the License for the specific language governing permissions and
16  * limitations under the License.
17  */
18 package org.apache.hadoop.mapred.gridmix;
19 
20 import java.io.IOException;
21 
22 import org.junit.Test;
23 import static org.junit.Assert.*;
24 
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.fs.FileSystem;
27 import org.apache.hadoop.fs.Path;
28 import org.apache.hadoop.mapreduce.MapContext;
29 import org.apache.hadoop.mapreduce.StatusReporter;
30 import org.apache.hadoop.mapreduce.TaskAttemptID;
31 import org.apache.hadoop.mapreduce.TaskInputOutputContext;
32 import org.apache.hadoop.util.ResourceCalculatorPlugin;
33 import org.apache.hadoop.util.ResourceCalculatorPlugin.ProcResourceValues;
34 import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
35 import org.apache.hadoop.util.DummyResourceCalculatorPlugin;
36 import org.apache.hadoop.mapred.TaskTracker;
37 import org.apache.hadoop.mapred.gridmix.LoadJob.ResourceUsageMatcherRunner;
38 import org.apache.hadoop.mapred.gridmix.emulators.resourceusage.CumulativeCpuUsageEmulatorPlugin;
39 import org.apache.hadoop.mapred.gridmix.emulators.resourceusage.ResourceUsageEmulatorPlugin;
40 import org.apache.hadoop.mapred.gridmix.emulators.resourceusage.ResourceUsageMatcher;
41 import org.apache.hadoop.mapred.gridmix.emulators.resourceusage.CumulativeCpuUsageEmulatorPlugin.DefaultCpuUsageEmulator;
42 
43 /**
44  * Test Gridmix's resource emulator framework and supported plugins.
45  */
46 public class TestResourceUsageEmulators {
47   /**
48    * A {@link ResourceUsageEmulatorPlugin} implementation for testing purpose.
49    * It essentially creates a file named 'test' in the test directory.
50    */
51   static class TestResourceUsageEmulatorPlugin
52   implements ResourceUsageEmulatorPlugin {
53     static final Path rootTempDir =
54         new Path(System.getProperty("test.build.data", "/tmp"));
55     static final Path tempDir =
56       new Path(rootTempDir, "TestResourceUsageEmulatorPlugin");
57     static final String DEFAULT_IDENTIFIER = "test";
58 
59     private Path touchPath = null;
60     private FileSystem fs = null;
61 
62     @Override
emulate()63     public void emulate() throws IOException, InterruptedException {
64       // add some time between 2 calls to emulate()
65       try {
66         Thread.sleep(1000); // sleep for 1s
67       } catch (Exception e){}
68 
69       try {
70         fs.delete(touchPath, false); // delete the touch file
71         //TODO Search for a better touch utility
72         fs.create(touchPath).close(); // recreate it
73       } catch (Exception e) {
74         throw new RuntimeException(e);
75       }
76     }
77 
getIdentifier()78     protected String getIdentifier() {
79       return DEFAULT_IDENTIFIER;
80     }
81 
getFilePath(String id)82     private static Path getFilePath(String id) {
83       return new Path(tempDir, id);
84     }
85 
getInitFilePath(String id)86     private static Path getInitFilePath(String id) {
87       return new Path(tempDir, id + ".init");
88     }
89 
90     @Override
initialize(Configuration conf, ResourceUsageMetrics metrics, ResourceCalculatorPlugin monitor, Progressive progress)91     public void initialize(Configuration conf, ResourceUsageMetrics metrics,
92         ResourceCalculatorPlugin monitor, Progressive progress) {
93       // add some time between 2 calls to initialize()
94       try {
95         Thread.sleep(1000); // sleep for 1s
96       } catch (Exception e){}
97 
98       try {
99         fs = FileSystem.getLocal(conf);
100 
101         Path initPath = getInitFilePath(getIdentifier());
102         fs.delete(initPath, false); // delete the old file
103         fs.create(initPath).close(); // create a new one
104 
105         touchPath = getFilePath(getIdentifier());
106         fs.delete(touchPath, false);
107       } catch (Exception e) {
108 
109       } finally {
110         if (fs != null) {
111           try {
112             fs.deleteOnExit(tempDir);
113           } catch (IOException ioe){}
114         }
115       }
116     }
117 
118     // test if the emulation framework successfully loaded this plugin
testInitialization(String id, Configuration conf)119     static long testInitialization(String id, Configuration conf)
120     throws IOException {
121       Path testPath = getInitFilePath(id);
122       FileSystem fs = FileSystem.getLocal(conf);
123       return fs.exists(testPath)
124              ? fs.getFileStatus(testPath).getModificationTime()
125              : 0;
126     }
127 
128     // test if the emulation framework successfully loaded this plugin
testEmulation(String id, Configuration conf)129     static long testEmulation(String id, Configuration conf)
130     throws IOException {
131       Path testPath = getFilePath(id);
132       FileSystem fs = FileSystem.getLocal(conf);
133       return fs.exists(testPath)
134              ? fs.getFileStatus(testPath).getModificationTime()
135              : 0;
136     }
137 
138     @Override
getProgress()139     public float getProgress() {
140       try {
141         return fs.exists(touchPath) ? 1.0f : 0f;
142       } catch (IOException ioe) {}
143       return 0f;
144     }
145   }
146 
147   /**
148    * Test implementation of {@link ResourceUsageEmulatorPlugin} which creates
149    * a file named 'others' in the test directory.
150    */
151   static class TestOthers extends TestResourceUsageEmulatorPlugin {
152     static final String ID = "others";
153 
154     @Override
getIdentifier()155     protected String getIdentifier() {
156       return ID;
157     }
158   }
159 
160   /**
161    * Test implementation of {@link ResourceUsageEmulatorPlugin} which creates
162    * a file named 'cpu' in the test directory.
163    */
164   static class TestCpu extends TestResourceUsageEmulatorPlugin {
165     static final String ID = "cpu";
166 
167     @Override
getIdentifier()168     protected String getIdentifier() {
169       return ID;
170     }
171   }
172 
173   /**
174    * Test {@link ResourceUsageMatcher}.
175    */
176   @Test
testResourceUsageMatcher()177   public void testResourceUsageMatcher() throws Exception {
178     ResourceUsageMatcher matcher = new ResourceUsageMatcher();
179     Configuration conf = new Configuration();
180     conf.setClass(ResourceUsageMatcher.RESOURCE_USAGE_EMULATION_PLUGINS,
181                   TestResourceUsageEmulatorPlugin.class,
182                   ResourceUsageEmulatorPlugin.class);
183     long currentTime = System.currentTimeMillis();
184 
185     matcher.configure(conf, null, null, null);
186 
187     matcher.matchResourceUsage();
188 
189     String id = TestResourceUsageEmulatorPlugin.DEFAULT_IDENTIFIER;
190     long result =
191       TestResourceUsageEmulatorPlugin.testInitialization(id, conf);
192     assertTrue("Resource usage matcher failed to initialize the configured"
193                + " plugin", result > currentTime);
194     result = TestResourceUsageEmulatorPlugin.testEmulation(id, conf);
195     assertTrue("Resource usage matcher failed to load and emulate the"
196                + " configured plugin", result > currentTime);
197 
198     // test plugin order to first emulate cpu and then others
199     conf.setStrings(ResourceUsageMatcher.RESOURCE_USAGE_EMULATION_PLUGINS,
200                     TestCpu.class.getName() + "," + TestOthers.class.getName());
201 
202     matcher.configure(conf, null, null, null);
203 
204     // test the initialization order
205     long time1 =
206            TestResourceUsageEmulatorPlugin.testInitialization(TestCpu.ID, conf);
207     long time2 =
208            TestResourceUsageEmulatorPlugin.testInitialization(TestOthers.ID,
209                                                               conf);
210     assertTrue("Resource usage matcher failed to initialize the configured"
211                + " plugins in order", time1 < time2);
212 
213     matcher.matchResourceUsage();
214 
215     // Note that the cpu usage emulator plugin is configured 1st and then the
216     // others plugin.
217     time1 =
218       TestResourceUsageEmulatorPlugin.testInitialization(TestCpu.ID, conf);
219     time2 =
220       TestResourceUsageEmulatorPlugin.testInitialization(TestOthers.ID,
221                                                          conf);
222     assertTrue("Resource usage matcher failed to load the configured plugins",
223                time1 < time2);
224   }
225 
226   /**
227    * Fakes the cumulative usage using {@link FakeCpuUsageEmulatorCore}.
228    */
229   static class FakeResourceUsageMonitor extends DummyResourceCalculatorPlugin {
230     private FakeCpuUsageEmulatorCore core;
231 
FakeResourceUsageMonitor(FakeCpuUsageEmulatorCore core)232     public FakeResourceUsageMonitor(FakeCpuUsageEmulatorCore core) {
233       this.core = core;
234     }
235 
236     /**
237      * A dummy CPU usage monitor. Every call to
238      * {@link ResourceCalculatorPlugin#getCumulativeCpuTime()} will return the
239      * value of {@link FakeCpuUsageEmulatorCore#getNumCalls()}.
240      */
241     @Override
getCumulativeCpuTime()242     public long getCumulativeCpuTime() {
243       return core.getCpuUsage();
244     }
245 
246     /**
247      * Returns a {@link ProcResourceValues} with cumulative cpu usage
248      * computed using {@link #getCumulativeCpuTime()}.
249      */
250     @Override
getProcResourceValues()251     public ProcResourceValues getProcResourceValues() {
252       long usageValue = getCumulativeCpuTime();
253       return new ProcResourceValues(usageValue, -1, -1);
254     }
255   }
256 
257   /**
258    * A dummy {@link Progressive} implementation that allows users to set the
259    * progress for testing. The {@link Progressive#getProgress()} call will
260    * return the last progress value set using
261    * {@link FakeProgressive#setProgress(float)}.
262    */
263   static class FakeProgressive implements Progressive {
264     private float progress = 0F;
265     @Override
getProgress()266     public float getProgress() {
267       return progress;
268     }
269 
setProgress(float progress)270     void setProgress(float progress) {
271       this.progress = progress;
272     }
273   }
274 
275   /**
276    * A dummy reporter for {@link LoadJob.ResourceUsageMatcherRunner}.
277    */
278   private static class DummyReporter extends StatusReporter {
279     private Progressive progress;
280 
DummyReporter(Progressive progress)281     DummyReporter(Progressive progress) {
282       this.progress = progress;
283     }
284 
285     @Override
getCounter(Enum<?> name)286     public org.apache.hadoop.mapreduce.Counter getCounter(Enum<?> name) {
287       return null;
288     }
289 
290     @Override
getCounter(String group, String name)291     public org.apache.hadoop.mapreduce.Counter getCounter(String group,
292                                                           String name) {
293       return null;
294     }
295 
296     @Override
progress()297     public void progress() {
298     }
299 
300     @Override
getProgress()301     public float getProgress() {
302       return progress.getProgress();
303     }
304 
305     @Override
setStatus(String status)306     public void setStatus(String status) {
307     }
308   }
309 
310   // Extends ResourceUsageMatcherRunner for testing.
311   @SuppressWarnings("unchecked")
312   private static class FakeResourceUsageMatcherRunner
313   extends ResourceUsageMatcherRunner {
FakeResourceUsageMatcherRunner(TaskInputOutputContext context, ResourceUsageMetrics metrics)314     FakeResourceUsageMatcherRunner(TaskInputOutputContext context,
315                                    ResourceUsageMetrics metrics) {
316       super(context, metrics);
317     }
318 
319     // test ResourceUsageMatcherRunner
test()320     void test() throws Exception {
321       super.match();
322     }
323   }
324 
325   /**
326    * Test {@link LoadJob.ResourceUsageMatcherRunner}.
327    */
328   @Test
329   @SuppressWarnings("unchecked")
testResourceUsageMatcherRunner()330   public void testResourceUsageMatcherRunner() throws Exception {
331     Configuration conf = new Configuration();
332     FakeProgressive progress = new FakeProgressive();
333 
334     // set the resource calculator plugin
335     conf.setClass(TaskTracker.TT_RESOURCE_CALCULATOR_PLUGIN,
336                   DummyResourceCalculatorPlugin.class,
337                   ResourceCalculatorPlugin.class);
338     // set the resources
339     // set the resource implementation class
340     conf.setClass(ResourceUsageMatcher.RESOURCE_USAGE_EMULATION_PLUGINS,
341                   TestResourceUsageEmulatorPlugin.class,
342                   ResourceUsageEmulatorPlugin.class);
343 
344     long currentTime = System.currentTimeMillis();
345 
346     // initialize the matcher class
347     TaskAttemptID id = new TaskAttemptID("test", 1, true, 1, 1);
348     StatusReporter reporter = new DummyReporter(progress);
349     TaskInputOutputContext context =
350       new MapContext(conf, id, null, null, null, reporter, null);
351     FakeResourceUsageMatcherRunner matcher =
352       new FakeResourceUsageMatcherRunner(context, null);
353 
354     // check if the matcher initialized the plugin
355     String identifier = TestResourceUsageEmulatorPlugin.DEFAULT_IDENTIFIER;
356     long initTime =
357       TestResourceUsageEmulatorPlugin.testInitialization(identifier, conf);
358     assertTrue("ResourceUsageMatcherRunner failed to initialize the"
359                + " configured plugin", initTime > currentTime);
360 
361     // check the progress
362     assertEquals("Progress mismatch in ResourceUsageMatcherRunner",
363                  0, progress.getProgress(), 0D);
364 
365     // call match() and check progress
366     progress.setProgress(0.01f);
367     currentTime = System.currentTimeMillis();
368     matcher.test();
369     long emulateTime =
370       TestResourceUsageEmulatorPlugin.testEmulation(identifier, conf);
371     assertTrue("ProgressBasedResourceUsageMatcher failed to load and emulate"
372                + " the configured plugin", emulateTime > currentTime);
373   }
374 
375   /**
376    * Test {@link CumulativeCpuUsageEmulatorPlugin}'s core CPU usage emulation
377    * engine.
378    */
379   @Test
testCpuUsageEmulator()380   public void testCpuUsageEmulator() throws IOException {
381     // test CpuUsageEmulator calibration with fake resource calculator plugin
382     long target = 100000L; // 100 secs
383     int unitUsage = 50;
384     FakeCpuUsageEmulatorCore fakeCpuEmulator = new FakeCpuUsageEmulatorCore();
385     fakeCpuEmulator.setUnitUsage(unitUsage);
386     FakeResourceUsageMonitor fakeMonitor =
387       new FakeResourceUsageMonitor(fakeCpuEmulator);
388 
389     // calibrate for 100ms
390     fakeCpuEmulator.calibrate(fakeMonitor, target);
391 
392     // by default, CpuUsageEmulator.calibrate() will consume 100ms of CPU usage
393     assertEquals("Fake calibration failed",
394                  100, fakeMonitor.getCumulativeCpuTime());
395     assertEquals("Fake calibration failed",
396                  100, fakeCpuEmulator.getCpuUsage());
397     // by default, CpuUsageEmulator.performUnitComputation() will be called
398     // twice
399     assertEquals("Fake calibration failed",
400                  2, fakeCpuEmulator.getNumCalls());
401   }
402 
403   /**
404    * This is a dummy class that fakes CPU usage.
405    */
406   private static class FakeCpuUsageEmulatorCore
407   extends DefaultCpuUsageEmulator {
408     private int numCalls = 0;
409     private int unitUsage = 1;
410     private int cpuUsage = 0;
411 
412     @Override
performUnitComputation()413     protected void performUnitComputation() {
414       ++numCalls;
415       cpuUsage += unitUsage;
416     }
417 
getNumCalls()418     int getNumCalls() {
419       return numCalls;
420     }
421 
getCpuUsage()422     int getCpuUsage() {
423       return cpuUsage;
424     }
425 
reset()426     void reset() {
427       numCalls = 0;
428       cpuUsage = 0;
429     }
430 
setUnitUsage(int unitUsage)431     void setUnitUsage(int unitUsage) {
432       this.unitUsage = unitUsage;
433     }
434   }
435 
436   // Creates a ResourceUsageMetrics object from the target usage
createMetrics(long target)437   static ResourceUsageMetrics createMetrics(long target) {
438     ResourceUsageMetrics metrics = new ResourceUsageMetrics();
439     metrics.setCumulativeCpuUsage(target);
440     metrics.setVirtualMemoryUsage(target);
441     metrics.setPhysicalMemoryUsage(target);
442     metrics.setHeapUsage(target);
443     return metrics;
444   }
445 
446   /**
447    * Test {@link CumulativeCpuUsageEmulatorPlugin}.
448    */
449   @Test
testCumulativeCpuUsageEmulatorPlugin()450   public void testCumulativeCpuUsageEmulatorPlugin() throws Exception {
451     Configuration conf = new Configuration();
452     long targetCpuUsage = 1000L;
453     int unitCpuUsage = 50;
454 
455     // fake progress indicator
456     FakeProgressive fakeProgress = new FakeProgressive();
457 
458     // fake cpu usage generator
459     FakeCpuUsageEmulatorCore fakeCore = new FakeCpuUsageEmulatorCore();
460     fakeCore.setUnitUsage(unitCpuUsage);
461 
462     // a cumulative cpu usage emulator with fake core
463     CumulativeCpuUsageEmulatorPlugin cpuPlugin =
464       new CumulativeCpuUsageEmulatorPlugin(fakeCore);
465 
466     // test with invalid or missing resource usage value
467     ResourceUsageMetrics invalidUsage = createMetrics(0);
468     cpuPlugin.initialize(conf, invalidUsage, null, null);
469 
470     // test if disabled cpu emulation plugin's emulate() call is a no-operation
471     // this will test if the emulation plugin is disabled or not
472     int numCallsPre = fakeCore.getNumCalls();
473     long cpuUsagePre = fakeCore.getCpuUsage();
474     cpuPlugin.emulate();
475     int numCallsPost = fakeCore.getNumCalls();
476     long cpuUsagePost = fakeCore.getCpuUsage();
477 
478     //  test if no calls are made cpu usage emulator core
479     assertEquals("Disabled cumulative CPU usage emulation plugin works!",
480                  numCallsPre, numCallsPost);
481 
482     //  test if no calls are made cpu usage emulator core
483     assertEquals("Disabled cumulative CPU usage emulation plugin works!",
484                  cpuUsagePre, cpuUsagePost);
485 
486     // test with valid resource usage value
487     ResourceUsageMetrics metrics = createMetrics(targetCpuUsage);
488 
489     // fake monitor
490     ResourceCalculatorPlugin monitor = new FakeResourceUsageMonitor(fakeCore);
491 
492     // test with default emulation interval
493     testEmulationAccuracy(conf, fakeCore, monitor, metrics, cpuPlugin,
494                           targetCpuUsage, targetCpuUsage / unitCpuUsage);
495 
496     // test with custom value for emulation interval of 20%
497     conf.setFloat(CumulativeCpuUsageEmulatorPlugin.CPU_EMULATION_PROGRESS_INTERVAL,
498                   0.2F);
499     testEmulationAccuracy(conf, fakeCore, monitor, metrics, cpuPlugin,
500                           targetCpuUsage, targetCpuUsage / unitCpuUsage);
501 
502     // test if emulation interval boundary is respected (unit usage = 1)
503     //  test the case where the current progress is less than threshold
504     fakeProgress = new FakeProgressive(); // initialize
505     fakeCore.reset();
506     fakeCore.setUnitUsage(1);
507     conf.setFloat(CumulativeCpuUsageEmulatorPlugin.CPU_EMULATION_PROGRESS_INTERVAL,
508                   0.25F);
509     cpuPlugin.initialize(conf, metrics, monitor, fakeProgress);
510     // take a snapshot after the initialization
511     long initCpuUsage = monitor.getCumulativeCpuTime();
512     long initNumCalls = fakeCore.getNumCalls();
513     // test with 0 progress
514     testEmulationBoundary(0F, fakeCore, fakeProgress, cpuPlugin, initCpuUsage,
515                           initNumCalls, "[no-op, 0 progress]");
516     // test with 24% progress
517     testEmulationBoundary(0.24F, fakeCore, fakeProgress, cpuPlugin,
518                           initCpuUsage, initNumCalls, "[no-op, 24% progress]");
519     // test with 25% progress
520     //  target = 1000ms, target emulation at 25% = 250ms,
521     //  weighed target = 1000 * 0.25^4 (we are using progress^4 as the weight)
522     //                 ~ 4
523     //  but current usage = init-usage = 100, hence expected = 100
524     testEmulationBoundary(0.25F, fakeCore, fakeProgress, cpuPlugin,
525                           initCpuUsage, initNumCalls, "[op, 25% progress]");
526 
527     // test with 80% progress
528     //  target = 1000ms, target emulation at 80% = 800ms,
529     //  weighed target = 1000 * 0.25^4 (we are using progress^4 as the weight)
530     //                 ~ 410
531     //  current-usage = init-usage = 100, hence expected-usage = 410
532     testEmulationBoundary(0.80F, fakeCore, fakeProgress, cpuPlugin, 410, 410,
533                           "[op, 80% progress]");
534 
535     // now test if the final call with 100% progress ramps up the CPU usage
536     testEmulationBoundary(1F, fakeCore, fakeProgress, cpuPlugin, targetCpuUsage,
537                           targetCpuUsage, "[op, 100% progress]");
538 
539     // test if emulation interval boundary is respected (unit usage = 50)
540     //  test the case where the current progress is less than threshold
541     fakeProgress = new FakeProgressive(); // initialize
542     fakeCore.reset();
543     fakeCore.setUnitUsage(unitCpuUsage);
544     conf.setFloat(CumulativeCpuUsageEmulatorPlugin.CPU_EMULATION_PROGRESS_INTERVAL,
545                   0.40F);
546     cpuPlugin.initialize(conf, metrics, monitor, fakeProgress);
547     // take a snapshot after the initialization
548     initCpuUsage = monitor.getCumulativeCpuTime();
549     initNumCalls = fakeCore.getNumCalls();
550     // test with 0 progress
551     testEmulationBoundary(0F, fakeCore, fakeProgress, cpuPlugin, initCpuUsage,
552                           initNumCalls, "[no-op, 0 progress]");
553     // test with 39% progress
554     testEmulationBoundary(0.39F, fakeCore, fakeProgress, cpuPlugin,
555                           initCpuUsage, initNumCalls, "[no-op, 39% progress]");
556     // test with 40% progress
557     //  target = 1000ms, target emulation at 40% = 4000ms,
558     //  weighed target = 1000 * 0.40^4 (we are using progress^4 as the weight)
559     //                 ~ 26
560     // current-usage = init-usage = 100, hence expected-usage = 100
561     testEmulationBoundary(0.40F, fakeCore, fakeProgress, cpuPlugin,
562                           initCpuUsage, initNumCalls, "[op, 40% progress]");
563 
564     // test with 90% progress
565     //  target = 1000ms, target emulation at 90% = 900ms,
566     //  weighed target = 1000 * 0.90^4 (we are using progress^4 as the weight)
567     //                 ~ 657
568     //  current-usage = init-usage = 100, hence expected-usage = 657 but
569     //  the fake-core increases in steps of 50, hence final target = 700
570     testEmulationBoundary(0.90F, fakeCore, fakeProgress, cpuPlugin, 700,
571                           700 / unitCpuUsage, "[op, 90% progress]");
572 
573     // now test if the final call with 100% progress ramps up the CPU usage
574     testEmulationBoundary(1F, fakeCore, fakeProgress, cpuPlugin, targetCpuUsage,
575                           targetCpuUsage / unitCpuUsage, "[op, 100% progress]");
576   }
577 
578   // test whether the CPU usage emulator achieves the desired target using
579   // desired calls to the underling core engine.
testEmulationAccuracy(Configuration conf, FakeCpuUsageEmulatorCore fakeCore, ResourceCalculatorPlugin monitor, ResourceUsageMetrics metrics, CumulativeCpuUsageEmulatorPlugin cpuPlugin, long expectedTotalCpuUsage, long expectedTotalNumCalls)580   private static void testEmulationAccuracy(Configuration conf,
581                         FakeCpuUsageEmulatorCore fakeCore,
582                         ResourceCalculatorPlugin monitor,
583                         ResourceUsageMetrics metrics,
584                         CumulativeCpuUsageEmulatorPlugin cpuPlugin,
585                         long expectedTotalCpuUsage, long expectedTotalNumCalls)
586   throws Exception {
587     FakeProgressive fakeProgress = new FakeProgressive();
588     fakeCore.reset();
589     cpuPlugin.initialize(conf, metrics, monitor, fakeProgress);
590     int numLoops = 0;
591     while (fakeProgress.getProgress() < 1) {
592       ++numLoops;
593       float progress = (float)numLoops / 100;
594       fakeProgress.setProgress(progress);
595       cpuPlugin.emulate();
596     }
597 
598     // test if the resource plugin shows the expected invocations
599     assertEquals("Cumulative cpu usage emulator plugin failed (num calls)!",
600                  expectedTotalNumCalls, fakeCore.getNumCalls(), 0L);
601     // test if the resource plugin shows the expected usage
602     assertEquals("Cumulative cpu usage emulator plugin failed (total usage)!",
603                  expectedTotalCpuUsage, fakeCore.getCpuUsage(), 0L);
604   }
605 
606   // tests if the CPU usage emulation plugin emulates only at the expected
607   // progress gaps
testEmulationBoundary(float progress, FakeCpuUsageEmulatorCore fakeCore, FakeProgressive fakeProgress, CumulativeCpuUsageEmulatorPlugin cpuPlugin, long expectedTotalCpuUsage, long expectedTotalNumCalls, String info)608   private static void testEmulationBoundary(float progress,
609       FakeCpuUsageEmulatorCore fakeCore, FakeProgressive fakeProgress,
610       CumulativeCpuUsageEmulatorPlugin cpuPlugin, long expectedTotalCpuUsage,
611       long expectedTotalNumCalls, String info) throws Exception {
612     fakeProgress.setProgress(progress);
613     cpuPlugin.emulate();
614 
615     assertEquals("Emulation interval test for cpu usage failed " + info + "!",
616                  expectedTotalCpuUsage, fakeCore.getCpuUsage(), 0L);
617     assertEquals("Emulation interval test for num calls failed " + info + "!",
618                  expectedTotalNumCalls, fakeCore.getNumCalls(), 0L);
619   }
620 }