Track build file dependencies.

Summary:
Only invalidate affected cached build rules when:
1) the build file containing the rules is changed or removed
2) a file the build file transitively includes is changed or removed or
3) a file that could be in a glob in the build file is added or removed
This should greatly improve the performance of the buckd build file cache,
which currently invalidates the entire cache when anything changes.

Test Plan: ant && ant test
diff --git a/.buckconfig b/.buckconfig
index b6a3f51..d22dc6e 100644
--- a/.buckconfig
+++ b/.buckconfig
@@ -15,7 +15,7 @@
       test/com/facebook/buck/java/testdata, \
       test/com/facebook/buck/junit/testdata, \
       test/com/facebook/buck/parser/testdata, \
-      test/com/facebook/buck/shell/testdata, \
+      test/com/facebook/buck/shell/testdata,
 [test]
     # Maximum timeout of 1 minute per test.
     timeout = 60000
diff --git a/.idea/misc.xml b/.idea/misc.xml
index f4c7cd6..4a38ed5 100644
--- a/.idea/misc.xml
+++ b/.idea/misc.xml
@@ -6,6 +6,7 @@
   <component name="FrameworkDetectionExcludesConfiguration">
     <type id="android" />
   </component>
+  <component name="IdProvider" IDEtalkID="0DE2DAD55CCA0E8D48163E3F370D8939" />
   <component name="ProjectResources">
     <default-html-doctype>http://www.w3.org/1999/xhtml</default-html-doctype>
   </component>
