blob: fc675b57ea8472d7dbb8deadd150292fdd6e9e3a [file] [log] [blame]
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01001// Copyright 2014 Google Inc. All rights reserved.
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7// http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14package com.google.devtools.build.skyframe;
15
16import static com.google.common.collect.Iterables.getOnlyElement;
17import static com.google.common.truth.Truth.assertThat;
18import static com.google.common.truth.Truth.assertWithMessage;
Ulf Adams37cc83d2015-07-23 08:18:18 +000019import static com.google.devtools.build.lib.testutil.MoreAsserts.assertContainsEvent;
20import static com.google.devtools.build.lib.testutil.MoreAsserts.assertEventCount;
21import static com.google.devtools.build.lib.testutil.MoreAsserts.assertGreaterThan;
22import static com.google.devtools.build.lib.testutil.MoreAsserts.assertLessThan;
23import static com.google.devtools.build.lib.testutil.MoreAsserts.assertNoEvents;
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +010024import static com.google.devtools.build.skyframe.GraphTester.CONCATENATE;
25import static org.junit.Assert.assertEquals;
26import static org.junit.Assert.assertFalse;
27import static org.junit.Assert.assertNotNull;
Mark Schallercc34ba02015-03-03 23:55:30 +000028import static org.junit.Assert.assertNull;
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +010029import static org.junit.Assert.assertSame;
30import static org.junit.Assert.assertTrue;
31import static org.junit.Assert.fail;
32
Janak Ramakrishnan39ad9662015-07-15 12:02:53 +000033import com.google.common.base.Predicate;
Mark Schaller4752dbb2015-08-20 18:57:44 +000034import com.google.common.base.Supplier;
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +010035import com.google.common.collect.ImmutableList;
36import com.google.common.collect.ImmutableMap;
37import com.google.common.collect.ImmutableSet;
38import com.google.common.collect.Iterables;
39import com.google.common.collect.Lists;
40import com.google.common.collect.Sets;
41import com.google.common.util.concurrent.Uninterruptibles;
42import com.google.devtools.build.lib.events.Event;
43import com.google.devtools.build.lib.events.EventCollector;
44import com.google.devtools.build.lib.events.EventHandler;
45import com.google.devtools.build.lib.events.EventKind;
46import com.google.devtools.build.lib.events.OutputFilter.RegexOutputFilter;
47import com.google.devtools.build.lib.events.Reporter;
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +010048import com.google.devtools.build.lib.testutil.TestThread;
49import com.google.devtools.build.lib.testutil.TestUtils;
50import com.google.devtools.build.skyframe.GraphTester.StringValue;
51import com.google.devtools.build.skyframe.NotifyingInMemoryGraph.EventType;
52import com.google.devtools.build.skyframe.NotifyingInMemoryGraph.Listener;
53import com.google.devtools.build.skyframe.NotifyingInMemoryGraph.Order;
54import com.google.devtools.build.skyframe.SkyFunctionException.Transience;
55
Janak Ramakrishnana67bb8b2015-09-02 17:56:08 +000056import org.junit.After;
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +010057import org.junit.Assert;
58import org.junit.Before;
59import org.junit.Test;
60import org.junit.runner.RunWith;
61import org.junit.runners.JUnit4;
62
63import java.util.ArrayList;
64import java.util.Arrays;
65import java.util.List;
66import java.util.Map;
67import java.util.Set;
68import java.util.concurrent.CountDownLatch;
69import java.util.concurrent.Semaphore;
70import java.util.concurrent.TimeUnit;
71import java.util.concurrent.atomic.AtomicBoolean;
72import java.util.concurrent.atomic.AtomicInteger;
73import java.util.concurrent.atomic.AtomicReference;
74
75import javax.annotation.Nullable;
76
77/**
78 * Tests for {@link ParallelEvaluator}.
79 */
80@RunWith(JUnit4.class)
81public class ParallelEvaluatorTest {
82 protected ProcessableGraph graph;
83 protected IntVersion graphVersion = new IntVersion(0);
84 protected GraphTester tester = new GraphTester();
85
86 private EventCollector eventCollector;
87 private EventHandler reporter;
88
89 private EvaluationProgressReceiver revalidationReceiver;
90
91 @Before
92 public void initializeReporter() {
93 eventCollector = new EventCollector(EventKind.ALL_EVENTS);
94 reporter = new Reporter(eventCollector);
95 }
96
Janak Ramakrishnana67bb8b2015-09-02 17:56:08 +000097 @After
98 public void assertNoTrackedErrors() {
99 TrackingAwaiter.INSTANCE.assertNoErrors();
Janak Ramakrishnan9c232032015-09-09 01:08:32 +0000100 if (graph instanceof NotifyingInMemoryGraph) {
101 ((NotifyingInMemoryGraph) graph).assertNoExceptions();
102 }
Janak Ramakrishnana67bb8b2015-09-02 17:56:08 +0000103 }
104
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100105 private ParallelEvaluator makeEvaluator(ProcessableGraph graph,
Janak Ramakrishnan39ad9662015-07-15 12:02:53 +0000106 ImmutableMap<SkyFunctionName, ? extends SkyFunction> builders, boolean keepGoing,
107 Predicate<Event> storedEventFilter) {
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100108 Version oldGraphVersion = graphVersion;
109 graphVersion = graphVersion.next();
110 return new ParallelEvaluator(graph, oldGraphVersion,
Janak Ramakrishnan39ad9662015-07-15 12:02:53 +0000111 builders, reporter, new MemoizingEvaluator.EmittedEventState(), storedEventFilter,
112 keepGoing, 150, revalidationReceiver, new DirtyKeyTrackerImpl());
113 }
114
115 private ParallelEvaluator makeEvaluator(ProcessableGraph graph,
116 ImmutableMap<SkyFunctionName, ? extends SkyFunction> builders, boolean keepGoing) {
117 return makeEvaluator(graph, builders, keepGoing,
118 InMemoryMemoizingEvaluator.DEFAULT_STORED_EVENT_FILTER);
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100119 }
120
121 /** Convenience method for eval-ing a single value. */
122 protected SkyValue eval(boolean keepGoing, SkyKey key) throws InterruptedException {
123 return eval(keepGoing, ImmutableList.of(key)).get(key);
124 }
125
126 protected ErrorInfo evalValueInError(SkyKey key) throws InterruptedException {
127 return eval(true, ImmutableList.of(key)).getError(key);
128 }
129
130 protected <T extends SkyValue> EvaluationResult<T> eval(boolean keepGoing, SkyKey... keys)
131 throws InterruptedException {
132 return eval(keepGoing, ImmutableList.copyOf(keys));
133 }
134
135 protected <T extends SkyValue> EvaluationResult<T> eval(boolean keepGoing, Iterable<SkyKey> keys)
136 throws InterruptedException {
137 ParallelEvaluator evaluator = makeEvaluator(graph,
138 ImmutableMap.of(GraphTester.NODE_TYPE, tester.createDelegatingFunction()),
139 keepGoing);
140 return evaluator.eval(keys);
141 }
142
143 protected GraphTester.TestFunction set(String name, String value) {
144 return tester.set(name, new StringValue(value));
145 }
146
147 @Test
148 public void smoke() throws Exception {
149 graph = new InMemoryGraph();
150 set("a", "a");
151 set("b", "b");
152 tester.getOrCreate("ab").addDependency("a").addDependency("b").setComputedValue(CONCATENATE);
153 StringValue value = (StringValue) eval(false, GraphTester.toSkyKey("ab"));
154 assertEquals("ab", value.getValue());
Ulf Adams37cc83d2015-07-23 08:18:18 +0000155 assertNoEvents(eventCollector);
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100156 }
157
158 /**
159 * Test interruption handling when a long-running SkyFunction gets interrupted.
160 */
161 @Test
162 public void interruptedFunction() throws Exception {
163 runInterruptionTest(new SkyFunctionFactory() {
164 @Override
165 public SkyFunction create(final Semaphore threadStarted, final String[] errorMessage) {
166 return new SkyFunction() {
167 @Override
168 public SkyValue compute(SkyKey key, Environment env) throws InterruptedException {
169 // Signal the waiting test thread that the evaluator thread has really started.
170 threadStarted.release();
171
172 // Simulate a SkyFunction that runs for 10 seconds (this number was chosen arbitrarily).
173 // The main thread should interrupt it shortly after it got started.
174 Thread.sleep(10 * 1000);
175
176 // Set an error message to indicate that the expected interruption didn't happen.
177 // We can't use Assert.fail(String) on an async thread.
178 errorMessage[0] = "SkyFunction should have been interrupted";
179 return null;
180 }
181
182 @Nullable
183 @Override
184 public String extractTag(SkyKey skyKey) {
185 return null;
186 }
187 };
188 }
189 });
190 }
191
192 /**
193 * Test interruption handling when the Evaluator is in-between running SkyFunctions.
194 *
195 * <p>This is the point in time after a SkyFunction requested a dependency which is not yet built
196 * so the builder returned null to the Evaluator, and the latter is about to schedule evaluation
197 * of the missing dependency but gets interrupted before the dependency's SkyFunction could start.
198 */
199 @Test
200 public void interruptedEvaluatorThread() throws Exception {
201 runInterruptionTest(new SkyFunctionFactory() {
202 @Override
203 public SkyFunction create(final Semaphore threadStarted, final String[] errorMessage) {
204 return new SkyFunction() {
205 // No need to synchronize access to this field; we always request just one more
206 // dependency, so it's only one SkyFunction running at any time.
207 private int valueIdCounter = 0;
208
209 @Override
210 public SkyValue compute(SkyKey key, Environment env) {
211 // Signal the waiting test thread that the Evaluator thread has really started.
212 threadStarted.release();
213
214 // Keep the evaluator busy until the test's thread gets scheduled and can
215 // interrupt the Evaluator's thread.
216 env.getValue(GraphTester.toSkyKey("a" + valueIdCounter++));
217
218 // This method never throws InterruptedException, therefore it's the responsibility
219 // of the Evaluator to detect the interrupt and avoid calling subsequent SkyFunctions.
220 return null;
221 }
222
223 @Nullable
224 @Override
225 public String extractTag(SkyKey skyKey) {
226 return null;
227 }
228 };
229 }
230 });
231 }
232
233 private void runPartialResultOnInterruption(boolean buildFastFirst) throws Exception {
234 graph = new InMemoryGraph();
235 // Two runs for fastKey's builder and one for the start of waitKey's builder.
236 final CountDownLatch allValuesReady = new CountDownLatch(3);
237 final SkyKey waitKey = GraphTester.toSkyKey("wait");
238 final SkyKey fastKey = GraphTester.toSkyKey("fast");
239 SkyKey leafKey = GraphTester.toSkyKey("leaf");
240 tester.getOrCreate(waitKey).setBuilder(new SkyFunction() {
241 @Override
242 public SkyValue compute(SkyKey skyKey, Environment env) throws InterruptedException {
243 allValuesReady.countDown();
244 Thread.sleep(10000);
245 throw new AssertionError("Should have been interrupted");
246 }
247
248 @Override
249 public String extractTag(SkyKey skyKey) {
250 return null;
251 }
252 });
253 tester.getOrCreate(fastKey).setBuilder(new ChainedFunction(null, null, allValuesReady, false,
254 new StringValue("fast"), ImmutableList.of(leafKey)));
255 tester.set(leafKey, new StringValue("leaf"));
256 if (buildFastFirst) {
257 eval(/*keepGoing=*/false, fastKey);
258 }
259 final Set<SkyKey> receivedValues = Sets.newConcurrentHashSet();
260 revalidationReceiver = new EvaluationProgressReceiver() {
261 @Override
Mark Schallerd12c3982015-08-10 19:46:03 +0000262 public void invalidated(SkyKey skyKey, InvalidationState state) {}
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100263
264 @Override
265 public void enqueueing(SkyKey key) {}
266
267 @Override
Nathan Harmata6f094bb2015-09-03 19:27:38 +0000268 public void computed(SkyKey skyKey, long elapsedTimeNanos) {}
269
270 @Override
Mark Schaller4752dbb2015-08-20 18:57:44 +0000271 public void evaluated(SkyKey skyKey, Supplier<SkyValue> skyValueSupplier,
272 EvaluationState state) {
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100273 receivedValues.add(skyKey);
274 }
275 };
276 TestThread evalThread = new TestThread() {
277 @Override
278 public void runTest() throws Exception {
279 try {
280 eval(/*keepGoing=*/true, waitKey, fastKey);
281 fail();
282 } catch (InterruptedException e) {
283 // Expected.
284 }
285 }
286 };
287 evalThread.start();
288 assertTrue(allValuesReady.await(TestUtils.WAIT_TIMEOUT_SECONDS, TimeUnit.SECONDS));
289 evalThread.interrupt();
290 evalThread.join(TestUtils.WAIT_TIMEOUT_MILLISECONDS);
291 assertFalse(evalThread.isAlive());
292 if (buildFastFirst) {
293 // If leafKey was already built, it is not reported to the receiver.
294 assertThat(receivedValues).containsExactly(fastKey);
295 } else {
296 // On first time being built, leafKey is registered too.
297 assertThat(receivedValues).containsExactly(fastKey, leafKey);
298 }
299 }
300
301 @Test
302 public void partialResultOnInterruption() throws Exception {
303 runPartialResultOnInterruption(/*buildFastFirst=*/false);
304 }
305
306 @Test
307 public void partialCachedResultOnInterruption() throws Exception {
308 runPartialResultOnInterruption(/*buildFastFirst=*/true);
309 }
310
311 /**
312 * Factory for SkyFunctions for interruption testing (see {@link #runInterruptionTest}).
313 */
314 private interface SkyFunctionFactory {
315 /**
316 * Creates a SkyFunction suitable for a specific test scenario.
317 *
318 * @param threadStarted a latch which the returned SkyFunction must
319 * {@link Semaphore#release() release} once it started (otherwise the test won't work)
320 * @param errorMessage a single-element array; the SkyFunction can put a error message in it
321 * to indicate that an assertion failed (calling {@code fail} from async thread doesn't
322 * work)
323 */
324 SkyFunction create(final Semaphore threadStarted, final String[] errorMessage);
325 }
326
327 /**
328 * Test that we can handle the Evaluator getting interrupted at various points.
329 *
330 * <p>This method creates an Evaluator with the specified SkyFunction for GraphTested.NODE_TYPE,
331 * then starts a thread, requests evaluation and asserts that evaluation started. It then
332 * interrupts the Evaluator thread and asserts that it acknowledged the interruption.
333 *
334 * @param valueBuilderFactory creates a SkyFunction which may or may not handle interruptions
335 * (depending on the test)
336 */
337 private void runInterruptionTest(SkyFunctionFactory valueBuilderFactory) throws Exception {
338 final Semaphore threadStarted = new Semaphore(0);
339 final Semaphore threadInterrupted = new Semaphore(0);
340 final String[] wasError = new String[] { null };
341 final ParallelEvaluator evaluator = makeEvaluator(new InMemoryGraph(),
342 ImmutableMap.of(GraphTester.NODE_TYPE, valueBuilderFactory.create(threadStarted, wasError)),
343 false);
344
345 Thread t = new Thread(new Runnable() {
346 @Override
347 public void run() {
348 try {
349 evaluator.eval(ImmutableList.of(GraphTester.toSkyKey("a")));
350
351 // There's no real need to set an error here. If the thread is not interrupted then
352 // threadInterrupted is not released and the test thread will fail to acquire it.
353 wasError[0] = "evaluation should have been interrupted";
354 } catch (InterruptedException e) {
355 // This is the interrupt we are waiting for. It should come straight from the
356 // evaluator (more precisely, the AbstractQueueVisitor).
357 // Signal the waiting test thread that the interrupt was acknowledged.
358 threadInterrupted.release();
359 }
360 }
361 });
362
363 // Start the thread and wait for a semaphore. This ensures that the thread was really started.
364 t.start();
365 assertTrue(threadStarted.tryAcquire(TestUtils.WAIT_TIMEOUT_MILLISECONDS,
366 TimeUnit.MILLISECONDS));
367
368 // Interrupt the thread and wait for a semaphore. This ensures that the thread was really
369 // interrupted and this fact was acknowledged.
370 t.interrupt();
371 assertTrue(threadInterrupted.tryAcquire(TestUtils.WAIT_TIMEOUT_MILLISECONDS,
372 TimeUnit.MILLISECONDS));
373
374 // The SkyFunction may have reported an error.
375 if (wasError[0] != null) {
376 fail(wasError[0]);
377 }
378
379 // Wait for the thread to finish.
380 t.join(TestUtils.WAIT_TIMEOUT_MILLISECONDS);
381 }
382
383 @Test
384 public void unrecoverableError() throws Exception {
385 class CustomRuntimeException extends RuntimeException {}
386 final CustomRuntimeException expected = new CustomRuntimeException();
387
388 final SkyFunction builder = new SkyFunction() {
389 @Override
390 @Nullable
391 public SkyValue compute(SkyKey skyKey, Environment env)
392 throws SkyFunctionException, InterruptedException {
393 throw expected;
394 }
395
396 @Override
397 @Nullable
398 public String extractTag(SkyKey skyKey) {
399 return null;
400 }
401 };
402
403 final ParallelEvaluator evaluator = makeEvaluator(new InMemoryGraph(),
404 ImmutableMap.of(GraphTester.NODE_TYPE, builder),
405 false);
406
407 SkyKey valueToEval = GraphTester.toSkyKey("a");
408 try {
409 evaluator.eval(ImmutableList.of(valueToEval));
410 } catch (RuntimeException re) {
411 assertTrue(re.getMessage()
412 .contains("Unrecoverable error while evaluating node '" + valueToEval.toString() + "'"));
413 assertTrue(re.getCause() instanceof CustomRuntimeException);
414 }
415 }
416
417 @Test
418 public void simpleWarning() throws Exception {
419 graph = new InMemoryGraph();
420 set("a", "a").setWarning("warning on 'a'");
421 StringValue value = (StringValue) eval(false, GraphTester.toSkyKey("a"));
422 assertEquals("a", value.getValue());
Ulf Adams37cc83d2015-07-23 08:18:18 +0000423 assertContainsEvent(eventCollector, "warning on 'a'");
424 assertEventCount(1, eventCollector);
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100425 }
426
427 @Test
428 public void warningMatchesRegex() throws Exception {
429 graph = new InMemoryGraph();
430 ((Reporter) reporter).setOutputFilter(RegexOutputFilter.forRegex("a"));
431 set("example", "a value").setWarning("warning message");
432 SkyKey a = GraphTester.toSkyKey("example");
433 tester.getOrCreate(a).setTag("a");
434 StringValue value = (StringValue) eval(false, a);
435 assertEquals("a value", value.getValue());
Ulf Adams37cc83d2015-07-23 08:18:18 +0000436 assertContainsEvent(eventCollector, "warning message");
437 assertEventCount(1, eventCollector);
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100438 }
439
440 @Test
441 public void warningMatchesRegexOnlyTag() throws Exception {
442 graph = new InMemoryGraph();
443 ((Reporter) reporter).setOutputFilter(RegexOutputFilter.forRegex("a"));
444 set("a", "a value").setWarning("warning on 'a'");
445 SkyKey a = GraphTester.toSkyKey("a");
446 tester.getOrCreate(a).setTag("b");
447 StringValue value = (StringValue) eval(false, a);
448 assertEquals("a value", value.getValue());
Ulf Adams37cc83d2015-07-23 08:18:18 +0000449 assertEventCount(0, eventCollector); }
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100450
451 @Test
452 public void warningDoesNotMatchRegex() throws Exception {
453 graph = new InMemoryGraph();
454 ((Reporter) reporter).setOutputFilter(RegexOutputFilter.forRegex("b"));
455 set("a", "a").setWarning("warning on 'a'");
456 SkyKey a = GraphTester.toSkyKey("a");
457 tester.getOrCreate(a).setTag("a");
458 StringValue value = (StringValue) eval(false, a);
459 assertEquals("a", value.getValue());
Ulf Adams37cc83d2015-07-23 08:18:18 +0000460 assertEventCount(0, eventCollector);
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100461 }
462
463 /** Regression test: events from already-done value not replayed. */
464 @Test
465 public void eventFromDoneChildRecorded() throws Exception {
466 graph = new InMemoryGraph();
467 set("a", "a").setWarning("warning on 'a'");
468 SkyKey a = GraphTester.toSkyKey("a");
469 SkyKey top = GraphTester.toSkyKey("top");
470 tester.getOrCreate(top).addDependency(a).setComputedValue(CONCATENATE);
471 // Build a so that it is already in the graph.
472 eval(false, a);
Ulf Adams37cc83d2015-07-23 08:18:18 +0000473 assertEventCount(1, eventCollector);
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100474 eventCollector.clear();
475 // Build top. The warning from a should be reprinted.
476 eval(false, top);
Ulf Adams37cc83d2015-07-23 08:18:18 +0000477 assertEventCount(1, eventCollector);
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100478 eventCollector.clear();
479 // Build top again. The warning should have been stored in the value.
480 eval(false, top);
Ulf Adams37cc83d2015-07-23 08:18:18 +0000481 assertEventCount(1, eventCollector);
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100482 }
483
484 @Test
Janak Ramakrishnan39ad9662015-07-15 12:02:53 +0000485 public void storedEventFilter() throws Exception {
486 graph = new InMemoryGraph();
487 SkyKey a = GraphTester.toSkyKey("a");
488 final AtomicBoolean evaluated = new AtomicBoolean(false);
489 tester.getOrCreate(a).setBuilder(new SkyFunction() {
490 @Nullable
491 @Override
492 public SkyValue compute(SkyKey skyKey, Environment env) {
493 evaluated.set(true);
494 env.getListener().handle(Event.error(null, "boop"));
495 env.getListener().handle(Event.warn(null, "beep"));
496 return new StringValue("a");
497 }
498
499 @Nullable
500 @Override
501 public String extractTag(SkyKey skyKey) {
502 return null;
503 }
504 });
505 ParallelEvaluator evaluator = makeEvaluator(graph,
506 ImmutableMap.of(GraphTester.NODE_TYPE, tester.createDelegatingFunction()),
507 /*keepGoing=*/false, new Predicate<Event>() {
508 @Override
509 public boolean apply(Event event) {
510 return event.getKind() == EventKind.ERROR;
511 }
512 });
513 evaluator.eval(ImmutableList.of(a));
514 assertTrue(evaluated.get());
Ulf Adams37cc83d2015-07-23 08:18:18 +0000515 assertEventCount(2, eventCollector);
516 assertContainsEvent(eventCollector, "boop");
517 assertContainsEvent(eventCollector, "beep");
Janak Ramakrishnan39ad9662015-07-15 12:02:53 +0000518 eventCollector.clear();
519 evaluator = makeEvaluator(graph,
520 ImmutableMap.of(GraphTester.NODE_TYPE, tester.createDelegatingFunction()),
521 /*keepGoing=*/false);
522 evaluated.set(false);
523 evaluator.eval(ImmutableList.of(a));
524 assertFalse(evaluated.get());
Ulf Adams37cc83d2015-07-23 08:18:18 +0000525 assertEventCount(1, eventCollector);
526 assertContainsEvent(eventCollector, "boop");
Janak Ramakrishnan39ad9662015-07-15 12:02:53 +0000527 }
528
529 @Test
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100530 public void shouldCreateErrorValueWithRootCause() throws Exception {
531 graph = new InMemoryGraph();
532 set("a", "a");
533 SkyKey parentErrorKey = GraphTester.toSkyKey("parent");
534 SkyKey errorKey = GraphTester.toSkyKey("error");
535 tester.getOrCreate(parentErrorKey).addDependency("a").addDependency(errorKey)
536 .setComputedValue(CONCATENATE);
537 tester.getOrCreate(errorKey).setHasError(true);
538 ErrorInfo error = evalValueInError(parentErrorKey);
539 assertThat(error.getRootCauses()).containsExactly(errorKey);
540 }
541
542 @Test
543 public void shouldBuildOneTarget() throws Exception {
544 graph = new InMemoryGraph();
545 set("a", "a");
546 set("b", "b");
547 SkyKey parentErrorKey = GraphTester.toSkyKey("parent");
548 SkyKey errorFreeKey = GraphTester.toSkyKey("ab");
549 SkyKey errorKey = GraphTester.toSkyKey("error");
550 tester.getOrCreate(parentErrorKey).addDependency(errorKey).addDependency("a")
551 .setComputedValue(CONCATENATE);
552 tester.getOrCreate(errorKey).setHasError(true);
553 tester.getOrCreate(errorFreeKey).addDependency("a").addDependency("b")
554 .setComputedValue(CONCATENATE);
555 EvaluationResult<StringValue> result = eval(true, parentErrorKey, errorFreeKey);
556 ErrorInfo error = result.getError(parentErrorKey);
557 assertThat(error.getRootCauses()).containsExactly(errorKey);
558 StringValue abValue = result.get(errorFreeKey);
559 assertEquals("ab", abValue.getValue());
560 }
561
562 @Test
563 public void catastropheHaltsBuild_KeepGoing_KeepEdges() throws Exception {
564 catastrophicBuild(true, true);
565 }
566
567 @Test
568 public void catastropheHaltsBuild_KeepGoing_NoKeepEdges() throws Exception {
569 catastrophicBuild(true, false);
570 }
571
572 @Test
573 public void catastropheInBuild_NoKeepGoing_KeepEdges() throws Exception {
574 catastrophicBuild(false, true);
575 }
576
577 private void catastrophicBuild(boolean keepGoing, boolean keepEdges) throws Exception {
578 graph = new InMemoryGraph(keepEdges);
579
580 SkyKey catastropheKey = GraphTester.toSkyKey("catastrophe");
581 SkyKey otherKey = GraphTester.toSkyKey("someKey");
582
583 tester.getOrCreate(catastropheKey).setBuilder(new SkyFunction() {
584 @Nullable
585 @Override
586 public SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException {
587 throw new SkyFunctionException(new SomeErrorException("bad"),
588 Transience.PERSISTENT) {
589 @Override
590 public boolean isCatastrophic() {
591 return true;
592 }
593 };
594 }
595
596 @Nullable
597 @Override
598 public String extractTag(SkyKey skyKey) {
599 return null;
600 }
601 });
602
603 tester.getOrCreate(otherKey).setBuilder(new SkyFunction() {
604 @Nullable
605 @Override
606 public SkyValue compute(SkyKey skyKey, Environment env) throws InterruptedException {
607 new CountDownLatch(1).await();
608 throw new RuntimeException("can't get here");
609 }
610
611 @Nullable
612 @Override
613 public String extractTag(SkyKey skyKey) {
614 return null;
615 }
616 });
617
618 SkyKey topKey = GraphTester.toSkyKey("top");
619 tester.getOrCreate(topKey).addDependency(catastropheKey).setComputedValue(CONCATENATE);
620 EvaluationResult<StringValue> result = eval(keepGoing, topKey, otherKey);
621 if (!keepGoing) {
622 ErrorInfo error = result.getError(topKey);
623 assertThat(error.getRootCauses()).containsExactly(catastropheKey);
624 } else {
625 assertTrue(result.hasError());
626 assertThat(result.errorMap()).isEmpty();
627 }
628 }
629
630 @Test
631 public void parentFailureDoesntAffectChild() throws Exception {
632 graph = new InMemoryGraph();
633 SkyKey parentKey = GraphTester.toSkyKey("parent");
634 tester.getOrCreate(parentKey).setHasError(true);
635 SkyKey childKey = GraphTester.toSkyKey("child");
636 set("child", "onions");
637 tester.getOrCreate(parentKey).addDependency(childKey).setComputedValue(CONCATENATE);
638 EvaluationResult<StringValue> result = eval(/*keepGoing=*/true, parentKey, childKey);
639 // Child is guaranteed to complete successfully before parent can run (and fail),
640 // since parent depends on it.
641 StringValue childValue = result.get(childKey);
642 Assert.assertNotNull(childValue);
643 assertEquals("onions", childValue.getValue());
644 ErrorInfo error = result.getError(parentKey);
645 Assert.assertNotNull(error);
646 assertThat(error.getRootCauses()).containsExactly(parentKey);
647 }
648
649 @Test
650 public void newParentOfErrorShouldHaveError() throws Exception {
651 graph = new InMemoryGraph();
652 SkyKey errorKey = GraphTester.toSkyKey("error");
653 tester.getOrCreate(errorKey).setHasError(true);
654 ErrorInfo error = evalValueInError(errorKey);
655 assertThat(error.getRootCauses()).containsExactly(errorKey);
656 SkyKey parentKey = GraphTester.toSkyKey("parent");
657 tester.getOrCreate(parentKey).addDependency("error").setComputedValue(CONCATENATE);
658 error = evalValueInError(parentKey);
659 assertThat(error.getRootCauses()).containsExactly(errorKey);
660 }
661
662 @Test
663 public void errorTwoLevelsDeep() throws Exception {
664 graph = new InMemoryGraph();
665 SkyKey parentKey = GraphTester.toSkyKey("parent");
666 SkyKey errorKey = GraphTester.toSkyKey("error");
667 tester.getOrCreate(errorKey).setHasError(true);
668 tester.getOrCreate("mid").addDependency(errorKey).setComputedValue(CONCATENATE);
669 tester.getOrCreate(parentKey).addDependency("mid").setComputedValue(CONCATENATE);
670 ErrorInfo error = evalValueInError(parentKey);
671 assertThat(error.getRootCauses()).containsExactly(errorKey);
672 }
673
674 /**
675 * A recreation of BuildViewTest#testHasErrorRaceCondition. Also similar to errorTwoLevelsDeep,
676 * except here we request multiple toplevel values.
677 */
678 @Test
679 public void errorPropagationToTopLevelValues() throws Exception {
680 graph = new InMemoryGraph();
681 SkyKey topKey = GraphTester.toSkyKey("top");
682 SkyKey midKey = GraphTester.toSkyKey("mid");
683 SkyKey badKey = GraphTester.toSkyKey("bad");
684 tester.getOrCreate(topKey).addDependency(midKey).setComputedValue(CONCATENATE);
685 tester.getOrCreate(midKey).addDependency(badKey).setComputedValue(CONCATENATE);
686 tester.getOrCreate(badKey).setHasError(true);
687 EvaluationResult<SkyValue> result = eval(/*keepGoing=*/false, topKey, midKey);
688 assertThat(result.getError(midKey).getRootCauses()).containsExactly(badKey);
689 // Do it again with keepGoing. We should also see an error for the top key this time.
690 result = eval(/*keepGoing=*/true, topKey, midKey);
691 assertThat(result.getError(midKey).getRootCauses()).containsExactly(badKey);
692 assertThat(result.getError(topKey).getRootCauses()).containsExactly(badKey);
693 }
694
695 @Test
696 public void valueNotUsedInFailFastErrorRecovery() throws Exception {
697 graph = new InMemoryGraph();
698 SkyKey topKey = GraphTester.toSkyKey("top");
699 SkyKey recoveryKey = GraphTester.toSkyKey("midRecovery");
700 SkyKey badKey = GraphTester.toSkyKey("bad");
701
702 tester.getOrCreate(topKey).addDependency(recoveryKey).setComputedValue(CONCATENATE);
703 tester.getOrCreate(recoveryKey).addErrorDependency(badKey, new StringValue("i recovered"))
704 .setComputedValue(CONCATENATE);
705 tester.getOrCreate(badKey).setHasError(true);
706
707 EvaluationResult<SkyValue> result = eval(/*keepGoing=*/true, ImmutableList.of(recoveryKey));
708 assertThat(result.errorMap()).isEmpty();
709 assertTrue(result.hasError());
710 assertEquals(new StringValue("i recovered"), result.get(recoveryKey));
711
712 result = eval(/*keepGoing=*/false, ImmutableList.of(topKey));
713 assertTrue(result.hasError());
714 assertThat(result.keyNames()).isEmpty();
715 assertEquals(1, result.errorMap().size());
716 assertNotNull(result.getError(topKey).getException());
717 }
718
719 /**
720 * Regression test: "clearing incomplete values on --keep_going build is racy".
721 * Tests that if a value is requested on the first (non-keep-going) build and its child throws
722 * an error, when the second (keep-going) build runs, there is not a race that keeps it as a
723 * reverse dep of its children.
724 */
725 @Test
726 public void raceClearingIncompleteValues() throws Exception {
727 SkyKey topKey = GraphTester.toSkyKey("top");
728 final SkyKey midKey = GraphTester.toSkyKey("mid");
729 SkyKey badKey = GraphTester.toSkyKey("bad");
730 final AtomicBoolean waitForSecondCall = new AtomicBoolean(false);
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100731 final CountDownLatch otherThreadWinning = new CountDownLatch(1);
732 final AtomicReference<Thread> firstThread = new AtomicReference<>();
Janak Ramakrishnana67bb8b2015-09-02 17:56:08 +0000733 graph =
734 new NotifyingInMemoryGraph(
735 new Listener() {
736 @Override
737 public void accept(SkyKey key, EventType type, Order order, Object context) {
738 if (!waitForSecondCall.get()) {
739 return;
740 }
741 if (key.equals(midKey)) {
742 if (type == EventType.CREATE_IF_ABSENT) {
743 // The first thread to create midKey will not be the first thread to add a
744 // reverse dep to it.
745 firstThread.compareAndSet(null, Thread.currentThread());
746 return;
747 }
748 if (type == EventType.ADD_REVERSE_DEP) {
749 if (order == Order.BEFORE && Thread.currentThread().equals(firstThread.get())) {
750 // If this thread created midKey, block until the other thread adds a dep on
751 // it.
752 TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
753 otherThreadWinning, "other thread didn't pass this one");
754 } else if (order == Order.AFTER
755 && !Thread.currentThread().equals(firstThread.get())) {
756 // This thread has added a dep. Allow the other thread to proceed.
757 otherThreadWinning.countDown();
758 }
759 }
760 }
761 }
762 });
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100763 tester.getOrCreate(topKey).addDependency(midKey).setComputedValue(CONCATENATE);
764 tester.getOrCreate(midKey).addDependency(badKey).setComputedValue(CONCATENATE);
765 tester.getOrCreate(badKey).setHasError(true);
766 EvaluationResult<SkyValue> result = eval(/*keepGoing=*/false, topKey, midKey);
767 assertThat(result.getError(midKey).getRootCauses()).containsExactly(badKey);
768 waitForSecondCall.set(true);
769 result = eval(/*keepGoing=*/true, topKey, midKey);
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +0100770 assertNotNull(firstThread.get());
771 assertEquals(0, otherThreadWinning.getCount());
772 assertThat(result.getError(midKey).getRootCauses()).containsExactly(badKey);
773 assertThat(result.getError(topKey).getRootCauses()).containsExactly(badKey);
774 }
775
776 @Test
777 public void multipleRootCauses() throws Exception {
778 graph = new InMemoryGraph();
779 SkyKey parentKey = GraphTester.toSkyKey("parent");
780 SkyKey errorKey = GraphTester.toSkyKey("error");
781 SkyKey errorKey2 = GraphTester.toSkyKey("error2");
782 SkyKey errorKey3 = GraphTester.toSkyKey("error3");
783 tester.getOrCreate(errorKey).setHasError(true);
784 tester.getOrCreate(errorKey2).setHasError(true);
785 tester.getOrCreate(errorKey3).setHasError(true);
786 tester.getOrCreate("mid").addDependency(errorKey).addDependency(errorKey2)
787 .setComputedValue(CONCATENATE);
788 tester.getOrCreate(parentKey)
789 .addDependency("mid").addDependency(errorKey2).addDependency(errorKey3)
790 .setComputedValue(CONCATENATE);
791 ErrorInfo error = evalValueInError(parentKey);
792 assertThat(error.getRootCauses()).containsExactly(errorKey, errorKey2, errorKey3);
793 }
794
795 @Test
796 public void rootCauseWithNoKeepGoing() throws Exception {
797 graph = new InMemoryGraph();
798 SkyKey parentKey = GraphTester.toSkyKey("parent");
799 SkyKey errorKey = GraphTester.toSkyKey("error");
800 tester.getOrCreate(errorKey).setHasError(true);
801 tester.getOrCreate("mid").addDependency(errorKey).setComputedValue(CONCATENATE);
802 tester.getOrCreate(parentKey).addDependency("mid").setComputedValue(CONCATENATE);
803 EvaluationResult<StringValue> result = eval(false, ImmutableList.of(parentKey));
804 Map.Entry<SkyKey, ErrorInfo> error = Iterables.getOnlyElement(result.errorMap().entrySet());
805 assertEquals(parentKey, error.getKey());
806 assertThat(error.getValue().getRootCauses()).containsExactly(errorKey);
807 }
808
809 @Test
810 public void errorBubblesToParentsOfTopLevelValue() throws Exception {
811 graph = new InMemoryGraph();
812 SkyKey parentKey = GraphTester.toSkyKey("parent");
813 final SkyKey errorKey = GraphTester.toSkyKey("error");
814 final CountDownLatch latch = new CountDownLatch(1);
815 tester.getOrCreate(errorKey).setBuilder(new ChainedFunction(null, /*waitToFinish=*/latch, null,
816 false, /*value=*/null, ImmutableList.<SkyKey>of()));
817 tester.getOrCreate(parentKey).setBuilder(new ChainedFunction(/*notifyStart=*/latch, null, null,
818 false, new StringValue("unused"), ImmutableList.of(errorKey)));
819 EvaluationResult<StringValue> result = eval( /*keepGoing=*/false,
820 ImmutableList.of(parentKey, errorKey));
821 assertEquals(result.toString(), 2, result.errorMap().size());
822 }
823
824 @Test
825 public void noKeepGoingAfterKeepGoingFails() throws Exception {
826 graph = new InMemoryGraph();
827 SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
828 tester.getOrCreate(errorKey).setHasError(true);
829 SkyKey parentKey = GraphTester.toSkyKey("parent");
830 tester.getOrCreate(parentKey).addDependency(errorKey);
831 ErrorInfo error = evalValueInError(parentKey);
832 assertThat(error.getRootCauses()).containsExactly(errorKey);
833 SkyKey[] list = { parentKey };
834 EvaluationResult<StringValue> result = eval(false, list);
835 ErrorInfo errorInfo = result.getError();
836 assertEquals(errorKey, Iterables.getOnlyElement(errorInfo.getRootCauses()));
837 assertEquals(errorKey.toString(), errorInfo.getException().getMessage());
838 }
839
840 @Test
841 public void twoErrors() throws Exception {
842 graph = new InMemoryGraph();
843 SkyKey firstError = GraphTester.toSkyKey("error1");
844 SkyKey secondError = GraphTester.toSkyKey("error2");
845 CountDownLatch firstStart = new CountDownLatch(1);
846 CountDownLatch secondStart = new CountDownLatch(1);
847 tester.getOrCreate(firstError).setBuilder(new ChainedFunction(firstStart, secondStart,
848 /*notifyFinish=*/null, /*waitForException=*/false, /*value=*/null,
849 ImmutableList.<SkyKey>of()));
850 tester.getOrCreate(secondError).setBuilder(new ChainedFunction(secondStart, firstStart,
851 /*notifyFinish=*/null, /*waitForException=*/false, /*value=*/null,
852 ImmutableList.<SkyKey>of()));
853 EvaluationResult<StringValue> result = eval(/*keepGoing=*/false, firstError, secondError);
854 assertTrue(result.toString(), result.hasError());
855 // With keepGoing=false, the eval call will terminate with exactly one error (the first one
856 // thrown). But the first one thrown here is non-deterministic since we synchronize the
857 // builders so that they run at roughly the same time.
858 assertThat(ImmutableSet.of(firstError, secondError)).contains(
859 Iterables.getOnlyElement(result.errorMap().keySet()));
860 }
861
862 @Test
863 public void simpleCycle() throws Exception {
864 graph = new InMemoryGraph();
865 SkyKey aKey = GraphTester.toSkyKey("a");
866 SkyKey bKey = GraphTester.toSkyKey("b");
867 tester.getOrCreate(aKey).addDependency(bKey);
868 tester.getOrCreate(bKey).addDependency(aKey);
869 ErrorInfo errorInfo = eval(false, ImmutableList.of(aKey)).getError();
870 assertEquals(null, errorInfo.getException());
871 CycleInfo cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
872 assertThat(cycleInfo.getCycle()).containsExactly(aKey, bKey).inOrder();
873 assertTrue(cycleInfo.getPathToCycle().isEmpty());
874 }
875
876 @Test
877 public void cycleWithHead() throws Exception {
878 graph = new InMemoryGraph();
879 SkyKey aKey = GraphTester.toSkyKey("a");
880 SkyKey bKey = GraphTester.toSkyKey("b");
881 SkyKey topKey = GraphTester.toSkyKey("top");
882 SkyKey midKey = GraphTester.toSkyKey("mid");
883 tester.getOrCreate(topKey).addDependency(midKey);
884 tester.getOrCreate(midKey).addDependency(aKey);
885 tester.getOrCreate(aKey).addDependency(bKey);
886 tester.getOrCreate(bKey).addDependency(aKey);
887 ErrorInfo errorInfo = eval(false, ImmutableList.of(topKey)).getError();
888 assertEquals(null, errorInfo.getException());
889 CycleInfo cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
890 assertThat(cycleInfo.getCycle()).containsExactly(aKey, bKey).inOrder();
891 assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey, midKey).inOrder();
892 }
893
894 @Test
895 public void selfEdgeWithHead() throws Exception {
896 graph = new InMemoryGraph();
897 SkyKey aKey = GraphTester.toSkyKey("a");
898 SkyKey topKey = GraphTester.toSkyKey("top");
899 SkyKey midKey = GraphTester.toSkyKey("mid");
900 tester.getOrCreate(topKey).addDependency(midKey);
901 tester.getOrCreate(midKey).addDependency(aKey);
902 tester.getOrCreate(aKey).addDependency(aKey);
903 ErrorInfo errorInfo = eval(false, ImmutableList.of(topKey)).getError();
904 assertEquals(null, errorInfo.getException());
905 CycleInfo cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
906 assertThat(cycleInfo.getCycle()).containsExactly(aKey).inOrder();
907 assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey, midKey).inOrder();
908 }
909
910 @Test
911 public void cycleWithKeepGoing() throws Exception {
912 graph = new InMemoryGraph();
913 SkyKey aKey = GraphTester.toSkyKey("a");
914 SkyKey bKey = GraphTester.toSkyKey("b");
915 SkyKey topKey = GraphTester.toSkyKey("top");
916 SkyKey midKey = GraphTester.toSkyKey("mid");
917 SkyKey goodKey = GraphTester.toSkyKey("good");
918 StringValue goodValue = new StringValue("good");
919 tester.set(goodKey, goodValue);
920 tester.getOrCreate(topKey).addDependency(midKey);
921 tester.getOrCreate(midKey).addDependency(aKey);
922 tester.getOrCreate(aKey).addDependency(bKey);
923 tester.getOrCreate(bKey).addDependency(aKey);
924 EvaluationResult<StringValue> result = eval(true, topKey, goodKey);
925 assertEquals(goodValue, result.get(goodKey));
926 assertEquals(null, result.get(topKey));
927 ErrorInfo errorInfo = result.getError(topKey);
928 CycleInfo cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
929 assertThat(cycleInfo.getCycle()).containsExactly(aKey, bKey).inOrder();
930 assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey, midKey).inOrder();
931 }
932
933 @Test
934 public void twoCycles() throws Exception {
935 graph = new InMemoryGraph();
936 SkyKey aKey = GraphTester.toSkyKey("a");
937 SkyKey bKey = GraphTester.toSkyKey("b");
938 SkyKey cKey = GraphTester.toSkyKey("c");
939 SkyKey dKey = GraphTester.toSkyKey("d");
940 SkyKey topKey = GraphTester.toSkyKey("top");
941 tester.getOrCreate(topKey).addDependency(aKey).addDependency(cKey);
942 tester.getOrCreate(aKey).addDependency(bKey);
943 tester.getOrCreate(bKey).addDependency(aKey);
944 tester.getOrCreate(cKey).addDependency(dKey);
945 tester.getOrCreate(dKey).addDependency(cKey);
946 EvaluationResult<StringValue> result = eval(false, ImmutableList.of(topKey));
947 assertEquals(null, result.get(topKey));
948 ErrorInfo errorInfo = result.getError(topKey);
949 Iterable<CycleInfo> cycles = CycleInfo.prepareCycles(topKey,
950 ImmutableList.of(new CycleInfo(ImmutableList.of(aKey, bKey)),
951 new CycleInfo(ImmutableList.of(cKey, dKey))));
952 assertThat(cycles).contains(getOnlyElement(errorInfo.getCycleInfo()));
953 }
954
955
956 @Test
957 public void twoCyclesKeepGoing() throws Exception {
958 graph = new InMemoryGraph();
959 SkyKey aKey = GraphTester.toSkyKey("a");
960 SkyKey bKey = GraphTester.toSkyKey("b");
961 SkyKey cKey = GraphTester.toSkyKey("c");
962 SkyKey dKey = GraphTester.toSkyKey("d");
963 SkyKey topKey = GraphTester.toSkyKey("top");
964 tester.getOrCreate(topKey).addDependency(aKey).addDependency(cKey);
965 tester.getOrCreate(aKey).addDependency(bKey);
966 tester.getOrCreate(bKey).addDependency(aKey);
967 tester.getOrCreate(cKey).addDependency(dKey);
968 tester.getOrCreate(dKey).addDependency(cKey);
969 EvaluationResult<StringValue> result = eval(true, ImmutableList.of(topKey));
970 assertEquals(null, result.get(topKey));
971 ErrorInfo errorInfo = result.getError(topKey);
972 CycleInfo aCycle = new CycleInfo(ImmutableList.of(topKey), ImmutableList.of(aKey, bKey));
973 CycleInfo cCycle = new CycleInfo(ImmutableList.of(topKey), ImmutableList.of(cKey, dKey));
974 assertThat(errorInfo.getCycleInfo()).containsExactly(aCycle, cCycle);
975 }
976
977 @Test
978 public void triangleBelowHeadCycle() throws Exception {
979 graph = new InMemoryGraph();
980 SkyKey aKey = GraphTester.toSkyKey("a");
981 SkyKey bKey = GraphTester.toSkyKey("b");
982 SkyKey cKey = GraphTester.toSkyKey("c");
983 SkyKey topKey = GraphTester.toSkyKey("top");
984 tester.getOrCreate(topKey).addDependency(aKey);
985 tester.getOrCreate(aKey).addDependency(bKey).addDependency(cKey);
986 tester.getOrCreate(bKey).addDependency(cKey);
987 tester.getOrCreate(cKey).addDependency(topKey);
988 EvaluationResult<StringValue> result = eval(true, ImmutableList.of(topKey));
989 assertEquals(null, result.get(topKey));
990 ErrorInfo errorInfo = result.getError(topKey);
991 CycleInfo topCycle = new CycleInfo(ImmutableList.of(topKey, aKey, cKey));
992 assertThat(errorInfo.getCycleInfo()).containsExactly(topCycle);
993 }
994
995 @Test
996 public void longCycle() throws Exception {
997 graph = new InMemoryGraph();
998 SkyKey aKey = GraphTester.toSkyKey("a");
999 SkyKey bKey = GraphTester.toSkyKey("b");
1000 SkyKey cKey = GraphTester.toSkyKey("c");
1001 SkyKey topKey = GraphTester.toSkyKey("top");
1002 tester.getOrCreate(topKey).addDependency(aKey);
1003 tester.getOrCreate(aKey).addDependency(bKey);
1004 tester.getOrCreate(bKey).addDependency(cKey);
1005 tester.getOrCreate(cKey).addDependency(topKey);
1006 EvaluationResult<StringValue> result = eval(true, ImmutableList.of(topKey));
1007 assertEquals(null, result.get(topKey));
1008 ErrorInfo errorInfo = result.getError(topKey);
1009 CycleInfo topCycle = new CycleInfo(ImmutableList.of(topKey, aKey, bKey, cKey));
1010 assertThat(errorInfo.getCycleInfo()).containsExactly(topCycle);
1011 }
1012
1013 @Test
1014 public void cycleWithTail() throws Exception {
1015 graph = new InMemoryGraph();
1016 SkyKey aKey = GraphTester.toSkyKey("a");
1017 SkyKey bKey = GraphTester.toSkyKey("b");
1018 SkyKey cKey = GraphTester.toSkyKey("c");
1019 SkyKey topKey = GraphTester.toSkyKey("top");
1020 tester.getOrCreate(topKey).addDependency(aKey).addDependency(cKey);
1021 tester.getOrCreate(aKey).addDependency(bKey);
1022 tester.getOrCreate(bKey).addDependency(aKey).addDependency(cKey);
1023 tester.getOrCreate(cKey);
1024 tester.set(cKey, new StringValue("cValue"));
1025 EvaluationResult<StringValue> result = eval(false, ImmutableList.of(topKey));
1026 assertEquals(null, result.get(topKey));
1027 ErrorInfo errorInfo = result.getError(topKey);
1028 CycleInfo cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
1029 assertThat(cycleInfo.getCycle()).containsExactly(aKey, bKey).inOrder();
1030 assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey).inOrder();
1031 }
1032
1033 /** Regression test: "value cannot be ready in a cycle". */
1034 @Test
1035 public void selfEdgeWithExtraChildrenUnderCycle() throws Exception {
1036 graph = new InMemoryGraph();
1037 SkyKey aKey = GraphTester.toSkyKey("a");
1038 SkyKey bKey = GraphTester.toSkyKey("b");
1039 SkyKey cKey = GraphTester.toSkyKey("c");
1040 tester.getOrCreate(aKey).addDependency(bKey);
1041 tester.getOrCreate(bKey).addDependency(cKey).addDependency(bKey);
1042 tester.getOrCreate(cKey).addDependency(aKey);
1043 EvaluationResult<StringValue> result = eval(/*keepGoing=*/true, ImmutableList.of(aKey));
1044 assertEquals(null, result.get(aKey));
1045 ErrorInfo errorInfo = result.getError(aKey);
1046 CycleInfo cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
1047 assertThat(cycleInfo.getCycle()).containsExactly(bKey).inOrder();
1048 assertThat(cycleInfo.getPathToCycle()).containsExactly(aKey).inOrder();
1049 }
1050
1051 /** Regression test: "value cannot be ready in a cycle". */
1052 @Test
1053 public void cycleWithExtraChildrenUnderCycle() throws Exception {
1054 graph = new InMemoryGraph();
1055 SkyKey aKey = GraphTester.toSkyKey("a");
1056 SkyKey bKey = GraphTester.toSkyKey("b");
1057 SkyKey cKey = GraphTester.toSkyKey("c");
1058 SkyKey dKey = GraphTester.toSkyKey("d");
1059 tester.getOrCreate(aKey).addDependency(bKey);
1060 tester.getOrCreate(bKey).addDependency(cKey).addDependency(dKey);
1061 tester.getOrCreate(cKey).addDependency(aKey);
1062 tester.getOrCreate(dKey).addDependency(bKey);
1063 EvaluationResult<StringValue> result = eval(/*keepGoing=*/true, ImmutableList.of(aKey));
1064 assertEquals(null, result.get(aKey));
1065 ErrorInfo errorInfo = result.getError(aKey);
1066 CycleInfo cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
1067 assertThat(cycleInfo.getCycle()).containsExactly(bKey, dKey).inOrder();
1068 assertThat(cycleInfo.getPathToCycle()).containsExactly(aKey).inOrder();
1069 }
1070
1071 /** Regression test: "value cannot be ready in a cycle". */
1072 @Test
1073 public void cycleAboveIndependentCycle() throws Exception {
1074 graph = new InMemoryGraph();
1075 SkyKey aKey = GraphTester.toSkyKey("a");
1076 SkyKey bKey = GraphTester.toSkyKey("b");
1077 SkyKey cKey = GraphTester.toSkyKey("c");
1078 tester.getOrCreate(aKey).addDependency(bKey);
1079 tester.getOrCreate(bKey).addDependency(cKey);
1080 tester.getOrCreate(cKey).addDependency(aKey).addDependency(bKey);
1081 EvaluationResult<StringValue> result = eval(/*keepGoing=*/true, ImmutableList.of(aKey));
1082 assertEquals(null, result.get(aKey));
1083 assertThat(result.getError(aKey).getCycleInfo()).containsExactly(
1084 new CycleInfo(ImmutableList.of(aKey, bKey, cKey)),
1085 new CycleInfo(ImmutableList.of(aKey), ImmutableList.of(bKey, cKey)));
1086 }
1087
1088 public void valueAboveCycleAndExceptionReportsException() throws Exception {
1089 graph = new InMemoryGraph();
1090 SkyKey aKey = GraphTester.toSkyKey("a");
1091 SkyKey errorKey = GraphTester.toSkyKey("error");
1092 SkyKey bKey = GraphTester.toSkyKey("b");
1093 tester.getOrCreate(aKey).addDependency(bKey).addDependency(errorKey);
1094 tester.getOrCreate(bKey).addDependency(bKey);
1095 tester.getOrCreate(errorKey).setHasError(true);
1096 EvaluationResult<StringValue> result = eval(/*keepGoing=*/true, ImmutableList.of(aKey));
1097 assertEquals(null, result.get(aKey));
1098 assertNotNull(result.getError(aKey).getException());
1099 CycleInfo cycleInfo = Iterables.getOnlyElement(result.getError(aKey).getCycleInfo());
1100 assertThat(cycleInfo.getCycle()).containsExactly(bKey).inOrder();
1101 assertThat(cycleInfo.getPathToCycle()).containsExactly(aKey).inOrder();
1102 }
1103
1104 @Test
1105 public void errorValueStored() throws Exception {
1106 graph = new InMemoryGraph();
1107 SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
1108 tester.getOrCreate(errorKey).setHasError(true);
1109 EvaluationResult<StringValue> result = eval(false, ImmutableList.of(errorKey));
1110 assertThat(result.keyNames()).isEmpty();
1111 assertThat(result.errorMap().keySet()).containsExactly(errorKey);
1112 ErrorInfo errorInfo = result.getError();
1113 assertThat(errorInfo.getRootCauses()).containsExactly(errorKey);
1114 // Update value. But builder won't rebuild it.
1115 tester.getOrCreate(errorKey).setHasError(false);
1116 tester.set(errorKey, new StringValue("no error?"));
1117 result = eval(false, ImmutableList.of(errorKey));
1118 assertThat(result.keyNames()).isEmpty();
1119 assertThat(result.errorMap().keySet()).containsExactly(errorKey);
1120 errorInfo = result.getError();
1121 assertThat(errorInfo.getRootCauses()).containsExactly(errorKey);
1122 }
1123
1124 /**
1125 * Regression test: "OOM in Skyframe cycle detection".
1126 * We only store the first 20 cycles found below any given root value.
1127 */
1128 @Test
1129 public void manyCycles() throws Exception {
1130 graph = new InMemoryGraph();
1131 SkyKey topKey = GraphTester.toSkyKey("top");
1132 for (int i = 0; i < 100; i++) {
1133 SkyKey dep = GraphTester.toSkyKey(Integer.toString(i));
1134 tester.getOrCreate(topKey).addDependency(dep);
1135 tester.getOrCreate(dep).addDependency(dep);
1136 }
1137 EvaluationResult<StringValue> result = eval(/*keepGoing=*/true, ImmutableList.of(topKey));
1138 assertEquals(null, result.get(topKey));
1139 assertManyCycles(result.getError(topKey), topKey, /*selfEdge=*/false);
1140 }
1141
1142 /**
1143 * Regression test: "OOM in Skyframe cycle detection".
1144 * We filter out multiple paths to a cycle that go through the same child value.
1145 */
1146 @Test
1147 public void manyPathsToCycle() throws Exception {
1148 graph = new InMemoryGraph();
1149 SkyKey topKey = GraphTester.toSkyKey("top");
1150 SkyKey midKey = GraphTester.toSkyKey("mid");
1151 SkyKey cycleKey = GraphTester.toSkyKey("cycle");
1152 tester.getOrCreate(topKey).addDependency(midKey);
1153 tester.getOrCreate(cycleKey).addDependency(cycleKey);
1154 for (int i = 0; i < 100; i++) {
1155 SkyKey dep = GraphTester.toSkyKey(Integer.toString(i));
1156 tester.getOrCreate(midKey).addDependency(dep);
1157 tester.getOrCreate(dep).addDependency(cycleKey);
1158 }
1159 EvaluationResult<StringValue> result = eval(/*keepGoing=*/true, ImmutableList.of(topKey));
1160 assertEquals(null, result.get(topKey));
1161 CycleInfo cycleInfo = Iterables.getOnlyElement(result.getError(topKey).getCycleInfo());
1162 assertEquals(1, cycleInfo.getCycle().size());
1163 assertEquals(3, cycleInfo.getPathToCycle().size());
1164 assertThat(cycleInfo.getPathToCycle().subList(0, 2)).containsExactly(topKey, midKey).inOrder();
1165 }
1166
1167 /**
1168 * Checks that errorInfo has many self-edge cycles, and that one of them is a self-edge of
1169 * topKey, if {@code selfEdge} is true.
1170 */
1171 private static void assertManyCycles(ErrorInfo errorInfo, SkyKey topKey, boolean selfEdge) {
Ulf Adams37cc83d2015-07-23 08:18:18 +00001172 assertGreaterThan(1, Iterables.size(errorInfo.getCycleInfo()));
1173 assertLessThan(50, Iterables.size(errorInfo.getCycleInfo()));
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01001174 boolean foundSelfEdge = false;
1175 for (CycleInfo cycle : errorInfo.getCycleInfo()) {
1176 assertEquals(1, cycle.getCycle().size()); // Self-edge.
1177 if (!Iterables.isEmpty(cycle.getPathToCycle())) {
1178 assertThat(cycle.getPathToCycle()).containsExactly(topKey).inOrder();
1179 } else {
1180 assertThat(cycle.getCycle()).containsExactly(topKey).inOrder();
1181 foundSelfEdge = true;
1182 }
1183 }
1184 assertEquals(errorInfo + ", " + topKey, selfEdge, foundSelfEdge);
1185 }
1186
1187 @Test
1188 public void manyUnprocessedValuesInCycle() throws Exception {
1189 graph = new InMemoryGraph();
1190 SkyKey lastSelfKey = GraphTester.toSkyKey("lastSelf");
1191 SkyKey firstSelfKey = GraphTester.toSkyKey("firstSelf");
1192 SkyKey midSelfKey = GraphTester.toSkyKey("midSelf");
1193 // We add firstSelf first so that it is processed last in cycle detection (LIFO), meaning that
1194 // none of the dep values have to be cleared from firstSelf.
1195 tester.getOrCreate(firstSelfKey).addDependency(firstSelfKey);
1196 for (int i = 0; i < 100; i++) {
1197 SkyKey firstDep = GraphTester.toSkyKey("first" + i);
1198 SkyKey midDep = GraphTester.toSkyKey("mid" + i);
1199 SkyKey lastDep = GraphTester.toSkyKey("last" + i);
1200 tester.getOrCreate(firstSelfKey).addDependency(firstDep);
1201 tester.getOrCreate(midSelfKey).addDependency(midDep);
1202 tester.getOrCreate(lastSelfKey).addDependency(lastDep);
1203 if (i == 90) {
1204 // Most of the deps will be cleared from midSelf.
1205 tester.getOrCreate(midSelfKey).addDependency(midSelfKey);
1206 }
1207 tester.getOrCreate(firstDep).addDependency(firstDep);
1208 tester.getOrCreate(midDep).addDependency(midDep);
1209 tester.getOrCreate(lastDep).addDependency(lastDep);
1210 }
1211 // All the deps will be cleared from lastSelf.
1212 tester.getOrCreate(lastSelfKey).addDependency(lastSelfKey);
1213 EvaluationResult<StringValue> result = eval(/*keepGoing=*/true,
1214 ImmutableList.of(lastSelfKey, firstSelfKey, midSelfKey));
1215 assertWithMessage(result.toString()).that(result.keyNames()).isEmpty();
1216 assertThat(result.errorMap().keySet()).containsExactly(lastSelfKey, firstSelfKey, midSelfKey);
1217
1218 // Check lastSelfKey.
1219 ErrorInfo errorInfo = result.getError(lastSelfKey);
1220 assertEquals(errorInfo.toString(), 1, Iterables.size(errorInfo.getCycleInfo()));
1221 CycleInfo cycleInfo = Iterables.getOnlyElement(errorInfo.getCycleInfo());
1222 assertThat(cycleInfo.getCycle()).containsExactly(lastSelfKey);
1223 assertThat(cycleInfo.getPathToCycle()).isEmpty();
1224
1225 // Check firstSelfKey. It should not have discovered its own self-edge, because there were too
1226 // many other values before it in the queue.
1227 assertManyCycles(result.getError(firstSelfKey), firstSelfKey, /*selfEdge=*/false);
1228
1229 // Check midSelfKey. It should have discovered its own self-edge.
1230 assertManyCycles(result.getError(midSelfKey), midSelfKey, /*selfEdge=*/true);
1231 }
1232
1233 @Test
1234 public void errorValueStoredWithKeepGoing() throws Exception {
1235 graph = new InMemoryGraph();
1236 SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
1237 tester.getOrCreate(errorKey).setHasError(true);
1238 EvaluationResult<StringValue> result = eval(true, ImmutableList.of(errorKey));
1239 assertThat(result.keyNames()).isEmpty();
1240 assertThat(result.errorMap().keySet()).containsExactly(errorKey);
1241 ErrorInfo errorInfo = result.getError();
1242 assertThat(errorInfo.getRootCauses()).containsExactly(errorKey);
1243 // Update value. But builder won't rebuild it.
1244 tester.getOrCreate(errorKey).setHasError(false);
1245 tester.set(errorKey, new StringValue("no error?"));
1246 result = eval(true, ImmutableList.of(errorKey));
1247 assertThat(result.keyNames()).isEmpty();
1248 assertThat(result.errorMap().keySet()).containsExactly(errorKey);
1249 errorInfo = result.getError();
1250 assertThat(errorInfo.getRootCauses()).containsExactly(errorKey);
1251 }
1252
1253 @Test
1254 public void continueWithErrorDep() throws Exception {
1255 graph = new InMemoryGraph();
1256 SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
1257 tester.getOrCreate(errorKey).setHasError(true);
1258 tester.set("after", new StringValue("after"));
1259 SkyKey parentKey = GraphTester.toSkyKey("parent");
1260 tester.getOrCreate(parentKey).addErrorDependency(errorKey, new StringValue("recovered"))
1261 .setComputedValue(CONCATENATE).addDependency("after");
1262 EvaluationResult<StringValue> result = eval(/*keepGoing=*/true, ImmutableList.of(parentKey));
1263 assertThat(result.errorMap()).isEmpty();
1264 assertEquals("recoveredafter", result.get(parentKey).getValue());
1265 result = eval(/*keepGoing=*/false, ImmutableList.of(parentKey));
1266 assertThat(result.keyNames()).isEmpty();
1267 Map.Entry<SkyKey, ErrorInfo> error = Iterables.getOnlyElement(result.errorMap().entrySet());
1268 assertEquals(parentKey, error.getKey());
1269 assertThat(error.getValue().getRootCauses()).containsExactly(errorKey);
1270 }
1271
1272 @Test
1273 public void breakWithErrorDep() throws Exception {
1274 graph = new InMemoryGraph();
1275 SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
1276 tester.getOrCreate(errorKey).setHasError(true);
1277 tester.set("after", new StringValue("after"));
1278 SkyKey parentKey = GraphTester.toSkyKey("parent");
1279 tester.getOrCreate(parentKey).addErrorDependency(errorKey, new StringValue("recovered"))
1280 .setComputedValue(CONCATENATE).addDependency("after");
1281 EvaluationResult<StringValue> result = eval(/*keepGoing=*/false, ImmutableList.of(parentKey));
1282 assertThat(result.keyNames()).isEmpty();
1283 Map.Entry<SkyKey, ErrorInfo> error = Iterables.getOnlyElement(result.errorMap().entrySet());
1284 assertEquals(parentKey, error.getKey());
1285 assertThat(error.getValue().getRootCauses()).containsExactly(errorKey);
1286 result = eval(/*keepGoing=*/true, ImmutableList.of(parentKey));
1287 assertThat(result.errorMap()).isEmpty();
1288 assertEquals("recoveredafter", result.get(parentKey).getValue());
1289 }
1290
1291 @Test
1292 public void breakWithInterruptibleErrorDep() throws Exception {
1293 graph = new InMemoryGraph();
1294 SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
1295 tester.getOrCreate(errorKey).setHasError(true);
1296 SkyKey parentKey = GraphTester.toSkyKey("parent");
1297 tester.getOrCreate(parentKey).addErrorDependency(errorKey, new StringValue("recovered"))
1298 .setComputedValue(CONCATENATE);
1299 // When the error value throws, the propagation will cause an interrupted exception in parent.
1300 EvaluationResult<StringValue> result = eval(/*keepGoing=*/false, ImmutableList.of(parentKey));
1301 assertThat(result.keyNames()).isEmpty();
1302 Map.Entry<SkyKey, ErrorInfo> error = Iterables.getOnlyElement(result.errorMap().entrySet());
1303 assertEquals(parentKey, error.getKey());
1304 assertThat(error.getValue().getRootCauses()).containsExactly(errorKey);
1305 assertFalse(Thread.interrupted());
1306 result = eval(/*keepGoing=*/true, ImmutableList.of(parentKey));
1307 assertThat(result.errorMap()).isEmpty();
1308 assertEquals("recovered", result.get(parentKey).getValue());
1309 }
1310
1311 @Test
1312 public void transformErrorDep() throws Exception {
1313 graph = new InMemoryGraph();
1314 SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
1315 tester.getOrCreate(errorKey).setHasError(true);
1316 SkyKey parentErrorKey = GraphTester.toSkyKey("parent");
1317 tester.getOrCreate(parentErrorKey).addErrorDependency(errorKey, new StringValue("recovered"))
1318 .setHasError(true);
1319 EvaluationResult<StringValue> result = eval(
1320 /*keepGoing=*/false, ImmutableList.of(parentErrorKey));
1321 assertThat(result.keyNames()).isEmpty();
1322 Map.Entry<SkyKey, ErrorInfo> error = Iterables.getOnlyElement(result.errorMap().entrySet());
1323 assertEquals(parentErrorKey, error.getKey());
1324 assertThat(error.getValue().getRootCauses()).containsExactly(parentErrorKey);
1325 }
1326
1327 @Test
1328 public void transformErrorDepKeepGoing() throws Exception {
1329 graph = new InMemoryGraph();
1330 SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
1331 tester.getOrCreate(errorKey).setHasError(true);
1332 SkyKey parentErrorKey = GraphTester.toSkyKey("parent");
1333 tester.getOrCreate(parentErrorKey).addErrorDependency(errorKey, new StringValue("recovered"))
1334 .setHasError(true);
1335 EvaluationResult<StringValue> result = eval(
1336 /*keepGoing=*/true, ImmutableList.of(parentErrorKey));
1337 assertThat(result.keyNames()).isEmpty();
1338 Map.Entry<SkyKey, ErrorInfo> error = Iterables.getOnlyElement(result.errorMap().entrySet());
1339 assertEquals(parentErrorKey, error.getKey());
1340 assertThat(error.getValue().getRootCauses()).containsExactly(parentErrorKey);
1341 }
1342
1343 @Test
1344 public void transformErrorDepOneLevelDownKeepGoing() throws Exception {
1345 graph = new InMemoryGraph();
1346 SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
1347 tester.getOrCreate(errorKey).setHasError(true);
1348 tester.set("after", new StringValue("after"));
1349 SkyKey parentErrorKey = GraphTester.toSkyKey("parent");
1350 tester.getOrCreate(parentErrorKey).addErrorDependency(errorKey, new StringValue("recovered"));
1351 tester.set(parentErrorKey, new StringValue("parent value"));
1352 SkyKey topKey = GraphTester.toSkyKey("top");
1353 tester.getOrCreate(topKey).addDependency(parentErrorKey).addDependency("after")
1354 .setComputedValue(CONCATENATE);
1355 EvaluationResult<StringValue> result = eval(/*keepGoing=*/true, ImmutableList.of(topKey));
1356 assertThat(ImmutableList.<String>copyOf(result.<String>keyNames())).containsExactly("top");
1357 assertEquals("parent valueafter", result.get(topKey).getValue());
1358 assertThat(result.errorMap()).isEmpty();
1359 }
1360
1361 @Test
1362 public void transformErrorDepOneLevelDownNoKeepGoing() throws Exception {
1363 graph = new InMemoryGraph();
1364 SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
1365 tester.getOrCreate(errorKey).setHasError(true);
1366 tester.set("after", new StringValue("after"));
1367 SkyKey parentErrorKey = GraphTester.toSkyKey("parent");
1368 tester.getOrCreate(parentErrorKey).addErrorDependency(errorKey, new StringValue("recovered"));
1369 tester.set(parentErrorKey, new StringValue("parent value"));
1370 SkyKey topKey = GraphTester.toSkyKey("top");
1371 tester.getOrCreate(topKey).addDependency(parentErrorKey).addDependency("after")
1372 .setComputedValue(CONCATENATE);
1373 EvaluationResult<StringValue> result = eval(/*keepGoing=*/false, ImmutableList.of(topKey));
1374 assertThat(result.keyNames()).isEmpty();
1375 Map.Entry<SkyKey, ErrorInfo> error = Iterables.getOnlyElement(result.errorMap().entrySet());
1376 assertEquals(topKey, error.getKey());
1377 assertThat(error.getValue().getRootCauses()).containsExactly(errorKey);
1378 }
1379
1380 /**
1381 * Make sure that multiple unfinished children can be cleared from a cycle value.
1382 */
1383 @Test
1384 public void cycleWithMultipleUnfinishedChildren() throws Exception {
1385 graph = new InMemoryGraph();
1386 tester = new GraphTester();
1387 SkyKey cycleKey = GraphTester.toSkyKey("cycle");
1388 SkyKey midKey = GraphTester.toSkyKey("mid");
1389 SkyKey topKey = GraphTester.toSkyKey("top");
1390 SkyKey selfEdge1 = GraphTester.toSkyKey("selfEdge1");
1391 SkyKey selfEdge2 = GraphTester.toSkyKey("selfEdge2");
1392 tester.getOrCreate(topKey).addDependency(midKey).setComputedValue(CONCATENATE);
1393 // selfEdge* come before cycleKey, so cycleKey's path will be checked first (LIFO), and the
1394 // cycle with mid will be detected before the selfEdge* cycles are.
1395 tester.getOrCreate(midKey).addDependency(selfEdge1).addDependency(selfEdge2)
1396 .addDependency(cycleKey)
1397 .setComputedValue(CONCATENATE);
1398 tester.getOrCreate(cycleKey).addDependency(midKey);
1399 tester.getOrCreate(selfEdge1).addDependency(selfEdge1);
1400 tester.getOrCreate(selfEdge2).addDependency(selfEdge2);
1401 EvaluationResult<StringValue> result = eval(/*keepGoing=*/true, ImmutableSet.of(topKey));
1402 assertThat(result.errorMap().keySet()).containsExactly(topKey);
1403 Iterable<CycleInfo> cycleInfos = result.getError(topKey).getCycleInfo();
1404 CycleInfo cycleInfo = Iterables.getOnlyElement(cycleInfos);
1405 assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey);
1406 assertThat(cycleInfo.getCycle()).containsExactly(midKey, cycleKey);
1407 }
1408
1409 /**
1410 * Regression test: "value in cycle depends on error".
1411 * The mid value will have two parents -- top and cycle. Error bubbles up from mid to cycle, and
1412 * we should detect cycle.
1413 */
1414 private void cycleAndErrorInBubbleUp(boolean keepGoing) throws Exception {
1415 graph = new DeterministicInMemoryGraph();
1416 tester = new GraphTester();
1417 SkyKey errorKey = GraphTester.toSkyKey("error");
1418 SkyKey cycleKey = GraphTester.toSkyKey("cycle");
1419 SkyKey midKey = GraphTester.toSkyKey("mid");
1420 SkyKey topKey = GraphTester.toSkyKey("top");
1421 tester.getOrCreate(topKey).addDependency(midKey).setComputedValue(CONCATENATE);
1422 tester.getOrCreate(midKey).addDependency(errorKey).addDependency(cycleKey)
1423 .setComputedValue(CONCATENATE);
1424
1425 // We need to ensure that cycle value has finished his work, and we have recorded dependencies
1426 CountDownLatch cycleFinish = new CountDownLatch(1);
1427 tester.getOrCreate(cycleKey).setBuilder(new ChainedFunction(null,
1428 null, cycleFinish, false, new StringValue(""), ImmutableSet.<SkyKey>of(midKey)));
1429 tester.getOrCreate(errorKey).setBuilder(new ChainedFunction(null, cycleFinish,
1430 null, /*waitForException=*/false, null, ImmutableSet.<SkyKey>of()));
1431
1432 EvaluationResult<StringValue> result = eval(keepGoing, ImmutableSet.of(topKey));
1433 assertThat(result.errorMap().keySet()).containsExactly(topKey);
1434 Iterable<CycleInfo> cycleInfos = result.getError(topKey).getCycleInfo();
1435 if (keepGoing) {
1436 // The error thrown will only be recorded in keep_going mode.
1437 assertThat(result.getError().getRootCauses()).containsExactly(errorKey);
1438 }
1439 assertThat(cycleInfos).isNotEmpty();
1440 CycleInfo cycleInfo = Iterables.getOnlyElement(cycleInfos);
1441 assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey);
1442 assertThat(cycleInfo.getCycle()).containsExactly(midKey, cycleKey);
1443 }
1444
1445 @Test
1446 public void cycleAndErrorInBubbleUpNoKeepGoing() throws Exception {
1447 cycleAndErrorInBubbleUp(false);
1448 }
1449
1450 @Test
1451 public void cycleAndErrorInBubbleUpKeepGoing() throws Exception {
1452 cycleAndErrorInBubbleUp(true);
1453 }
1454
1455 /**
1456 * Regression test: "value in cycle depends on error".
1457 * We add another value that won't finish building before the threadpool shuts down, to check that
1458 * the cycle detection can handle unfinished values.
1459 */
1460 @Test
1461 public void cycleAndErrorAndOtherInBubbleUp() throws Exception {
1462 graph = new DeterministicInMemoryGraph();
1463 tester = new GraphTester();
1464 SkyKey errorKey = GraphTester.toSkyKey("error");
1465 SkyKey cycleKey = GraphTester.toSkyKey("cycle");
1466 SkyKey midKey = GraphTester.toSkyKey("mid");
1467 SkyKey topKey = GraphTester.toSkyKey("top");
1468 tester.getOrCreate(topKey).addDependency(midKey).setComputedValue(CONCATENATE);
1469 // We should add cycleKey first and errorKey afterwards. Otherwise there is a chance that
1470 // during error propagation cycleKey will not be processed, and we will not detect the cycle.
1471 tester.getOrCreate(midKey).addDependency(errorKey).addDependency(cycleKey)
1472 .setComputedValue(CONCATENATE);
1473 SkyKey otherTop = GraphTester.toSkyKey("otherTop");
1474 CountDownLatch topStartAndCycleFinish = new CountDownLatch(2);
1475 // In nokeep_going mode, otherTop will wait until the threadpool has received an exception,
1476 // then request its own dep. This guarantees that there is a value that is not finished when
1477 // cycle detection happens.
1478 tester.getOrCreate(otherTop).setBuilder(new ChainedFunction(topStartAndCycleFinish,
1479 new CountDownLatch(0), null, /*waitForException=*/true, new StringValue("never returned"),
1480 ImmutableSet.<SkyKey>of(GraphTester.toSkyKey("dep that never builds"))));
1481
1482 tester.getOrCreate(cycleKey).setBuilder(new ChainedFunction(null, null,
1483 topStartAndCycleFinish, /*waitForException=*/false, new StringValue(""),
1484 ImmutableSet.<SkyKey>of(midKey)));
1485 // error waits until otherTop starts and cycle finishes, to make sure otherTop will request
1486 // its dep before the threadpool shuts down.
1487 tester.getOrCreate(errorKey).setBuilder(new ChainedFunction(null, topStartAndCycleFinish,
1488 null, /*waitForException=*/false, null,
1489 ImmutableSet.<SkyKey>of()));
1490 EvaluationResult<StringValue> result =
1491 eval(/*keepGoing=*/false, ImmutableSet.of(topKey, otherTop));
1492 assertThat(result.errorMap().keySet()).containsExactly(topKey);
1493 Iterable<CycleInfo> cycleInfos = result.getError(topKey).getCycleInfo();
1494 assertThat(cycleInfos).isNotEmpty();
1495 CycleInfo cycleInfo = Iterables.getOnlyElement(cycleInfos);
1496 assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey);
1497 assertThat(cycleInfo.getCycle()).containsExactly(midKey, cycleKey);
1498 }
1499
1500 /**
1501 * Regression test: "value in cycle depends on error".
1502 * Here, we add an additional top-level key in error, just to mix it up.
1503 */
1504 private void cycleAndErrorAndError(boolean keepGoing) throws Exception {
1505 graph = new DeterministicInMemoryGraph();
1506 tester = new GraphTester();
1507 SkyKey errorKey = GraphTester.toSkyKey("error");
1508 SkyKey cycleKey = GraphTester.toSkyKey("cycle");
1509 SkyKey midKey = GraphTester.toSkyKey("mid");
1510 SkyKey topKey = GraphTester.toSkyKey("top");
1511 tester.getOrCreate(topKey).addDependency(midKey).setComputedValue(CONCATENATE);
1512 tester.getOrCreate(midKey).addDependency(errorKey).addDependency(cycleKey)
1513 .setComputedValue(CONCATENATE);
1514 SkyKey otherTop = GraphTester.toSkyKey("otherTop");
1515 CountDownLatch topStartAndCycleFinish = new CountDownLatch(2);
1516 // In nokeep_going mode, otherTop will wait until the threadpool has received an exception,
1517 // then throw its own exception. This guarantees that its exception will not be the one
1518 // bubbling up, but that there is a top-level value with an exception by the time the bubbling
1519 // up starts.
1520 tester.getOrCreate(otherTop).setBuilder(new ChainedFunction(topStartAndCycleFinish,
1521 new CountDownLatch(0), null, /*waitForException=*/!keepGoing, null,
1522 ImmutableSet.<SkyKey>of()));
1523 // error waits until otherTop starts and cycle finishes, to make sure otherTop will request
1524 // its dep before the threadpool shuts down.
1525 tester.getOrCreate(errorKey).setBuilder(new ChainedFunction(null, topStartAndCycleFinish,
1526 null, /*waitForException=*/false, null,
1527 ImmutableSet.<SkyKey>of()));
1528 tester.getOrCreate(cycleKey).setBuilder(new ChainedFunction(null, null,
1529 topStartAndCycleFinish, /*waitForException=*/false, new StringValue(""),
1530 ImmutableSet.<SkyKey>of(midKey)));
1531 EvaluationResult<StringValue> result =
1532 eval(keepGoing, ImmutableSet.of(topKey, otherTop));
1533 if (keepGoing) {
1534 assertThat(result.errorMap().keySet()).containsExactly(otherTop, topKey);
1535 assertThat(result.getError(otherTop).getRootCauses()).containsExactly(otherTop);
1536 // The error thrown will only be recorded in keep_going mode.
1537 assertThat(result.getError(topKey).getRootCauses()).containsExactly(errorKey);
1538 }
1539 Iterable<CycleInfo> cycleInfos = result.getError(topKey).getCycleInfo();
1540 assertThat(cycleInfos).isNotEmpty();
1541 CycleInfo cycleInfo = Iterables.getOnlyElement(cycleInfos);
1542 assertThat(cycleInfo.getPathToCycle()).containsExactly(topKey);
1543 assertThat(cycleInfo.getCycle()).containsExactly(midKey, cycleKey);
1544 }
1545
1546 @Test
1547 public void cycleAndErrorAndErrorNoKeepGoing() throws Exception {
1548 cycleAndErrorAndError(false);
1549 }
1550
1551 @Test
1552 public void cycleAndErrorAndErrorKeepGoing() throws Exception {
1553 cycleAndErrorAndError(true);
1554 }
1555
1556 @Test
1557 public void testFunctionCrashTrace() throws Exception {
Michajlo Matijkiw02830402015-06-25 22:00:25 +00001558 final SkyFunctionName childType = SkyFunctionName.create("child");
1559 final SkyFunctionName parentType = SkyFunctionName.create("parent");
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01001560
1561 class ChildFunction implements SkyFunction {
1562 @Override
1563 public SkyValue compute(SkyKey skyKey, Environment env) {
1564 throw new IllegalStateException("I WANT A PONY!!!");
1565 }
1566
1567 @Override public String extractTag(SkyKey skyKey) { return null; }
1568 }
1569
1570 class ParentFunction implements SkyFunction {
1571 @Override
1572 public SkyValue compute(SkyKey skyKey, Environment env) {
1573 SkyValue dep = env.getValue(new SkyKey(childType, "billy the kid"));
1574 if (dep == null) {
1575 return null;
1576 }
1577 throw new IllegalStateException(); // Should never get here.
1578 }
1579
1580 @Override public String extractTag(SkyKey skyKey) { return null; }
1581 }
1582
1583 ImmutableMap<SkyFunctionName, SkyFunction> skyFunctions = ImmutableMap.of(
1584 childType, new ChildFunction(),
1585 parentType, new ParentFunction());
1586 ParallelEvaluator evaluator = makeEvaluator(new InMemoryGraph(),
1587 skyFunctions, false);
1588
1589 try {
1590 evaluator.eval(ImmutableList.of(new SkyKey(parentType, "octodad")));
1591 fail();
1592 } catch (RuntimeException e) {
1593 assertEquals("I WANT A PONY!!!", e.getCause().getMessage());
1594 assertEquals("Unrecoverable error while evaluating node 'child:billy the kid' "
1595 + "(requested by nodes 'parent:octodad')", e.getMessage());
1596 }
1597 }
1598
1599 private static class SomeOtherErrorException extends Exception {
1600 public SomeOtherErrorException(String msg) {
1601 super(msg);
1602 }
1603 }
1604
1605 private void unexpectedErrorDep(boolean keepGoing) throws Exception {
1606 graph = new InMemoryGraph();
1607 SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
1608 final SomeOtherErrorException exception = new SomeOtherErrorException("error exception");
1609 tester.getOrCreate(errorKey).setBuilder(new SkyFunction() {
1610 @Override
1611 public SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException {
1612 throw new SkyFunctionException(exception, Transience.PERSISTENT) {};
1613 }
1614
1615 @Override
1616 public String extractTag(SkyKey skyKey) {
1617 throw new UnsupportedOperationException();
1618 }
1619 });
1620 SkyKey topKey = GraphTester.toSkyKey("top");
1621 tester.getOrCreate(topKey).addErrorDependency(errorKey, new StringValue("recovered"))
1622 .setComputedValue(CONCATENATE);
1623 EvaluationResult<StringValue> result = eval(keepGoing, ImmutableList.of(topKey));
1624 assertThat(result.keyNames()).isEmpty();
1625 assertSame(exception, result.getError(topKey).getException());
1626 assertThat(result.getError(topKey).getRootCauses()).containsExactly(errorKey);
1627 }
1628
1629 /**
1630 * This and the following three tests are in response a bug: "Skyframe error propagation model is
1631 * problematic". They ensure that exceptions a child throws that a value does not specify it can
1632 * handle in getValueOrThrow do not cause a crash.
1633 */
1634 @Test
1635 public void unexpectedErrorDepKeepGoing() throws Exception {
1636 unexpectedErrorDep(true);
1637 }
1638
1639 @Test
1640 public void unexpectedErrorDepNoKeepGoing() throws Exception {
1641 unexpectedErrorDep(false);
1642 }
1643
1644 private void unexpectedErrorDepOneLevelDown(final boolean keepGoing) throws Exception {
1645 graph = new InMemoryGraph();
1646 SkyKey errorKey = GraphTester.toSkyKey("my_error_value");
1647 final SomeErrorException exception = new SomeErrorException("error exception");
1648 final SomeErrorException topException = new SomeErrorException("top exception");
1649 final StringValue topValue = new StringValue("top");
1650 tester.getOrCreate(errorKey).setBuilder(new SkyFunction() {
1651 @Override
1652 public SkyValue compute(SkyKey skyKey, Environment env) throws GenericFunctionException {
1653 throw new GenericFunctionException(exception, Transience.PERSISTENT);
1654 }
1655
1656 @Override
1657 public String extractTag(SkyKey skyKey) {
1658 throw new UnsupportedOperationException();
1659 }
1660 });
1661 SkyKey topKey = GraphTester.toSkyKey("top");
1662 final SkyKey parentKey = GraphTester.toSkyKey("parent");
1663 tester.getOrCreate(parentKey).addDependency(errorKey).setComputedValue(CONCATENATE);
1664 tester.getOrCreate(topKey).setBuilder(new SkyFunction() {
1665 @Override
1666 public SkyValue compute(SkyKey skyKey, Environment env) throws GenericFunctionException {
1667 try {
1668 if (env.getValueOrThrow(parentKey, SomeErrorException.class) == null) {
1669 return null;
1670 }
1671 } catch (SomeErrorException e) {
1672 assertEquals(e.toString(), exception, e);
1673 }
1674 if (keepGoing) {
1675 return topValue;
1676 } else {
1677 throw new GenericFunctionException(topException, Transience.PERSISTENT);
1678 }
1679 }
1680 @Override
1681 public String extractTag(SkyKey skyKey) {
1682 throw new UnsupportedOperationException();
1683 }
1684 });
1685 tester.getOrCreate(topKey).addErrorDependency(errorKey, new StringValue("recovered"))
1686 .setComputedValue(CONCATENATE);
1687 EvaluationResult<StringValue> result = eval(keepGoing, ImmutableList.of(topKey));
1688 if (!keepGoing) {
1689 assertThat(result.keyNames()).isEmpty();
1690 assertEquals(topException, result.getError(topKey).getException());
1691 assertThat(result.getError(topKey).getRootCauses()).containsExactly(topKey);
1692 assertTrue(result.hasError());
1693 } else {
1694 // result.hasError() is set to true even if the top-level value returned has recovered from
1695 // an error.
1696 assertTrue(result.hasError());
1697 assertSame(topValue, result.get(topKey));
1698 }
1699 }
1700
1701 @Test
1702 public void unexpectedErrorDepOneLevelDownKeepGoing() throws Exception {
1703 unexpectedErrorDepOneLevelDown(true);
1704 }
1705
1706 @Test
1707 public void unexpectedErrorDepOneLevelDownNoKeepGoing() throws Exception {
1708 unexpectedErrorDepOneLevelDown(false);
1709 }
1710
1711 /**
1712 * Exercises various situations involving groups of deps that overlap -- request one group, then
1713 * request another group that has a dep in common with the first group.
1714 *
1715 * @param sameFirst whether the dep in common in the two groups should be the first dep.
1716 * @param twoCalls whether the two groups should be requested in two different builder calls.
1717 * @param valuesOrThrow whether the deps should be requested using getValuesOrThrow.
1718 */
1719 private void sameDepInTwoGroups(final boolean sameFirst, final boolean twoCalls,
1720 final boolean valuesOrThrow) throws Exception {
1721 graph = new InMemoryGraph();
1722 SkyKey topKey = GraphTester.toSkyKey("top");
1723 final List<SkyKey> leaves = new ArrayList<>();
1724 for (int i = 1; i <= 3; i++) {
1725 SkyKey leaf = GraphTester.toSkyKey("leaf" + i);
1726 leaves.add(leaf);
1727 tester.set(leaf, new StringValue("leaf" + i));
1728 }
1729 final SkyKey leaf4 = GraphTester.toSkyKey("leaf4");
1730 tester.set(leaf4, new StringValue("leaf" + 4));
1731 tester.getOrCreate(topKey).setBuilder(new SkyFunction() {
1732 @Override
1733 public SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException,
1734 InterruptedException {
1735 if (valuesOrThrow) {
1736 env.getValuesOrThrow(leaves, SomeErrorException.class);
1737 } else {
1738 env.getValues(leaves);
1739 }
1740 if (twoCalls && env.valuesMissing()) {
1741 return null;
1742 }
1743 SkyKey first = sameFirst ? leaves.get(0) : leaf4;
1744 SkyKey second = sameFirst ? leaf4 : leaves.get(2);
1745 List<SkyKey> secondRequest = ImmutableList.of(first, second);
1746 if (valuesOrThrow) {
1747 env.getValuesOrThrow(secondRequest, SomeErrorException.class);
1748 } else {
1749 env.getValues(secondRequest);
1750 }
1751 if (env.valuesMissing()) {
1752 return null;
1753 }
1754 return new StringValue("top");
1755 }
1756
1757 @Override
1758 public String extractTag(SkyKey skyKey) {
1759 return null;
1760 }
1761 });
1762 eval(/*keepGoing=*/false, topKey);
1763 assertEquals(new StringValue("top"), eval(/*keepGoing=*/false, topKey));
1764 }
1765
1766 @Test
1767 public void sameDepInTwoGroups_Same_Two_Throw() throws Exception {
1768 sameDepInTwoGroups(/*sameFirst=*/true, /*twoCalls=*/true, /*valuesOrThrow=*/true);
1769 }
1770
1771 @Test
1772 public void sameDepInTwoGroups_Same_Two_Deps() throws Exception {
1773 sameDepInTwoGroups(/*sameFirst=*/true, /*twoCalls=*/true, /*valuesOrThrow=*/false);
1774 }
1775
1776 @Test
1777 public void sameDepInTwoGroups_Same_One_Throw() throws Exception {
1778 sameDepInTwoGroups(/*sameFirst=*/true, /*twoCalls=*/false, /*valuesOrThrow=*/true);
1779 }
1780
1781 @Test
1782 public void sameDepInTwoGroups_Same_One_Deps() throws Exception {
1783 sameDepInTwoGroups(/*sameFirst=*/true, /*twoCalls=*/false, /*valuesOrThrow=*/false);
1784 }
1785
1786 @Test
1787 public void sameDepInTwoGroups_Different_Two_Throw() throws Exception {
1788 sameDepInTwoGroups(/*sameFirst=*/false, /*twoCalls=*/true, /*valuesOrThrow=*/true);
1789 }
1790
1791 @Test
1792 public void sameDepInTwoGroups_Different_Two_Deps() throws Exception {
1793 sameDepInTwoGroups(/*sameFirst=*/false, /*twoCalls=*/true, /*valuesOrThrow=*/false);
1794 }
1795
1796 @Test
1797 public void sameDepInTwoGroups_Different_One_Throw() throws Exception {
1798 sameDepInTwoGroups(/*sameFirst=*/false, /*twoCalls=*/false, /*valuesOrThrow=*/true);
1799 }
1800
1801 @Test
1802 public void sameDepInTwoGroups_Different_One_Deps() throws Exception {
1803 sameDepInTwoGroups(/*sameFirst=*/false, /*twoCalls=*/false, /*valuesOrThrow=*/false);
1804 }
1805
1806 private void getValuesOrThrowWithErrors(boolean keepGoing) throws Exception {
1807 graph = new InMemoryGraph();
1808 SkyKey parentKey = GraphTester.toSkyKey("parent");
1809 final SkyKey errorDep = GraphTester.toSkyKey("errorChild");
1810 final SomeErrorException childExn = new SomeErrorException("child error");
1811 tester.getOrCreate(errorDep).setBuilder(new SkyFunction() {
1812 @Override
1813 public SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException {
1814 throw new GenericFunctionException(childExn, Transience.PERSISTENT);
1815 }
1816
1817 @Override
1818 public String extractTag(SkyKey skyKey) {
1819 return null;
1820 }
1821 });
1822 final List<SkyKey> deps = new ArrayList<>();
1823 for (int i = 1; i <= 3; i++) {
1824 SkyKey dep = GraphTester.toSkyKey("child" + i);
1825 deps.add(dep);
1826 tester.set(dep, new StringValue("child" + i));
1827 }
1828 final SomeErrorException parentExn = new SomeErrorException("parent error");
1829 tester.getOrCreate(parentKey).setBuilder(new SkyFunction() {
1830 @Override
1831 public SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException {
1832 try {
1833 SkyValue value = env.getValueOrThrow(errorDep, SomeErrorException.class);
1834 if (value == null) {
1835 return null;
1836 }
1837 } catch (SomeErrorException e) {
1838 // Recover from the child error.
1839 }
1840 env.getValues(deps);
1841 if (env.valuesMissing()) {
1842 return null;
1843 }
1844 throw new GenericFunctionException(parentExn, Transience.PERSISTENT);
1845 }
1846
1847 @Override
1848 public String extractTag(SkyKey skyKey) {
1849 return null;
1850 }
1851 });
1852 EvaluationResult<StringValue> evaluationResult = eval(keepGoing, ImmutableList.of(parentKey));
1853 assertTrue(evaluationResult.hasError());
1854 assertEquals(keepGoing ? parentExn : childExn, evaluationResult.getError().getException());
1855 }
1856
1857 @Test
1858 public void getValuesOrThrowWithErrors_NoKeepGoing() throws Exception {
1859 getValuesOrThrowWithErrors(/*keepGoing=*/false);
1860 }
1861
1862 @Test
1863 public void getValuesOrThrowWithErrors_KeepGoing() throws Exception {
1864 getValuesOrThrowWithErrors(/*keepGoing=*/true);
1865 }
1866
1867 @Test
1868 public void duplicateCycles() throws Exception {
1869 graph = new InMemoryGraph();
1870 SkyKey grandparentKey = GraphTester.toSkyKey("grandparent");
1871 SkyKey parentKey1 = GraphTester.toSkyKey("parent1");
1872 SkyKey parentKey2 = GraphTester.toSkyKey("parent2");
1873 SkyKey loopKey1 = GraphTester.toSkyKey("loop1");
1874 SkyKey loopKey2 = GraphTester.toSkyKey("loop2");
1875 tester.getOrCreate(loopKey1).addDependency(loopKey2);
1876 tester.getOrCreate(loopKey2).addDependency(loopKey1);
1877 tester.getOrCreate(parentKey1).addDependency(loopKey1);
1878 tester.getOrCreate(parentKey2).addDependency(loopKey2);
1879 tester.getOrCreate(grandparentKey).addDependency(parentKey1);
1880 tester.getOrCreate(grandparentKey).addDependency(parentKey2);
1881
1882 ErrorInfo errorInfo = evalValueInError(grandparentKey);
1883 List<ImmutableList<SkyKey>> cycles = Lists.newArrayList();
1884 for (CycleInfo cycleInfo : errorInfo.getCycleInfo()) {
1885 cycles.add(cycleInfo.getCycle());
1886 }
1887 // Skyframe doesn't automatically dedupe cycles that are the same except for entry point.
1888 assertEquals(2, cycles.size());
1889 int numUniqueCycles = 0;
1890 CycleDeduper<SkyKey> cycleDeduper = new CycleDeduper<SkyKey>();
1891 for (ImmutableList<SkyKey> cycle : cycles) {
1892 if (cycleDeduper.seen(cycle)) {
1893 numUniqueCycles++;
1894 }
1895 }
1896 assertEquals(1, numUniqueCycles);
1897 }
1898
1899 @Test
1900 public void signalValueEnqueuedAndEvaluated() throws Exception {
1901 final Set<SkyKey> enqueuedValues = Sets.newConcurrentHashSet();
1902 final Set<SkyKey> evaluatedValues = Sets.newConcurrentHashSet();
1903 EvaluationProgressReceiver progressReceiver = new EvaluationProgressReceiver() {
1904 @Override
Mark Schallerd12c3982015-08-10 19:46:03 +00001905 public void invalidated(SkyKey skyKey, InvalidationState state) {
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01001906 throw new IllegalStateException();
1907 }
1908
1909 @Override
1910 public void enqueueing(SkyKey skyKey) {
1911 enqueuedValues.add(skyKey);
1912 }
1913
1914 @Override
Nathan Harmata6f094bb2015-09-03 19:27:38 +00001915 public void computed(SkyKey skyKey, long elapsedTimeNanos) {}
1916
1917 @Override
Mark Schaller4752dbb2015-08-20 18:57:44 +00001918 public void evaluated(SkyKey skyKey, Supplier<SkyValue> skyValueSupplier,
1919 EvaluationState state) {
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01001920 evaluatedValues.add(skyKey);
1921 }
1922 };
1923
1924 EventHandler reporter = new EventHandler() {
1925 @Override
1926 public void handle(Event e) {
1927 throw new IllegalStateException();
1928 }
1929 };
1930
1931 MemoizingEvaluator aug = new InMemoryMemoizingEvaluator(
1932 ImmutableMap.of(GraphTester.NODE_TYPE, tester.getFunction()), new RecordingDifferencer(),
1933 progressReceiver);
1934 SequentialBuildDriver driver = new SequentialBuildDriver(aug);
1935
1936 tester.getOrCreate("top1").setComputedValue(CONCATENATE)
1937 .addDependency("d1").addDependency("d2");
1938 tester.getOrCreate("top2").setComputedValue(CONCATENATE).addDependency("d3");
1939 tester.getOrCreate("top3");
1940 assertThat(enqueuedValues).isEmpty();
1941 assertThat(evaluatedValues).isEmpty();
1942
1943 tester.set("d1", new StringValue("1"));
1944 tester.set("d2", new StringValue("2"));
1945 tester.set("d3", new StringValue("3"));
1946
1947 driver.evaluate(ImmutableList.of(GraphTester.toSkyKey("top1")), false, 200, reporter);
1948 assertThat(enqueuedValues)
1949 .containsExactlyElementsIn(Arrays.asList(GraphTester.toSkyKeys("top1", "d1", "d2")));
1950 assertThat(evaluatedValues)
1951 .containsExactlyElementsIn(Arrays.asList(GraphTester.toSkyKeys("top1", "d1", "d2")));
1952 enqueuedValues.clear();
1953 evaluatedValues.clear();
1954
1955 driver.evaluate(ImmutableList.of(GraphTester.toSkyKey("top2")), false, 200, reporter);
1956 assertThat(enqueuedValues)
1957 .containsExactlyElementsIn(Arrays.asList(GraphTester.toSkyKeys("top2", "d3")));
1958 assertThat(evaluatedValues)
1959 .containsExactlyElementsIn(Arrays.asList(GraphTester.toSkyKeys("top2", "d3")));
1960 enqueuedValues.clear();
1961 evaluatedValues.clear();
1962
1963 driver.evaluate(ImmutableList.of(GraphTester.toSkyKey("top1")), false, 200, reporter);
1964 assertThat(enqueuedValues).isEmpty();
1965 assertThat(evaluatedValues)
1966 .containsExactlyElementsIn(Arrays.asList(GraphTester.toSkyKeys("top1")));
1967 }
1968
1969 public void runDepOnErrorHaltsNoKeepGoingBuildEagerly(boolean childErrorCached,
1970 final boolean handleChildError) throws Exception {
1971 graph = new InMemoryGraph();
1972 SkyKey parentKey = GraphTester.toSkyKey("parent");
1973 final SkyKey childKey = GraphTester.toSkyKey("child");
1974 tester.getOrCreate(childKey).setHasError(/*hasError=*/true);
1975 // The parent should be built exactly twice: once during normal evaluation and once
1976 // during error bubbling.
1977 final AtomicInteger numParentInvocations = new AtomicInteger(0);
1978 tester.getOrCreate(parentKey).setBuilder(new SkyFunction() {
1979 @Override
1980 public SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException {
1981 int invocations = numParentInvocations.incrementAndGet();
1982 if (handleChildError) {
1983 try {
1984 SkyValue value = env.getValueOrThrow(childKey, SomeErrorException.class);
1985 // On the first invocation, either the child error should already be cached and not
1986 // propagated, or it should be computed freshly and not propagated. On the second build
1987 // (error bubbling), the child error should be propagated.
1988 assertTrue("bogus non-null value " + value, value == null);
1989 assertEquals("parent incorrectly re-computed during normal evaluation", 1, invocations);
1990 assertFalse("child error not propagated during error bubbling",
1991 env.inErrorBubblingForTesting());
1992 return value;
1993 } catch (SomeErrorException e) {
1994 assertTrue("child error propagated during normal evaluation",
1995 env.inErrorBubblingForTesting());
1996 assertEquals(2, invocations);
1997 return null;
1998 }
1999 } else {
2000 if (invocations == 1) {
2001 assertFalse("parent's first computation should be during normal evaluation",
2002 env.inErrorBubblingForTesting());
2003 return env.getValue(childKey);
2004 } else {
2005 assertEquals(2, invocations);
2006 assertTrue("parent incorrectly re-computed during normal evaluation",
2007 env.inErrorBubblingForTesting());
2008 return env.getValue(childKey);
2009 }
2010 }
2011 }
2012
2013 @Override
2014 public String extractTag(SkyKey skyKey) {
2015 return null;
2016 }
2017 });
2018 if (childErrorCached) {
2019 // Ensure that the child is already in the graph.
2020 evalValueInError(childKey);
2021 }
2022 EvaluationResult<StringValue> result = eval(/*keepGoing=*/false, ImmutableList.of(parentKey));
2023 assertEquals(2, numParentInvocations.get());
2024 assertTrue(result.hasError());
2025 assertEquals(childKey, result.getError().getRootCauseOfException());
2026 }
2027
2028 @Test
2029 public void depOnErrorHaltsNoKeepGoingBuildEagerly_ChildErrorCachedAndHandled()
2030 throws Exception {
2031 runDepOnErrorHaltsNoKeepGoingBuildEagerly(/*childErrorCached=*/true,
2032 /*handleChildError=*/true);
2033 }
2034
2035 @Test
2036 public void depOnErrorHaltsNoKeepGoingBuildEagerly_ChildErrorCachedAndNotHandled()
2037 throws Exception {
2038 runDepOnErrorHaltsNoKeepGoingBuildEagerly(/*childErrorCached=*/true,
2039 /*handleChildError=*/false);
2040 }
2041
2042 @Test
2043 public void depOnErrorHaltsNoKeepGoingBuildEagerly_ChildErrorFreshAndHandled() throws Exception {
2044 runDepOnErrorHaltsNoKeepGoingBuildEagerly(/*childErrorCached=*/false,
2045 /*handleChildError=*/true);
2046 }
2047
2048 @Test
2049 public void depOnErrorHaltsNoKeepGoingBuildEagerly_ChildErrorFreshAndNotHandled()
2050 throws Exception {
2051 runDepOnErrorHaltsNoKeepGoingBuildEagerly(/*childErrorCached=*/false,
2052 /*handleChildError=*/false);
2053 }
2054
2055 @Test
2056 public void raceConditionWithNoKeepGoingErrors_InflightError() throws Exception {
Mark Schallercc34ba02015-03-03 23:55:30 +00002057 // Given a graph of two nodes, errorKey and otherErrorKey,
2058 final SkyKey errorKey = GraphTester.toSkyKey("errorKey");
2059 final SkyKey otherErrorKey = GraphTester.toSkyKey("otherErrorKey");
Han-Wen Nienhuys24e95712015-02-20 13:32:49 +00002060
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01002061 final CountDownLatch errorCommitted = new CountDownLatch(1);
Mark Schallercc34ba02015-03-03 23:55:30 +00002062
2063 final CountDownLatch otherStarted = new CountDownLatch(1);
Mark Schallercc34ba02015-03-03 23:55:30 +00002064
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01002065 final CountDownLatch otherDone = new CountDownLatch(1);
Mark Schallercc34ba02015-03-03 23:55:30 +00002066
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01002067 final AtomicInteger numOtherInvocations = new AtomicInteger(0);
Mark Schallercc34ba02015-03-03 23:55:30 +00002068 final AtomicReference<String> bogusInvocationMessage = new AtomicReference<>(null);
2069 final AtomicReference<String> nonNullValueMessage = new AtomicReference<>(null);
Han-Wen Nienhuys8d356be2015-02-17 11:15:16 +00002070
Janak Ramakrishnana67bb8b2015-09-02 17:56:08 +00002071 tester
2072 .getOrCreate(errorKey)
2073 .setBuilder(
2074 new SkyFunction() {
2075 @Override
2076 public SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException {
2077 // Given that errorKey waits for otherErrorKey to begin evaluation before completing
2078 // its evaluation,
2079 TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
2080 otherStarted, "otherErrorKey's SkyFunction didn't start in time.");
2081 // And given that errorKey throws an error,
2082 throw new GenericFunctionException(
2083 new SomeErrorException("error"), Transience.PERSISTENT);
2084 }
Mark Schallercc34ba02015-03-03 23:55:30 +00002085
Janak Ramakrishnana67bb8b2015-09-02 17:56:08 +00002086 @Override
2087 public String extractTag(SkyKey skyKey) {
2088 return null;
2089 }
2090 });
2091 tester
2092 .getOrCreate(otherErrorKey)
2093 .setBuilder(
2094 new SkyFunction() {
2095 @Override
2096 public SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException {
2097 otherStarted.countDown();
2098 int invocations = numOtherInvocations.incrementAndGet();
2099 // And given that otherErrorKey waits for errorKey's error to be committed before
2100 // trying to get errorKey's value,
2101 TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
2102 errorCommitted, "errorKey's error didn't get committed to the graph in time");
2103 try {
2104 SkyValue value = env.getValueOrThrow(errorKey, SomeErrorException.class);
2105 if (value != null) {
2106 nonNullValueMessage.set("bogus non-null value " + value);
2107 }
2108 if (invocations != 1) {
2109 bogusInvocationMessage.set("bogus invocation count: " + invocations);
2110 }
2111 otherDone.countDown();
2112 // And given that otherErrorKey throws an error,
2113 throw new GenericFunctionException(
2114 new SomeErrorException("other"), Transience.PERSISTENT);
2115 } catch (SomeErrorException e) {
2116 fail();
2117 return null;
2118 }
2119 }
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01002120
Janak Ramakrishnana67bb8b2015-09-02 17:56:08 +00002121 @Override
2122 public String extractTag(SkyKey skyKey) {
2123 return null;
2124 }
2125 });
2126 graph =
2127 new NotifyingInMemoryGraph(
2128 new Listener() {
2129 @Override
2130 public void accept(SkyKey key, EventType type, Order order, Object context) {
2131 if (key.equals(errorKey) && type == EventType.SET_VALUE && order == Order.AFTER) {
2132 errorCommitted.countDown();
2133 TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
2134 otherDone, "otherErrorKey's SkyFunction didn't finish in time.");
2135 }
2136 }
2137 });
Mark Schallercc34ba02015-03-03 23:55:30 +00002138
2139 // When the graph is evaluated in noKeepGoing mode,
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01002140 EvaluationResult<StringValue> result = eval(/*keepGoing=*/false,
Mark Schallercc34ba02015-03-03 23:55:30 +00002141 ImmutableList.of(errorKey, otherErrorKey));
2142
2143 // Then the result reports that an error occurred because of errorKey,
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01002144 assertTrue(result.hasError());
2145 assertEquals(errorKey, result.getError().getRootCauseOfException());
Mark Schallercc34ba02015-03-03 23:55:30 +00002146
2147 // And no value is committed for otherErrorKey,
2148 assertNull(graph.get(otherErrorKey));
2149
2150 // And no value was committed for errorKey,
2151 assertNull(nonNullValueMessage.get(), nonNullValueMessage.get());
2152
2153 // And the SkyFunction for otherErrorKey was evaluated exactly once.
2154 assertEquals(numOtherInvocations.get(), 1);
2155 assertNull(bogusInvocationMessage.get(), bogusInvocationMessage.get());
2156
2157 // NB: The SkyFunction for otherErrorKey gets evaluated exactly once--it does not get
2158 // re-evaluated during error bubbling. Why? When otherErrorKey throws, it is always the
2159 // second error encountered, because it waited for errorKey's error to be committed before
2160 // trying to get it. In fail-fast evaluations only the first failing SkyFunction's
2161 // newly-discovered-dependencies are registered. Therefore, there won't be a reverse-dep from
2162 // errorKey to otherErrorKey for the error to bubble through.
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01002163 }
2164
2165 @Test
2166 public void raceConditionWithNoKeepGoingErrors_FutureError() throws Exception {
2167 final CountDownLatch errorCommitted = new CountDownLatch(1);
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01002168 final CountDownLatch otherStarted = new CountDownLatch(1);
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01002169 final CountDownLatch otherParentSignaled = new CountDownLatch(1);
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01002170 final SkyKey errorParentKey = GraphTester.toSkyKey("errorParentKey");
2171 final SkyKey errorKey = GraphTester.toSkyKey("errorKey");
2172 final SkyKey otherParentKey = GraphTester.toSkyKey("otherParentKey");
2173 final SkyKey otherKey = GraphTester.toSkyKey("otherKey");
2174 final AtomicInteger numOtherParentInvocations = new AtomicInteger(0);
2175 final AtomicInteger numErrorParentInvocations = new AtomicInteger(0);
2176 tester.getOrCreate(otherParentKey).setBuilder(new SkyFunction() {
2177 @Override
2178 public SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException {
2179 int invocations = numOtherParentInvocations.incrementAndGet();
2180 assertEquals("otherParentKey should not be restarted", 1, invocations);
2181 return env.getValue(otherKey);
2182 }
2183
2184 @Override
2185 public String extractTag(SkyKey skyKey) {
2186 return null;
2187 }
2188 });
Janak Ramakrishnana67bb8b2015-09-02 17:56:08 +00002189 tester
2190 .getOrCreate(otherKey)
2191 .setBuilder(
2192 new SkyFunction() {
2193 @Override
2194 public SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException {
2195 otherStarted.countDown();
2196 TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
2197 errorCommitted, "error didn't get committed to the graph in time");
2198 return new StringValue("other");
2199 }
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01002200
Janak Ramakrishnana67bb8b2015-09-02 17:56:08 +00002201 @Override
2202 public String extractTag(SkyKey skyKey) {
2203 return null;
2204 }
2205 });
2206 tester
2207 .getOrCreate(errorKey)
2208 .setBuilder(
2209 new SkyFunction() {
2210 @Override
2211 public SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException {
2212 TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
2213 otherStarted, "other didn't start in time");
2214 throw new GenericFunctionException(
2215 new SomeErrorException("error"), Transience.PERSISTENT);
2216 }
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01002217
Janak Ramakrishnana67bb8b2015-09-02 17:56:08 +00002218 @Override
2219 public String extractTag(SkyKey skyKey) {
2220 return null;
2221 }
2222 });
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01002223 tester.getOrCreate(errorParentKey).setBuilder(new SkyFunction() {
2224 @Override
2225 public SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException {
2226 int invocations = numErrorParentInvocations.incrementAndGet();
2227 try {
2228 SkyValue value = env.getValueOrThrow(errorKey, SomeErrorException.class);
2229 assertTrue("bogus non-null value " + value, value == null);
2230 if (invocations == 1) {
2231 return null;
2232 } else {
2233 assertFalse(env.inErrorBubblingForTesting());
2234 fail("RACE CONDITION: errorParentKey was restarted!");
2235 return null;
2236 }
2237 } catch (SomeErrorException e) {
2238 assertTrue("child error propagated during normal evaluation",
2239 env.inErrorBubblingForTesting());
2240 assertEquals(2, invocations);
2241 return null;
2242 }
2243 }
2244
2245 @Override
2246 public String extractTag(SkyKey skyKey) {
2247 return null;
2248 }
2249 });
Janak Ramakrishnana67bb8b2015-09-02 17:56:08 +00002250 graph =
2251 new NotifyingInMemoryGraph(
2252 new Listener() {
2253 @Override
2254 public void accept(SkyKey key, EventType type, Order order, Object context) {
2255 if (key.equals(errorKey) && type == EventType.SET_VALUE && order == Order.AFTER) {
2256 errorCommitted.countDown();
2257 TrackingAwaiter.INSTANCE.awaitLatchAndTrackExceptions(
2258 otherParentSignaled, "otherParent didn't get signaled in time");
2259 // We try to give some time for ParallelEvaluator to incorrectly re-evaluate
2260 // 'otherParentKey'. This test case is testing for a real race condition and the
2261 // 10ms time was chosen experimentally to give a true positive rate of 99.8%
2262 // (without a sleep it has a 1% true positive rate). There's no good way to do
2263 // this without sleeping. We *could* introspect ParallelEvaulator's
2264 // AbstractQueueVisitor to see if the re-evaluation has been enqueued, but that's
2265 // relying on pretty low-level implementation details.
2266 Uninterruptibles.sleepUninterruptibly(10, TimeUnit.MILLISECONDS);
2267 }
2268 if (key.equals(otherParentKey)
2269 && type == EventType.SIGNAL
2270 && order == Order.AFTER) {
2271 otherParentSignaled.countDown();
2272 }
2273 }
2274 });
Han-Wen Nienhuysd08b27f2015-02-25 16:45:20 +01002275 EvaluationResult<StringValue> result = eval(/*keepGoing=*/false,
2276 ImmutableList.of(otherParentKey, errorParentKey));
2277 assertTrue(result.hasError());
2278 assertEquals(errorKey, result.getError().getRootCauseOfException());
2279 }
2280
2281 @Test
2282 public void cachedErrorsFromKeepGoingUsedOnNoKeepGoing() throws Exception {
2283 graph = new DeterministicInMemoryGraph();
2284 tester = new GraphTester();
2285 SkyKey errorKey = GraphTester.toSkyKey("error");
2286 SkyKey parent1Key = GraphTester.toSkyKey("parent1");
2287 SkyKey parent2Key = GraphTester.toSkyKey("parent2");
2288 tester.getOrCreate(parent1Key).addDependency(errorKey).setConstantValue(
2289 new StringValue("parent1"));
2290 tester.getOrCreate(parent2Key).addDependency(errorKey).setConstantValue(
2291 new StringValue("parent2"));
2292 tester.getOrCreate(errorKey).setHasError(true);
2293 EvaluationResult<StringValue> result = eval(/*keepGoing=*/true, ImmutableList.of(parent1Key));
2294 assertTrue(result.hasError());
2295 assertEquals(errorKey, result.getError().getRootCauseOfException());
2296 result = eval(/*keepGoing=*/false, ImmutableList.of(parent2Key));
2297 assertTrue(result.hasError());
2298 assertEquals(errorKey, result.getError(parent2Key).getRootCauseOfException());
2299 }
2300
2301 @Test
2302 public void cachedTopLevelErrorsShouldHaltNoKeepGoingBuildEarly() throws Exception {
2303 graph = new DeterministicInMemoryGraph();
2304 tester = new GraphTester();
2305 SkyKey errorKey = GraphTester.toSkyKey("error");
2306 tester.getOrCreate(errorKey).setHasError(true);
2307 EvaluationResult<StringValue> result = eval(/*keepGoing=*/true, ImmutableList.of(errorKey));
2308 assertTrue(result.hasError());
2309 assertEquals(errorKey, result.getError().getRootCauseOfException());
2310 SkyKey rogueKey = GraphTester.toSkyKey("rogue");
2311 tester.getOrCreate(rogueKey).setBuilder(new SkyFunction() {
2312 @Override
2313 public SkyValue compute(SkyKey skyKey, Environment env) {
2314 // This SkyFunction could do an arbitrarily bad computation, e.g. loop-forever. So we want
2315 // to make sure that it is never run when we want to fail-fast anyway.
2316 fail("eval call should have already terminated");
2317 return null;
2318 }
2319
2320 @Override
2321 public String extractTag(SkyKey skyKey) {
2322 return null;
2323 }
2324 });
2325 result = eval(/*keepGoing=*/false, ImmutableList.of(errorKey, rogueKey));
2326 assertTrue(result.hasError());
2327 assertEquals(errorKey, result.getError(errorKey).getRootCauseOfException());
2328 assertFalse(result.errorMap().containsKey(rogueKey));
2329 }
2330
2331 private void runUnhandledTransitiveErrors(boolean keepGoing,
2332 final boolean explicitlyPropagateError) throws Exception {
2333 graph = new DeterministicInMemoryGraph();
2334 tester = new GraphTester();
2335 SkyKey grandparentKey = GraphTester.toSkyKey("grandparent");
2336 final SkyKey parentKey = GraphTester.toSkyKey("parent");
2337 final SkyKey childKey = GraphTester.toSkyKey("child");
2338 final AtomicBoolean errorPropagated = new AtomicBoolean(false);
2339 tester.getOrCreate(grandparentKey).setBuilder(new SkyFunction() {
2340 @Override
2341 public SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException {
2342 try {
2343 return env.getValueOrThrow(parentKey, SomeErrorException.class);
2344 } catch (SomeErrorException e) {
2345 errorPropagated.set(true);
2346 throw new GenericFunctionException(e, Transience.PERSISTENT);
2347 }
2348 }
2349
2350 @Override
2351 public String extractTag(SkyKey skyKey) {
2352 return null;
2353 }
2354 });
2355 tester.getOrCreate(parentKey).setBuilder(new SkyFunction() {
2356 @Override
2357 public SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException {
2358 if (explicitlyPropagateError) {
2359 try {
2360 return env.getValueOrThrow(childKey, SomeErrorException.class);
2361 } catch (SomeErrorException e) {
2362 throw new GenericFunctionException(e, childKey);
2363 }
2364 } else {
2365 return env.getValue(childKey);
2366 }
2367 }
2368
2369 @Override
2370 public String extractTag(SkyKey skyKey) {
2371 return null;
2372 }
2373 });
2374 tester.getOrCreate(childKey).setHasError(/*hasError=*/true);
2375 EvaluationResult<StringValue> result = eval(keepGoing, ImmutableList.of(grandparentKey));
2376 assertTrue(result.hasError());
2377 assertTrue(errorPropagated.get());
2378 assertEquals(grandparentKey, result.getError().getRootCauseOfException());
2379 }
2380
2381 @Test
2382 public void unhandledTransitiveErrorsDuringErrorBubbling_ImplicitPropagation() throws Exception {
2383 runUnhandledTransitiveErrors(/*keepGoing=*/false, /*explicitlyPropagateError=*/false);
2384 }
2385
2386 @Test
2387 public void unhandledTransitiveErrorsDuringErrorBubbling_ExplicitPropagation() throws Exception {
2388 runUnhandledTransitiveErrors(/*keepGoing=*/false, /*explicitlyPropagateError=*/true);
2389 }
2390
2391 @Test
2392 public void unhandledTransitiveErrorsDuringNormalEvaluation_ImplicitPropagation()
2393 throws Exception {
2394 runUnhandledTransitiveErrors(/*keepGoing=*/true, /*explicitlyPropagateError=*/false);
2395 }
2396
2397 @Test
2398 public void unhandledTransitiveErrorsDuringNormalEvaluation_ExplicitPropagation()
2399 throws Exception {
2400 runUnhandledTransitiveErrors(/*keepGoing=*/true, /*explicitlyPropagateError=*/true);
2401 }
2402}