diff --git a/src/com/facebook/buck/json/ProjectBuildFileParser.java b/src/com/facebook/buck/json/ProjectBuildFileParser.java
index a7f94af..8516248 100644
--- a/src/com/facebook/buck/json/ProjectBuildFileParser.java
+++ b/src/com/facebook/buck/json/ProjectBuildFileParser.java
@@ -33,6 +33,7 @@
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -186,6 +187,20 @@
    */
   public List<Map<String, Object>> getAllRules(String buildFile)
       throws BuildFileParseException {
+    List<Map<String, Object>> result = getAllRulesAndMetaRules(buildFile);
+
+    // Strip out the __includes meta rule, which is the last rule.
+    return Collections.unmodifiableList(result.subList(0, result.size() - 1));
+  }
+
+  /**
+   * Collect all rules from a particular build file, along with meta rules about the rules, for
+   * example which build files the rules depend on.
+   *
+   * @param buildFile should be an absolute path to a build file. Must have rootPath as its prefix.
+   */
+  public List<Map<String, Object>> getAllRulesAndMetaRules(String buildFile)
+      throws BuildFileParseException {
     try {
       return getAllRulesInternal(Optional.of(buildFile));
     } catch (IOException e) {
diff --git a/src/com/facebook/buck/parser/Parser.java b/src/com/facebook/buck/parser/Parser.java
index b3bc987..69ced01 100644
--- a/src/com/facebook/buck/parser/Parser.java
+++ b/src/com/facebook/buck/parser/Parser.java
@@ -57,12 +57,16 @@
 import java.util.Set;
 
 import javax.annotation.Nullable;
+import javax.annotation.concurrent.NotThreadSafe;
 
 /**
  * High-level build file parsing machinery.  Primarily responsible for producing a
  * {@link DependencyGraph} based on a set of targets.  Also exposes some low-level facilities to
- * parse individual build files.
+ * parse individual build files. Caches build rules to minimise the number of calls to python and
+ * processes filesystem WatchEvents to invalidate the cache as files change. Expected to be used
+ * from a single thread, so methods are not synchronized or thread safe.
  */
+@NotThreadSafe
 public class Parser {
 
   private final BuildTargetParser buildTargetParser;
@@ -70,7 +74,7 @@
   /**
    * The build files that have been parsed and whose build rules are in {@link #knownBuildTargets}.
    */
-  private final ListMultimap<File, Map<String, Object>> parsedBuildFiles;
+  private final ListMultimap<Path, Map<String, Object>> parsedBuildFiles;
 
   /**
    * True if all build files have been parsed and so all rules are in {@link #knownBuildTargets}.
@@ -78,15 +82,19 @@
   private boolean allBuildFilesParsed;
 
   /**
-   * Files included by build files. Changing includes invalidates cached build rules.
+   * Files included by build files. If the default includes are changed, then build files need to be
+   * reevaluated with the new includes, so the includes used when populating the rule cache are
+   * stored between requests to parse build files and the cache is invalidated and build files
+   * reevaluated if the includes change.
    */
   @Nullable
-  private List<String> includes;
+  private List<String> cacheDefaultIncludes;
 
   /**
    * We parse a build file in search for one particular rule; however, we also keep track of the
    * other rules that were also parsed from it.
    */
+  // TODO(user): Stop caching these in addition to parsedBuildFiles?
   private final Map<BuildTarget, BuildRuleBuilder<?>> knownBuildTargets;
 
   private final ProjectFilesystem projectFilesystem;
@@ -95,6 +103,19 @@
   private final Console console;
 
   /**
+   * Key of the meta-rule that lists the build files executed while reading rules.
+   * The value is a list of strings with the root build file as the head and included
+   * build files as the tail, for example: {"__includes":["/jimp/BUCK", "/jimp/buck_includes"]}
+   */
+  private static final String INCLUDES_META_RULE = "__includes";
+
+  /**
+   * A map from absolute included files ({@code /jimp/BUILD_DEFS}, for example) to the build files
+   * that depend on them (typically {@code /jimp/BUCK} files).
+   */
+  private final ListMultimap<Path, Path> buildFileDependents;
+
+  /**
    * A cached BuildFileTree which can be invalidated and lazily constructs new BuildFileTrees.
    * TODO(user): refactor this as a generic CachingSupplier<T> when it's needed elsewhere.
    */
@@ -168,6 +189,7 @@
     this.buildTargetParser = Preconditions.checkNotNull(buildTargetParser);
     this.buildFileParserFactory = Preconditions.checkNotNull(buildFileParserFactory);
     this.parsedBuildFiles = ArrayListMultimap.create();
+    this.buildFileDependents = ArrayListMultimap.create();
   }
 
   public BuildTargetParser getBuildTargetParser() {
@@ -179,39 +201,50 @@
   }
 
   /**
-   * @param file the build file to look up in the {@link #parsedBuildFiles} cache.
+   * The rules in a build file are cached if that specific build file was parsed or all build
+   * files in the project were parsed and the includes haven't changed since the rules were
+   * cached.
+   *
+   * @param buildFile the build file to look up in the {@link #parsedBuildFiles} cache.
    * @param includes the files to include before executing the build file.
    * @return true if the build file has already been parsed and its rules are cached.
    */
-  private boolean isCached(File file, Iterable<String> includes) {
-    return isCacheValid(includes) && (allBuildFilesParsed || parsedBuildFiles.containsKey(file));
+  private boolean isCached(File buildFile, Iterable<String> includes) {
+    return !invalidateCacheOnIncludeChange(includes) && (allBuildFilesParsed ||
+        parsedBuildFiles.containsKey(normalize(buildFile.toPath())));
   }
 
   /**
+   * The cache is complete if all build files in the project were parsed and the includes haven't
+   * changed since the rules were cached.
+   *
    * @param includes the files to include before executing the build file.
    * @return true if all build files have already been parsed and their rules are cached.
    */
   private boolean isCacheComplete(Iterable<String> includes) {
-    return isCacheValid(includes) && allBuildFilesParsed;
+    return !invalidateCacheOnIncludeChange(includes) && allBuildFilesParsed;
   }
 
   /**
+   * Invalidates the cached build rules if {@code includes} have changed since the last call.
+   * If the cache is invalidated the new {@code includes} used to build the new cache are stored.
+   *
    * @param includes the files to include before executing the build file.
-   * @return true if the cached build rules are valid. Invalidates the cache if not.
+   * @return true if the cache was invalidated, false if the cache is still valid.
    */
-  private boolean isCacheValid(Iterable<String> includes) {
+  private boolean invalidateCacheOnIncludeChange(Iterable<String> includes) {
     List<String> includesList = Lists.newArrayList(includes);
-    if (!includesList.equals(this.includes)) {
+    if (!includesList.equals(this.cacheDefaultIncludes)) {
       invalidateCache();
-      this.includes = includesList;
-      return false;
+      this.cacheDefaultIncludes = includesList;
+      return true;
     }
-    return true;
+    return false;
   }
 
   private void invalidateCache() {
     if (console.getVerbosity() == Verbosity.ALL) {
-      console.getStdErr().println("Parser invalidating cache");
+      console.getStdErr().println("Parser invalidating entire cache");
     }
     parsedBuildFiles.clear();
     knownBuildTargets.clear();
@@ -401,10 +434,10 @@
             buildFile);
       }
 
-      parseRawRulesInternal(buildFileParser.getAllRules(buildFile.getPath()),
+      parseRawRulesInternal(buildFileParser.getAllRulesAndMetaRules(buildFile.getPath()),
           buildFile);
     }
-    return parsedBuildFiles.get(buildFile);
+    return parsedBuildFiles.get(normalize(buildFile.toPath()));
   }
 
   /**
@@ -415,6 +448,12 @@
   void parseRawRulesInternal(Iterable<Map<String, Object>> rules,
       @Nullable File source) throws NoSuchBuildTargetException {
     for (Map<String, Object> map : rules) {
+
+      if (isMetaRule(map)) {
+        parseMetaRule(map);
+        continue;
+      }
+
       BuildRuleType buildRuleType = parseBuildRuleTypeFromRawRule(map);
       BuildTarget target = parseBuildTargetFromRawRule(map, source);
       BuildRuleFactory<?> factory = buildRuleTypes.getFactory(buildRuleType);
@@ -434,11 +473,37 @@
       if (existingRule != null) {
         throw new RuntimeException("Duplicate definition for " + target.getFullyQualifiedName());
       }
-      parsedBuildFiles.put(target.getBuildFile(), map);
+      parsedBuildFiles.put(normalize(target.getBuildFile().toPath()), map);
     }
   }
 
   /**
+   * @param map a build rule read from a build file.
+   * @return true if map represents a meta rule.
+   */
+  private boolean isMetaRule(Map<String, Object> map) {
+    return map.containsKey(INCLUDES_META_RULE);
+  }
+
+  /**
+   * Processes build file meta rules and returns true if map represents a meta rule.
+   * @param map a meta rule read from a build file.
+   */
+  @SuppressWarnings("unchecked") // Needed for downcast from Object to List<String>.
+  private boolean parseMetaRule(Map<String, Object> map) {
+    Preconditions.checkState(isMetaRule(map));
+
+    // INCLUDES_META_RULE maps to a list of file paths: the head is a
+    // dependent build file and the tail is a list of the files it includes.
+    List<String> fileNames = ((List<String>) map.get(INCLUDES_META_RULE));
+    Path dependent = normalize(new File(fileNames.get(0)).toPath());
+    for (String fileName : fileNames) {
+      buildFileDependents.put(normalize(new File(fileName).toPath()), dependent);
+    }
+    return true;
+  }
+
+  /**
    * @param filter the test to apply to all targets that have been read from build files, or null.
    * @return the build targets that pass the test, or null if the filter was null.
    */
@@ -530,7 +595,7 @@
   private String createContextString(WatchEvent<?> event) {
     if (projectFilesystem.isPathChangeEvent(event)) {
       Path path = (Path) event.context();
-      return path.toAbsolutePath().toString();
+      return path.toAbsolutePath().normalize().toString();
     }
     return event.context().toString();
   }
@@ -547,33 +612,91 @@
           createContextString(event));
     }
 
-    boolean reconstructBuildFileTree = false;
     if (projectFilesystem.isPathChangeEvent(event)) {
-      String path = event.context().toString();
-      if (event.kind() == StandardWatchEventKinds.ENTRY_MODIFY) {
-        if (path.endsWith(".java")) {
-          // TODO(user): Track the files imported by build files
-          // Currently we just assume changed ".java" can't affect build rules.
-          // Adding or deleting ".java" files requires build files to be reevaluated due to globing.
-          return;
-        }
-      } else {
+      Path path = (Path) event.context();
+
+      if (isPathCreateOrDeleteEvent(event)) {
+
         if (path.endsWith(BuckConstant.BUILD_RULES_FILE_NAME)) {
-          // A BUCK file was added or deleted, so reconstruct the build file tree.
-          reconstructBuildFileTree = true;
+
+          // If a build file has been added or removed, reconstruct the build file tree.
+          buildFileTreeCache.invalidate();
+
         }
+
+        // Added or removed files can affect globs, so invalidate the package build file.
+        // TODO(user): avoid invalidating build files when backup files are added or removed.
+        String packageBuildFilePath =
+            buildFileTreeCache.get().getBasePathOfAncestorTarget(
+                projectFilesystem.getProjectRoot().toPath().relativize(path).toString());
+        invalidateDependents(
+            projectFilesystem.getFileForRelativePath(
+                packageBuildFilePath + '/' + BuckConstant.BUILD_RULES_FILE_NAME).toPath());
       }
+
+      // Invalidate the raw rules and targets dependent on this file.
+      invalidateDependents(path);
+
     } else {
-      // A non-path-change event happened: we have no idea what's going on,
-      // so reconstruct the build file tree to be safe.
-      reconstructBuildFileTree = true;
-    }
 
-    if (reconstructBuildFileTree) {
+      // Non-path change event, likely an overflow due to many change events: invalidate everything.
       buildFileTreeCache.invalidate();
+      invalidateCache();
+    }
+  }
+
+  private boolean isPathCreateOrDeleteEvent(WatchEvent<?> event) {
+    return event.kind() == StandardWatchEventKinds.ENTRY_CREATE ||
+        event.kind() == StandardWatchEventKinds.ENTRY_DELETE;
+  }
+
+  /**
+   * Remove the targets and rules defined by {@code path} from the cache and recursively remove the
+   * targets and rules defined by files that transitively include {@code path} from the cache.
+   * @param path The File that has changed.
+   */
+  private void invalidateDependents(Path path) {
+    // Normalize path to ensure it hashes equally with map keys.
+    path = normalize(path);
+
+    if (parsedBuildFiles.containsKey(path)) {
+      if (console.getVerbosity() == Verbosity.ALL) {
+        console.getStdErr().printf("Parser invalidating %s cache\n",
+            path.toAbsolutePath());
+      }
+
+      // Remove all targets defined by path from cache.
+      for (Map<String, Object> rawRule : parsedBuildFiles.get(path)) {
+        BuildTarget target = parseBuildTargetFromRawRule(rawRule, null);
+        knownBuildTargets.remove(target);
+      }
+
+      // Remove all rules defined in path from cache.
+      parsedBuildFiles.removeAll(path);
+
+      // All targets have no longer been parsed and cached.
+      allBuildFilesParsed = false;
     }
 
-    // TODO(user): invalidate affected build files, rather than nuking all rules completely.
-    invalidateCache();
+    // Recursively invalidate dependents.
+    for (Path dependent : buildFileDependents.get(path)) {
+
+      if (!dependent.equals(path)) {
+        invalidateDependents(dependent);
+      }
+    }
+
+    // Dependencies will be repopulated when files are re-parsed.
+    buildFileDependents.removeAll(path);
+  }
+
+  /**
+   * Always use Files created from absolute paths as they are returned from buck.py and must be
+   * created from consistent paths to be looked up correctly in maps.
+   * @param path A File to normalize.
+   * @return An equivalent file constructed from a normalized, absolute path to the given File.
+   */
+  private Path normalize(Path path) {
+    return path.toAbsolutePath().normalize();
   }
 }
diff --git a/src/com/facebook/buck/parser/buck.py b/src/com/facebook/buck/parser/buck.py
index 09862ac..19a7d22 100644
--- a/src/com/facebook/buck/parser/buck.py
+++ b/src/com/facebook/buck/parser/buck.py
@@ -645,6 +645,7 @@
     raise ValueError('include_defs argument "%s" must begin with //' % name)
   relative_path = name[2:]
   include_file = os.path.join(build_env['PROJECT_ROOT'], relative_path)
+  build_env['INCLUDES'].append(include_file)
   execfile(include_file, build_env['BUILD_FILE_SYMBOL_TABLE'])
 
 
@@ -713,6 +714,7 @@
     build_symbols = make_build_file_symbol_table(build_env)
     build_env['BUILD_FILE_SYMBOL_TABLE'] = build_symbols['symbol_table']
     build_env['LAZY_FUNCTIONS'] = build_symbols['lazy_functions']
+    build_env['INCLUDES'] = []
 
     # If there are any default includes, evaluate those first to populate the
     # build_env.
@@ -745,6 +747,7 @@
              build_env['BUILD_FILE_SYMBOL_TABLE'])
 
     values = build_env['RULES'].values()
+    values.append({"__includes": [build_file] + build_env['INCLUDES']})
     if self.server:
       print json.dumps(values)
     else:
diff --git a/src/com/facebook/buck/util/ProjectFilesystemWatcher.java b/src/com/facebook/buck/util/ProjectFilesystemWatcher.java
index 7d42b6d..ac57094 100644
--- a/src/com/facebook/buck/util/ProjectFilesystemWatcher.java
+++ b/src/com/facebook/buck/util/ProjectFilesystemWatcher.java
@@ -67,21 +67,48 @@
       if (dir == null) {
         continue; // Ignored or unknown directory.
       }
-      for (WatchEvent<?> event : key.pollEvents()) {
+      for (final WatchEvent<?> event : key.pollEvents()) {
         if (filesystem.isPathChangeEvent(event)) {
+
+          // Check against ignored directories.
           Path name = (Path) event.context();
-          Path child = dir.resolve(name);
+          final Path child = dir.resolve(name);
           if (shouldIgnore(child)) {
             continue;
           }
+
+          // If directory is created, watch its children.
           if (filesystem.isDirectory(child, LinkOption.NOFOLLOW_LINKS)) {
             if (event.kind() == StandardWatchEventKinds.ENTRY_CREATE) {
               registerAll(child);
             }
             continue; // TODO(user): post events about directories?
           }
+
+          // Path returned by event.context() is relative to key directory, so return resolved
+          // child Path instead to allow clients to access the full, absolute Path correctly.
+          eventBus.post(new WatchEvent<Path>(){
+
+            @Override
+            @SuppressWarnings("unchecked") // Needed for conversion from Kind<?> to Kind<Path>
+            public Kind<Path> kind() {
+              return (Kind<Path>) event.kind();
+            }
+
+            @Override
+            public int count() {
+              return event.count();
+            }
+
+            @Override
+            public Path context() {
+              return child;
+            }
+          });
+
+        } else {
+          eventBus.post(event);
         }
-        eventBus.post(event);
       }
 
       // Reset key and remove from set if directory no longer accessible
diff --git a/test/com/facebook/buck/parser/ParserTest.java b/test/com/facebook/buck/parser/ParserTest.java
index c82d0e9..f0830c7 100644
--- a/test/com/facebook/buck/parser/ParserTest.java
+++ b/test/com/facebook/buck/parser/ParserTest.java
@@ -18,10 +18,7 @@
 
 import static com.facebook.buck.parser.RawRulePredicates.alwaysFalse;
 import static com.facebook.buck.parser.RawRulePredicates.alwaysTrue;
-import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.expectLastCall;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
@@ -82,6 +79,9 @@
 public class ParserTest extends EasyMockSupport {
 
   private File testBuildFile;
+  private File includedByBuildFile;
+  private File includedByIncludeFile;
+  private File defaultIncludeFile;
   private Parser testParser;
   private KnownBuildRuleTypes buildRuleTypes;
   private ProjectFilesystem filesystem;
@@ -93,13 +93,36 @@
   public void setUp() throws IOException {
     tempDir.newFolder("java", "com", "facebook");
 
+    defaultIncludeFile = tempDir.newFile(
+        "java/com/facebook/defaultIncludeFile");
+    Files.write(
+        "\n",
+        defaultIncludeFile,
+        Charsets.UTF_8);
+
+    includedByIncludeFile = tempDir.newFile(
+        "java/com/facebook/includedByIncludeFile");
+    Files.write(
+        "\n",
+        includedByIncludeFile,
+        Charsets.UTF_8);
+
+    includedByBuildFile = tempDir.newFile(
+        "java/com/facebook/includedByBuildFile");
+    Files.write(
+        "include_defs('//java/com/facebook/includedByIncludeFile')\n",
+        includedByBuildFile,
+        Charsets.UTF_8);
+
     testBuildFile = tempDir.newFile(
         "java/com/facebook/" + BuckConstant.BUILD_RULES_FILE_NAME);
     Files.write(
+        "include_defs('//java/com/facebook/includedByBuildFile')\n" +
         "java_library(name = 'foo')\n" +
         "java_library(name = 'bar')\n",
         testBuildFile,
         Charsets.UTF_8);
+
     tempDir.newFile("bar.py");
 
     // Create a temp directory with some build files.
@@ -126,10 +149,18 @@
     };
   }
 
+  private Parser createParser(Map<BuildTarget, BuildRuleBuilder<?>> knownBuildTargets) {
+    return createParser(
+        Suppliers.ofInstance(BuildFileTree.constructBuildFileTree(filesystem)),
+        knownBuildTargets,
+        new TestProjectBuildFileParserFactory(filesystem),
+        new BuildTargetParser(filesystem));
+  }
+
   private Parser createParser(Map<BuildTarget, BuildRuleBuilder<?>> knownBuildTargets,
                               ProjectBuildFileParserFactory buildFileParserFactory) {
     return createParser(
-        Suppliers.ofInstance(new BuildFileTree(ImmutableSet.<String>of())),
+        Suppliers.ofInstance(BuildFileTree.constructBuildFileTree(filesystem)),
         knownBuildTargets,
         buildFileParserFactory,
         new BuildTargetParser(filesystem));
@@ -374,195 +405,464 @@
   }
 
   @Test
-  @SuppressWarnings("unchecked") // Needed to mock generic WatchEvent class.
   public void whenNotifiedOfNonPathEventThenCacheRulesAreInvalidated()
       throws BuildFileParseException, NoSuchBuildTargetException {
     TestProjectBuildFileParserFactory buildFileParserFactory =
         new TestProjectBuildFileParserFactory(filesystem);
     Parser parser = createParser(emptyBuildTargets(), buildFileParserFactory);
 
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
-    WatchEvent<Object> event = createMock(WatchEvent.class);
-    expect(event.kind()).andReturn(StandardWatchEventKinds.OVERFLOW).anyTimes();
-    replay(event);
-    parser.onFileSystemChange(event);
+    // Call filterAllTargetsInProject to populate the cache.
     parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
 
-    verify(event);
+    // Process event.
+    WatchEvent<Object> event = createOverflowEvent();
+    parser.onFileSystemChange(event);
+
+    // Call filterAllTargetsInProject to request cached rules.
+    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
+
+    // Test that the second parseBuildFile call repopulated the cache.
+    assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
+  }
+
+  // TODO(jimp/devjasta): clean up the horrible ProjectBuildFileParserFactory mess.
+  private void parseBuildFile(File buildFile, Parser parser,
+                              ProjectBuildFileParserFactory buildFileParserFactory)
+      throws BuildFileParseException, NoSuchBuildTargetException {
+    parser.parseBuildFile(buildFile,
+        /* defaultIncludes */ ImmutableList.<String>of(),
+        buildFileParserFactory.createParser(/* commonIncludes */ Lists.<String>newArrayList()));
+  }
+
+  private WatchEvent<Object> createOverflowEvent() {
+    return new WatchEvent<Object>() {
+      @Override
+      public Kind<Object> kind() {
+        return StandardWatchEventKinds.OVERFLOW;
+      }
+
+      @Override
+      public int count() {
+        return 0;
+      }
+
+      @Override
+      public Object context() {
+        return null;
+      }
+    };
+  }
+
+  private WatchEvent<Path> createEvent(final File file, final WatchEvent.Kind<Path> kind) {
+    return new WatchEvent<Path>() {
+      @Override
+      public Kind<Path> kind() {
+        return kind;
+      }
+
+      @Override
+      public int count() {
+        return 0;
+      }
+
+      @Override
+      public Path context() {
+        return file.toPath();
+      }
+    };
+  }
+
+  @Test
+  public void whenNotifiedOfBuildFileAddThenCacheRulesAreInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
+
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Process event.
+    WatchEvent<Path> event = createEvent(testBuildFile, StandardWatchEventKinds.ENTRY_CREATE);
+    parser.onFileSystemChange(event);
+
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call repopulated the cache.
     assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
   }
 
   @Test
-  @SuppressWarnings("unchecked") // Needed to mock generic WatchEvent class.
-  public void whenNotifiedOfNonSourcePathEventThenCacheRulesAreInvalidated()
+  public void whenNotifiedOfBuildFileChangeThenCacheRulesAreInvalidated()
       throws BuildFileParseException, NoSuchBuildTargetException {
     TestProjectBuildFileParserFactory buildFileParserFactory =
         new TestProjectBuildFileParserFactory(filesystem);
-    Parser parser = createParser(emptyBuildTargets(), buildFileParserFactory);
+    Parser parser = createParser(emptyBuildTargets());
 
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
-    WatchEvent<Path> event = createMock(WatchEvent.class);
-    expect(event.kind()).andReturn(StandardWatchEventKinds.ENTRY_MODIFY).anyTimes();
-    expect(event.context()).andReturn(new File(BuckConstant.BUILD_RULES_FILE_NAME).toPath());
-    replay(event);
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Process event.
+    WatchEvent<Path> event = createEvent(testBuildFile, StandardWatchEventKinds.ENTRY_MODIFY);
     parser.onFileSystemChange(event);
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
 
-    verify(event);
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call repopulated the cache.
     assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
   }
 
   @Test
-  @SuppressWarnings("unchecked") // Needed to mock generic WatchEvent class.
-  public void whenNotifiedOfSourcePathEventThenCacheRulesAreNotInvalidated()
+  public void whenNotifiedOfBuildFileDeleteThenCacheRulesAreInvalidated()
       throws BuildFileParseException, NoSuchBuildTargetException {
     TestProjectBuildFileParserFactory buildFileParserFactory =
         new TestProjectBuildFileParserFactory(filesystem);
-    Parser parser = createParser(emptyBuildTargets(), buildFileParserFactory);
+    Parser parser = createParser(emptyBuildTargets());
 
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
-    WatchEvent<Path> event = createMock(WatchEvent.class);
-    expect(event.kind()).andReturn(StandardWatchEventKinds.ENTRY_MODIFY).anyTimes();
-    expect(event.context()).andReturn(new File("./SomeClass.java").toPath());
-    replay(event);
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Process event.
+    WatchEvent<Path> event = createEvent(testBuildFile, StandardWatchEventKinds.ENTRY_DELETE);
     parser.onFileSystemChange(event);
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
 
-    verify(event);
-    assertEquals("Should have cached build rules.", 1, buildFileParserFactory.calls);
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call repopulated the cache.
+    assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
   }
 
   @Test
-  @SuppressWarnings("unchecked") // Needed to mock generic WatchEvent and Supplier classes.
-  public void whenNotifiedOfNewSourceFileBuildTreeIsNotReconstructed()
-      throws IOException, NoSuchBuildTargetException, BuildFileParseException {
+  public void whenNotifiedOfIncludeFileAddThenCacheRulesAreInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
 
-    Supplier<BuildFileTree> buildFileTreeSupplier = createStrictMock(Supplier.class);
-    expect(buildFileTreeSupplier.get()).andReturn(new BuildFileTree(ImmutableSet.<String>of()));
-    replay(buildFileTreeSupplier);
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
 
-    Parser parser = createParser(buildFileTreeSupplier,
-        emptyBuildTargets(),
-        new DefaultProjectBuildFileParserFactory(filesystem, BuckTestConstant.PYTHON_INTERPRETER),
-        new BuildTargetParser(filesystem));
-
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
-    WatchEvent<Path> event = createMock(WatchEvent.class);
-    expect(event.kind()).andReturn(StandardWatchEventKinds.ENTRY_CREATE).anyTimes();
-    expect(event.context()).andReturn(new File("./SomeClass.java").toPath());
-    replay(event);
+    // Process event.
+    WatchEvent<Path> event = createEvent(includedByBuildFile, StandardWatchEventKinds.ENTRY_CREATE);
     parser.onFileSystemChange(event);
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
 
-    // Check that event was processed and BuildFileTree was supplied once.
-    verify(event, buildFileTreeSupplier);
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call repopulated the cache.
+    assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
   }
 
   @Test
-  @SuppressWarnings("unchecked") // Needed to mock generic WatchEvent and Supplier classes.
-  public void whenNotifiedOfNewBuildFileBuildTreeIsReconstructed()
-      throws IOException, NoSuchBuildTargetException, BuildFileParseException {
+  public void whenNotifiedOfIncludeFileChangeThenCacheRulesAreInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
 
-    Supplier<BuildFileTree> buildFileTreeSupplier = createStrictMock(Supplier.class);
-    expect(buildFileTreeSupplier.get())
-        .andReturn(new BuildFileTree(ImmutableSet.<String>of())).times(2);
-    replay(buildFileTreeSupplier);
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
 
-    Parser parser = createParser(buildFileTreeSupplier,
-        emptyBuildTargets(),
-        new DefaultProjectBuildFileParserFactory(filesystem, BuckTestConstant.PYTHON_INTERPRETER),
-        new BuildTargetParser(filesystem));
-
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
-    WatchEvent<Path> event = createMock(WatchEvent.class);
-    expect(event.kind()).andReturn(StandardWatchEventKinds.ENTRY_CREATE).anyTimes();
-    expect(event.context()).andReturn(new File(BuckConstant.BUILD_RULES_FILE_NAME).toPath());
-    replay(event);
+    // Process event.
+    WatchEvent<Path> event = createEvent(includedByBuildFile, StandardWatchEventKinds.ENTRY_MODIFY);
     parser.onFileSystemChange(event);
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
 
-    // Check that event was processed and BuildFileTree was supplied twice.
-    verify(event, buildFileTreeSupplier);
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call repopulated the cache.
+    assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
   }
 
   @Test
-  @SuppressWarnings("unchecked") // Needed to mock generic WatchEvent and Supplier classes.
-  public void whenNotifiedOfMultipleNewBuildFilesBuildTreeIsReconstructedOnce()
-      throws IOException, NoSuchBuildTargetException, BuildFileParseException {
+  public void whenNotifiedOfIncludeFileDeleteThenCacheRulesAreInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
 
-    Supplier<BuildFileTree> buildFileTreeSupplier = createStrictMock(Supplier.class);
-    expect(buildFileTreeSupplier.get())
-        .andReturn(new BuildFileTree(ImmutableSet.<String>of())).times(2);
-    replay(buildFileTreeSupplier);
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
 
-    Parser parser = createParser(buildFileTreeSupplier,
-        emptyBuildTargets(),
-        new DefaultProjectBuildFileParserFactory(filesystem, BuckTestConstant.PYTHON_INTERPRETER),
-        new BuildTargetParser(filesystem));
-
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
-    WatchEvent<Path> event = createMock(WatchEvent.class);
-    expect(event.kind()).andReturn(StandardWatchEventKinds.ENTRY_CREATE).anyTimes();
-    expect(event.context())
-        .andReturn(new File(BuckConstant.BUILD_RULES_FILE_NAME).toPath()).anyTimes();
-    replay(event);
+    // Process event.
+    WatchEvent<Path> event = createEvent(includedByBuildFile, StandardWatchEventKinds.ENTRY_DELETE);
     parser.onFileSystemChange(event);
-    parser.onFileSystemChange(event);
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
 
-    // Check that event was processed and BuildFileTree was supplied twice.
-    verify(event, buildFileTreeSupplier);
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call repopulated the cache.
+    assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
   }
 
   @Test
-  @SuppressWarnings("unchecked") // Needed to mock generic WatchEvent and Supplier classes.
-  public void whenNotifiedOfBuildFileChangeBuildTreeIsNotReconstructed()
-      throws IOException, NoSuchBuildTargetException, BuildFileParseException {
+  public void whenNotifiedOf2ndOrderIncludeFileAddThenCacheRulesAreInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
 
-    Supplier<BuildFileTree> buildFileTreeSupplier = createStrictMock(Supplier.class);
-    expect(buildFileTreeSupplier.get()).andReturn(new BuildFileTree(ImmutableSet.<String>of()));
-    replay(buildFileTreeSupplier);
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
 
-    Parser parser = createParser(buildFileTreeSupplier,
-        emptyBuildTargets(),
-        new DefaultProjectBuildFileParserFactory(filesystem, BuckTestConstant.PYTHON_INTERPRETER),
-        new BuildTargetParser(filesystem));
-
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
-    WatchEvent<Path> event = createMock(WatchEvent.class);
-    expect(event.kind()).andReturn(StandardWatchEventKinds.ENTRY_MODIFY).anyTimes();
-    expect(event.context()).andReturn(new File(BuckConstant.BUILD_RULES_FILE_NAME).toPath());
-    replay(event);
+    // Process event.
+    WatchEvent<Path> event = createEvent(includedByIncludeFile,
+        StandardWatchEventKinds.ENTRY_CREATE);
     parser.onFileSystemChange(event);
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
 
-    // Check that event was processed and BuildFileTree was supplied once.
-    verify(event, buildFileTreeSupplier);
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call repopulated the cache.
+    assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
   }
 
   @Test
-  @SuppressWarnings("unchecked") // Needed to mock generic WatchEvent and Supplier classes.
-  public void whenNotifiedOfSourceFileChangeBuildTreeIsNotReconstructed()
-      throws IOException, NoSuchBuildTargetException, BuildFileParseException {
+  public void whenNotifiedOf2ndOrderIncludeFileChangeThenCacheRulesAreInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
 
-    Supplier<BuildFileTree> buildFileTreeSupplier = createStrictMock(Supplier.class);
-    expect(buildFileTreeSupplier.get()).andReturn(new BuildFileTree(ImmutableSet.<String>of()));
-    replay(buildFileTreeSupplier);
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
 
-    Parser parser = createParser(buildFileTreeSupplier,
-        emptyBuildTargets(),
-        new DefaultProjectBuildFileParserFactory(filesystem, BuckTestConstant.PYTHON_INTERPRETER),
-        new BuildTargetParser(filesystem));
-
-    WatchEvent<Path> event = createMock(WatchEvent.class);
-    expect(event.kind()).andReturn(StandardWatchEventKinds.ENTRY_MODIFY).anyTimes();
-    expect(event.context()).andReturn(new File("./SomeClass.java").toPath());
-    replay(event);
+    // Process event.
+    WatchEvent<Path> event = createEvent(includedByIncludeFile,
+        StandardWatchEventKinds.ENTRY_MODIFY);
     parser.onFileSystemChange(event);
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
 
-    // Check that event was processed and BuildFileTree was supplied once.
-    verify(event, buildFileTreeSupplier);
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call repopulated the cache.
+    assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
+  }
+
+  @Test
+  public void whenNotifiedOf2ndOrderIncludeFileDeleteThenCacheRulesAreInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
+
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Process event.
+    WatchEvent<Path> event = createEvent(includedByIncludeFile,
+        StandardWatchEventKinds.ENTRY_DELETE);
+    parser.onFileSystemChange(event);
+
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call repopulated the cache.
+    assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
+  }
+
+  @Test
+  public void whenNotifiedOfDefaultIncludeFileAddThenCacheRulesAreInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
+
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Process event.
+    WatchEvent<Path> event = createEvent(defaultIncludeFile,
+        StandardWatchEventKinds.ENTRY_CREATE);
+    parser.onFileSystemChange(event);
+
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call repopulated the cache.
+    assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
+  }
+
+  @Test
+  public void whenNotifiedOfDefaultIncludeFileChangeThenCacheRulesAreInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
+
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Process event.
+    WatchEvent<Path> event = createEvent(defaultIncludeFile,
+        StandardWatchEventKinds.ENTRY_MODIFY);
+    parser.onFileSystemChange(event);
+
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call repopulated the cache.
+    assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
+
+  }
+
+  @Test
+  public void whenNotifiedOfDefaultIncludeFileDeleteThenCacheRulesAreInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
+
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Process event.
+    WatchEvent<Path> event = createEvent(defaultIncludeFile,
+        StandardWatchEventKinds.ENTRY_DELETE);
+    parser.onFileSystemChange(event);
+
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call repopulated the cache.
+    assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
+  }
+
+  @Test
+  // TODO(user): avoid invalidation when arbitrary contained (possibly backup) files are added.
+  public void whenNotifiedOfContainedFileAddThenCacheRulesAreInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException, IOException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
+
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Process event.
+    WatchEvent<Path> event = createEvent(tempDir.newFile("java/com/facebook/SomeClass.java"),
+        StandardWatchEventKinds.ENTRY_CREATE);
+    parser.onFileSystemChange(event);
+
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call repopulated the cache.
+    assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
+  }
+
+  @Test
+  public void whenNotifiedOfContainedFileChangeThenCacheRulesAreNotInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException, IOException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
+
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Process event.
+    WatchEvent<Path> event = createEvent(tempDir.newFile("java/com/facebook/SomeClass.java"),
+        StandardWatchEventKinds.ENTRY_MODIFY);
+    parser.onFileSystemChange(event);
+
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call did not repopulate the cache.
+    assertEquals("Should have not invalidated cache.", 1, buildFileParserFactory.calls);
+  }
+
+  @Test
+  // TODO(user): avoid invalidation when arbitrary contained (possibly backup) files are deleted.
+  public void whenNotifiedOfContainedFileDeleteThenCacheRulesAreInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException, IOException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
+
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Process event.
+    WatchEvent<Path> event = createEvent(tempDir.newFile("java/com/facebook/SomeClass.java"),
+        StandardWatchEventKinds.ENTRY_DELETE);
+    parser.onFileSystemChange(event);
+
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call repopulated the cache.
+    assertEquals("Should have invalidated cache.", 2, buildFileParserFactory.calls);
+  }
+
+  @Test
+  public void whenNotifiedOfUnrelatedFileAddThenCacheRulesAreNotInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException, IOException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
+
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Process event.
+    WatchEvent<Path> event = createEvent(tempDir.newFile("SomeClass.java__backup"),
+        StandardWatchEventKinds.ENTRY_CREATE);
+    parser.onFileSystemChange(event);
+
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call did not repopulate the cache.
+    assertEquals("Should have not invalidated cache.", 1, buildFileParserFactory.calls);
+  }
+
+  @Test
+  public void whenNotifiedOfUnrelatedFileChangeThenCacheRulesAreNotInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException, IOException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
+
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Process event.
+    WatchEvent<Path> event = createEvent(tempDir.newFile("SomeClass.java__backup"),
+        StandardWatchEventKinds.ENTRY_MODIFY);
+    parser.onFileSystemChange(event);
+
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call did not repopulate the cache.
+    assertEquals("Should have not invalidated cache.", 1, buildFileParserFactory.calls);
+  }
+
+  @Test
+  public void whenNotifiedOfUnrelatedFileDeleteThenCacheRulesAreNotInvalidated()
+      throws BuildFileParseException, NoSuchBuildTargetException, IOException {
+    TestProjectBuildFileParserFactory buildFileParserFactory =
+        new TestProjectBuildFileParserFactory(filesystem);
+    Parser parser = createParser(emptyBuildTargets());
+
+    // Call parseBuildFile to populate the cache.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Process event.
+    WatchEvent<Path> event = createEvent(tempDir.newFile("SomeClass.java__backup"),
+        StandardWatchEventKinds.ENTRY_DELETE);
+    parser.onFileSystemChange(event);
+
+    // Call parseBuildFile to request cached rules.
+    parseBuildFile(testBuildFile, parser, buildFileParserFactory);
+
+    // Test that the second parseBuildFile call did not repopulate the cache.
+    assertEquals("Should have not invalidated cache.", 1, buildFileParserFactory.calls);
   }
 
   @Test
@@ -603,66 +903,6 @@
   }
 
   @Test
-  @SuppressWarnings("unchecked") // Needed to mock generic WatchEvent class.
-  public void whenSourceFileAddedThenCacheRulesAreInvalidated()
-      throws BuildFileParseException, NoSuchBuildTargetException {
-    TestProjectBuildFileParserFactory buildFileParserFactory =
-        new TestProjectBuildFileParserFactory(filesystem);
-    Parser parser = createParser(emptyBuildTargets(), buildFileParserFactory);
-
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
-    WatchEvent<Path> event = createMock(WatchEvent.class);
-    expect(event.kind()).andReturn(StandardWatchEventKinds.ENTRY_CREATE).anyTimes();
-    expect(event.context()).andReturn(new File("./SomeClass.java").toPath());
-    replay(event);
-    parser.onFileSystemChange(event);
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
-
-    verify(event);
-    assertEquals("Should not have cached build rules.", 2, buildFileParserFactory.calls);
-  }
-
-  @Test
-  @SuppressWarnings("unchecked") // Needed to mock generic WatchEvent class.
-  public void whenSourceFileModifiedThenCacheRulesAreNotInvalidated()
-      throws BuildFileParseException, NoSuchBuildTargetException {
-    TestProjectBuildFileParserFactory buildFileParserFactory =
-        new TestProjectBuildFileParserFactory(filesystem);
-    Parser parser = createParser(emptyBuildTargets(), buildFileParserFactory);
-
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
-    WatchEvent<Path> event = createMock(WatchEvent.class);
-    expect(event.kind()).andReturn(StandardWatchEventKinds.ENTRY_MODIFY).anyTimes();
-    expect(event.context()).andReturn(new File("./SomeClass.java").toPath());
-    replay(event);
-    parser.onFileSystemChange(event);
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
-
-    verify(event);
-    assertEquals("Should have cached build rules.", 1, buildFileParserFactory.calls);
-  }
-
-  @Test
-  @SuppressWarnings("unchecked") // Needed to mock generic WatchEvent class.
-  public void whenSourceFileDeletedThenCacheRulesAreInvalidated()
-      throws BuildFileParseException, NoSuchBuildTargetException {
-    TestProjectBuildFileParserFactory buildFileParserFactory =
-        new TestProjectBuildFileParserFactory(filesystem);
-    Parser parser = createParser(emptyBuildTargets(), buildFileParserFactory);
-
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
-    WatchEvent<Path> event = createMock(WatchEvent.class);
-    expect(event.kind()).andReturn(StandardWatchEventKinds.ENTRY_DELETE).anyTimes();
-    expect(event.context()).andReturn(new File("./SomeClass.java").toPath());
-    replay(event);
-    parser.onFileSystemChange(event);
-    parser.filterAllTargetsInProject(filesystem, Lists.<String>newArrayList(), alwaysTrue());
-
-    verify(event);
-    assertEquals("Should not have cached build rules.", 2, buildFileParserFactory.calls);
-  }
-
-  @Test
   public void whenAllRulesAreRequestedWithDifferingIncludesThenRulesAreParsedTwice()
       throws BuildFileParseException, NoSuchBuildTargetException {
     TestProjectBuildFileParserFactory buildFileParserFactory =
@@ -784,7 +1024,7 @@
 
     private class TestProjectBuildFileParser extends ProjectBuildFileParser {
       public TestProjectBuildFileParser() {
-        super(projectFilesystem, ImmutableList.<String>of(), "python");
+        super(projectFilesystem, ImmutableList.of("//java/com/facebook/defaultIncludeFile"), "python");
       }
 
       @Override