Complete ninja_graph, and allow its usage under the experimental flag.

- add output_groups attribute for specifying files to be build
- add NinjaRulesModule, and register it
- correct error handling, explicitly verify orphan artifacts so that
a nice message was displayed and not internal error exception
- add integration tests
- remove NinjaGraphProvider as not needed any more
- some code cleanup

Closes #10710.

PiperOrigin-RevId: 293578222
diff --git a/src/main/java/com/google/devtools/build/lib/BUILD b/src/main/java/com/google/devtools/build/lib/BUILD
index ce92199..0ea5446 100644
--- a/src/main/java/com/google/devtools/build/lib/BUILD
+++ b/src/main/java/com/google/devtools/build/lib/BUILD
@@ -662,12 +662,17 @@
     name = "bazel-ninja",
     srcs = glob(["bazel/rules/ninja/**/*.java"]),
     deps = [
+        ":packages-internal",
+        ":transitive-info-provider",
         "//src/main/java/com/google/devtools/build/lib:build-base",
         "//src/main/java/com/google/devtools/build/lib:packages",
+        "//src/main/java/com/google/devtools/build/lib:runtime",
         "//src/main/java/com/google/devtools/build/lib:util",
         "//src/main/java/com/google/devtools/build/lib/actions",
+        "//src/main/java/com/google/devtools/build/lib/cmdline",
         "//src/main/java/com/google/devtools/build/lib/collect/nestedset",
         "//src/main/java/com/google/devtools/build/lib/concurrent",
+        "//src/main/java/com/google/devtools/build/lib/skylarkbuildapi",
         "//src/main/java/com/google/devtools/build/lib/vfs",
         "//src/main/java/com/google/devtools/build/lib/vfs:pathfragment",
         "//src/main/java/com/google/devtools/build/skyframe",
diff --git a/src/main/java/com/google/devtools/build/lib/bazel/Bazel.java b/src/main/java/com/google/devtools/build/lib/bazel/Bazel.java
index eb3d303..7c40c11 100644
--- a/src/main/java/com/google/devtools/build/lib/bazel/Bazel.java
+++ b/src/main/java/com/google/devtools/build/lib/bazel/Bazel.java
@@ -61,6 +61,7 @@
           com.google.devtools.build.lib.sandbox.SandboxModule.class,
           com.google.devtools.build.lib.runtime.BuildSummaryStatsModule.class,
           com.google.devtools.build.lib.dynamic.DynamicExecutionModule.class,
+          com.google.devtools.build.lib.bazel.rules.ninja.actions.NinjaRulesModule.class,
           com.google.devtools.build.lib.bazel.rules.BazelRulesModule.class,
           com.google.devtools.build.lib.bazel.rules.BazelStrategyModule.class,
           com.google.devtools.build.lib.network.NoOpConnectivityModule.class,
diff --git a/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraph.java b/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraph.java
index 37fed0a..9c049a6 100644
--- a/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraph.java
+++ b/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraph.java
@@ -17,11 +17,12 @@
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.ImmutableSortedMap;
 import com.google.common.collect.ImmutableSortedSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.devtools.build.lib.actions.Artifact;
 import com.google.devtools.build.lib.actions.FileValue;
@@ -38,8 +39,9 @@
 import com.google.devtools.build.lib.bazel.rules.ninja.parser.NinjaTarget;
 import com.google.devtools.build.lib.bazel.rules.ninja.pipeline.NinjaPipeline;
 import com.google.devtools.build.lib.collect.nestedset.NestedSet;
+import com.google.devtools.build.lib.collect.nestedset.NestedSetBuilder;
+import com.google.devtools.build.lib.collect.nestedset.Order;
 import com.google.devtools.build.lib.concurrent.ExecutorUtil;
-import com.google.devtools.build.lib.packages.RuleClass.ConfiguredTargetFactory.RuleErrorException;
 import com.google.devtools.build.lib.packages.Type;
 import com.google.devtools.build.lib.repository.ExternalPackageUtil;
 import com.google.devtools.build.lib.vfs.Path;
@@ -50,9 +52,9 @@
 import com.google.devtools.build.skyframe.SkyKey;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.TreeMap;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.stream.Collectors;
 
@@ -77,20 +79,172 @@
     Artifact mainArtifact = ruleContext.getPrerequisiteArtifact("main", Mode.TARGET);
     ImmutableList<Artifact> ninjaSrcs =
         ruleContext.getPrerequisiteArtifacts("ninja_srcs", Mode.TARGET).list();
+    PathFragment outputRoot =
+        PathFragment.create(ruleContext.attributes().get("output_root", Type.STRING));
+    PathFragment workingDirectory =
+        PathFragment.create(ruleContext.attributes().get("working_directory", Type.STRING));
+    List<String> outputRootInputs =
+        ruleContext.attributes().get("output_root_inputs", Type.STRING_LIST);
+    Map<String, List<String>> outputGroupsMap =
+        ruleContext.attributes().get("output_groups", Type.STRING_LIST_DICT);
 
     Environment env = ruleContext.getAnalysisEnvironment().getSkyframeEnv();
     establishDependencyOnNinjaFiles(env, mainArtifact, ninjaSrcs);
-    ImmutableSortedSet<String> notSymlinkedDirs =
-        ExternalPackageUtil.getNotSymlinkedInExecrootDirectories(env);
-    if (env.valuesMissing()) {
+    checkDirectoriesAttributes(ruleContext, outputRoot, workingDirectory);
+
+    if (env.valuesMissing() || ruleContext.hasErrors()) {
       return null;
     }
 
-    List<Path> childNinjaFiles =
-        ninjaSrcs.stream().map(Artifact::getPath).collect(Collectors.toList());
+    Root sourceRoot = mainArtifact.getRoot().getRoot();
+    List<PathFragment> pathsToBuild =
+        outputGroupsMap.values().stream()
+            .flatMap(List::stream)
+            .map(PathFragment::create)
+            .collect(Collectors.toList());
+    NinjaGraphArtifactsHelper artifactsHelper =
+        new NinjaGraphArtifactsHelper(
+            ruleContext,
+            sourceRoot,
+            outputRoot,
+            workingDirectory,
+            createSrcsMap(ruleContext),
+            createDepsMap(ruleContext),
+            pathsToBuild);
+    if (ruleContext.hasErrors()) {
+      return null;
+    }
+    TargetsPreparer targetsPreparer = new TargetsPreparer();
 
-    PathFragment outputRoot =
-        PathFragment.create(ruleContext.attributes().get("output_root", Type.STRING));
+    try {
+      List<Path> childNinjaFiles =
+          ninjaSrcs.stream().map(Artifact::getPath).collect(Collectors.toList());
+      Path workspace =
+          Preconditions.checkNotNull(ruleContext.getConfiguration())
+              .getDirectories()
+              .getWorkspace();
+      String ownerTargetName = ruleContext.getLabel().getName();
+      List<NinjaTarget> ninjaTargets =
+          new NinjaPipeline(
+                  workspace.getRelative(workingDirectory),
+                  MoreExecutors.listeningDecorator(NINJA_POOL),
+                  childNinjaFiles,
+                  ownerTargetName)
+              .pipeline(mainArtifact.getPath());
+      targetsPreparer.process(ninjaTargets, artifactsHelper);
+      new NinjaActionsHelper(
+              ruleContext,
+              artifactsHelper,
+              outputRootInputs,
+              targetsPreparer.getUsualTargets(),
+              targetsPreparer.getPhonyTargetsMap())
+          .process();
+    } catch (GenericParsingException | IOException e) {
+      // IOException is possible with reading Ninja file, describing the action graph.
+      ruleContext.ruleError(e.getMessage());
+      return null;
+    }
+
+    if (!checkOrphanArtifacts(ruleContext)) {
+      return null;
+    }
+
+    NestedSetBuilder<Artifact> filesToBuild = NestedSetBuilder.stableOrder();
+    TreeMap<String, NestedSet<Artifact>> groups = Maps.newTreeMap();
+    for (Map.Entry<String, List<String>> entry : outputGroupsMap.entrySet()) {
+      NestedSet<Artifact> artifacts =
+          getGroupArtifacts(
+              ruleContext,
+              entry.getValue(),
+              targetsPreparer.getPhonyTargetsMap(),
+              artifactsHelper.getOutputsMap());
+      groups.put(entry.getKey(), artifacts);
+      filesToBuild.addTransitive(artifacts);
+    }
+
+    if (ruleContext.hasErrors()) {
+      return null;
+    }
+
+    return new RuleConfiguredTargetBuilder(ruleContext)
+        .addProvider(RunfilesProvider.class, RunfilesProvider.EMPTY)
+        .setFilesToBuild(filesToBuild.build())
+        .addOutputGroups(groups)
+        .build();
+  }
+
+  private static boolean checkOrphanArtifacts(RuleContext ruleContext) {
+    ImmutableSet<Artifact> orphanArtifacts =
+        ruleContext.getAnalysisEnvironment().getOrphanArtifacts();
+    if (!orphanArtifacts.isEmpty()) {
+      List<String> paths =
+          orphanArtifacts.stream().map(Artifact::getExecPathString).collect(Collectors.toList());
+      ruleContext.ruleError(
+          "The following artifacts do not have a generating action in Ninja file: "
+              + String.join(", ", paths));
+      return false;
+    }
+    return true;
+  }
+
+  private static class TargetsPreparer {
+    private ImmutableSortedMap<PathFragment, NinjaTarget> usualTargets;
+    private ImmutableSortedMap<PathFragment, NestedSet<Artifact>> phonyTargetsMap;
+
+    public ImmutableSortedMap<PathFragment, NinjaTarget> getUsualTargets() {
+      return usualTargets;
+    }
+
+    public ImmutableSortedMap<PathFragment, NestedSet<Artifact>> getPhonyTargetsMap() {
+      return phonyTargetsMap;
+    }
+
+    void process(List<NinjaTarget> ninjaTargets, NinjaGraphArtifactsHelper artifactsHelper)
+        throws GenericParsingException {
+      ImmutableSortedMap.Builder<PathFragment, NinjaTarget> usualTargetsBuilder =
+          ImmutableSortedMap.naturalOrder();
+      ImmutableSortedMap.Builder<PathFragment, NinjaTarget> phonyTargetsBuilder =
+          ImmutableSortedMap.naturalOrder();
+      separatePhonyTargets(ninjaTargets, usualTargetsBuilder, phonyTargetsBuilder);
+      usualTargets = usualTargetsBuilder.build();
+      phonyTargetsMap =
+          NinjaPhonyTargetsUtil.getPhonyPathsMap(
+              phonyTargetsBuilder.build(), artifactsHelper::getInputArtifact);
+    }
+
+    private static void separatePhonyTargets(
+        List<NinjaTarget> ninjaTargets,
+        ImmutableSortedMap.Builder<PathFragment, NinjaTarget> usualTargetsBuilder,
+        ImmutableSortedMap.Builder<PathFragment, NinjaTarget> phonyTargetsBuilder)
+        throws GenericParsingException {
+      for (NinjaTarget target : ninjaTargets) {
+        if ("phony".equals(target.getRuleName())) {
+          if (target.getAllOutputs().size() != 1) {
+            String allOutputs =
+                target.getAllOutputs().stream()
+                    .map(PathFragment::getPathString)
+                    .collect(Collectors.joining(" "));
+            throw new GenericParsingException(
+                String.format(
+                    "Ninja phony alias can only be used for single output, but found '%s'.",
+                    allOutputs));
+          }
+          phonyTargetsBuilder.put(Iterables.getOnlyElement(target.getAllOutputs()), target);
+        } else {
+          for (PathFragment output : target.getAllOutputs()) {
+            usualTargetsBuilder.put(output, target);
+          }
+        }
+      }
+    }
+  }
+
+  private void checkDirectoriesAttributes(
+      RuleContext ruleContext, PathFragment outputRoot, PathFragment workingDirectory)
+      throws InterruptedException {
+    Environment env = ruleContext.getAnalysisEnvironment().getSkyframeEnv();
+    ImmutableSortedSet<String> notSymlinkedDirs =
+        ExternalPackageUtil.getNotSymlinkedInExecrootDirectories(env);
     // We can compare strings because notSymlinkedDirs contains normalized directory names
     if (!notSymlinkedDirs.contains(outputRoot.getPathString())) {
       ruleContext.attributeError(
@@ -101,8 +255,6 @@
               outputRoot.getPathString()));
     }
 
-    PathFragment workingDirectory =
-        PathFragment.create(ruleContext.attributes().get("working_directory", Type.STRING));
     if (!workingDirectory.isEmpty() && !workingDirectory.equals(outputRoot)) {
       ruleContext.attributeError(
           "working_directory",
@@ -111,63 +263,30 @@
                   + " or be the same as output root '%s'.",
               workingDirectory.getPathString(), outputRoot.getPathString()));
     }
+  }
 
-    if (ruleContext.hasErrors()) {
-      return null;
+  private static NestedSet<Artifact> getGroupArtifacts(
+      RuleContext ruleContext,
+      List<String> targets,
+      ImmutableSortedMap<PathFragment, NestedSet<Artifact>> phonyTargetsMap,
+      ImmutableSortedMap<PathFragment, Artifact> outputsMap) {
+    NestedSetBuilder<Artifact> nestedSetBuilder = NestedSetBuilder.stableOrder();
+    for (String target : targets) {
+      PathFragment path = PathFragment.create(target);
+      NestedSet<Artifact> artifacts = phonyTargetsMap.get(path);
+      if (artifacts != null) {
+        nestedSetBuilder.addTransitive(artifacts);
+      } else {
+        Artifact usualArtifact = outputsMap.get(path);
+        if (usualArtifact == null) {
+          ruleContext.ruleError(
+              String.format("Required target '%s' is not created in ninja_graph.", path));
+          return NestedSetBuilder.emptySet(Order.STABLE_ORDER);
+        }
+        nestedSetBuilder.add(usualArtifact);
+      }
     }
-
-    List<String> outputRootInputs =
-        ruleContext.attributes().get("output_root_inputs", Type.STRING_LIST);
-
-    Path workspace =
-        Preconditions.checkNotNull(ruleContext.getConfiguration()).getDirectories().getWorkspace();
-    String ownerTargetName = ruleContext.getLabel().getName();
-    List<NinjaTarget> ninjaTargets =
-        readNinjaGraph(mainArtifact, childNinjaFiles, workingDirectory, workspace, ownerTargetName);
-
-    ImmutableSortedMap.Builder<PathFragment, NinjaTarget> usualTargetsBuilder =
-        ImmutableSortedMap.naturalOrder();
-    ImmutableSortedMap.Builder<PathFragment, NinjaTarget> phonyTargetsBuilder =
-        ImmutableSortedMap.naturalOrder();
-    separatePhonyTargets(ninjaTargets, usualTargetsBuilder, phonyTargetsBuilder);
-    ImmutableSortedMap<PathFragment, NinjaTarget> usualTargets = usualTargetsBuilder.build();
-    ImmutableSortedMap<PathFragment, NestedSet<Artifact>> phonyTargetsMap;
-
-    Root sourceRoot = mainArtifact.getRoot().getRoot();
-    ImmutableSortedMap<PathFragment, Artifact> depsMap = createDepsMap(ruleContext);
-    if (ruleContext.hasErrors()) {
-      return null;
-    }
-    NinjaGraphArtifactsHelper artifactsHelper =
-        new NinjaGraphArtifactsHelper(
-            ruleContext,
-            sourceRoot,
-            outputRoot,
-            workingDirectory,
-            createSrcsMap(ruleContext),
-            depsMap);
-    try {
-      phonyTargetsMap =
-          NinjaPhonyTargetsUtil.getPhonyPathsMap(
-              phonyTargetsBuilder.build(), artifactsHelper::getInputArtifact);
-      NinjaActionsHelper ninjaActionsHelper =
-          new NinjaActionsHelper(
-              ruleContext, artifactsHelper, outputRootInputs, usualTargets, phonyTargetsMap);
-      ninjaActionsHelper.process();
-    } catch (GenericParsingException e) {
-      throw new RuleErrorException(e.getMessage());
-    }
-
-    NinjaGraphProvider graphProvider =
-        new NinjaGraphProvider(
-            outputRoot, workingDirectory, phonyTargetsMap, artifactsHelper.getOutputsMap());
-
-    RuleConfiguredTargetBuilder builder = new RuleConfiguredTargetBuilder(ruleContext);
-    builder
-        .addProvider(NinjaGraphProvider.class, graphProvider)
-        .addProvider(RunfilesProvider.class, RunfilesProvider.EMPTY);
-
-    return builder.build();
+    return nestedSetBuilder.build();
   }
 
   private static ImmutableSortedMap<PathFragment, Artifact> createSrcsMap(RuleContext ruleContext) {
@@ -218,48 +337,4 @@
     return FileValue.key(
         RootedPath.toRootedPath(artifact.getRoot().getRoot(), artifact.getRootRelativePath()));
   }
-
-  private static void separatePhonyTargets(
-      List<NinjaTarget> ninjaTargets,
-      ImmutableSortedMap.Builder<PathFragment, NinjaTarget> usualTargetsBuilder,
-      ImmutableSortedMap.Builder<PathFragment, NinjaTarget> phonyTargetsBuilder)
-      throws RuleErrorException {
-    for (NinjaTarget target : ninjaTargets) {
-      if ("phony".equals(target.getRuleName())) {
-        if (target.getAllOutputs().size() != 1) {
-          String allOutputs =
-              target.getAllOutputs().stream()
-                  .map(PathFragment::getPathString)
-                  .collect(Collectors.joining(" "));
-          throw new RuleErrorException(
-              String.format(
-                  "Ninja phony alias can only be used for single output, but found '%s'.",
-                  allOutputs));
-        }
-        phonyTargetsBuilder.put(Iterables.getOnlyElement(target.getAllOutputs()), target);
-      } else {
-        for (PathFragment output : target.getAllOutputs()) {
-          usualTargetsBuilder.put(output, target);
-        }
-      }
-    }
-  }
-
-  private static List<NinjaTarget> readNinjaGraph(
-      Artifact mainArtifact,
-      Collection<Path> childNinjaFiles,
-      PathFragment workingDirectory,
-      Path workspace,
-      String ownerTargetName)
-      throws InterruptedException, RuleErrorException {
-    ListeningExecutorService service = MoreExecutors.listeningDecorator(NINJA_POOL);
-    try {
-      NinjaPipeline pipeline =
-          new NinjaPipeline(
-              workspace.getRelative(workingDirectory), service, childNinjaFiles, ownerTargetName);
-      return pipeline.pipeline(mainArtifact.getPath());
-    } catch (GenericParsingException | IOException e) {
-      throw new RuleErrorException(e.getMessage());
-    }
-  }
 }
diff --git a/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraphArtifactsHelper.java b/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraphArtifactsHelper.java
index 72d5801..7d2455c 100644
--- a/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraphArtifactsHelper.java
+++ b/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraphArtifactsHelper.java
@@ -26,6 +26,7 @@
 import com.google.devtools.build.lib.vfs.Path;
 import com.google.devtools.build.lib.vfs.PathFragment;
 import com.google.devtools.build.lib.vfs.Root;
+import java.util.List;
 import java.util.SortedMap;
 
 /**
@@ -33,12 +34,13 @@
  * All created output artifacts are accumulated in the NestedSetBuilder.
  *
  * <p>Input and putput paths are interpreted relative to the working directory, see
- * working_directory property in {@link NinjaGraphRule}. All input artifacts are searched by the
- * index, created in {@link #prepare} method. All output artifact are created under the derived
- * artifacts root <execroot>/<outputRoot>, see output_root property in {@link NinjaGraphRule}.
+ * working_directory property in {@link NinjaGraphRule}. All output artifact are created under the
+ * derived artifacts root <execroot>/<outputRoot>, see output_root property in {@link
+ * NinjaGraphRule}.
  */
 class NinjaGraphArtifactsHelper {
   private final RuleContext ruleContext;
+  private final List<PathFragment> pathsToBuild;
   private final Path outputRootInSources;
   private final PathFragment outputRootPath;
   private final PathFragment workingDirectory;
@@ -59,7 +61,7 @@
    * @param srcsMap mapping between the path fragment and artifact for the files passed in 'srcs'
    *     attribute
    * @param depsNameToArtifact mapping between the path fragment in the Ninja file and prebuilt
-   *     artifact
+   * @param pathsToBuild list of paths to files required in output groups
    */
   NinjaGraphArtifactsHelper(
       RuleContext ruleContext,
@@ -67,8 +69,10 @@
       PathFragment outputRootPath,
       PathFragment workingDirectory,
       ImmutableSortedMap<PathFragment, Artifact> srcsMap,
-      ImmutableSortedMap<PathFragment, Artifact> depsNameToArtifact) {
+      ImmutableSortedMap<PathFragment, Artifact> depsNameToArtifact,
+      List<PathFragment> pathsToBuild) {
     this.ruleContext = ruleContext;
+    this.pathsToBuild = pathsToBuild;
     this.outputRootInSources =
         Preconditions.checkNotNull(sourceRoot.asPath()).getRelative(outputRootPath);
     this.outputRootPath = outputRootPath;
@@ -101,7 +105,9 @@
     DerivedArtifact derivedArtifact =
         ruleContext.getDerivedArtifact(
             pathRelativeToWorkspaceRoot.relativeTo(outputRootPath), derivedOutputRoot);
-    outputsMap.put(pathRelativeToWorkingDirectory, derivedArtifact);
+    if (pathsToBuild.contains(pathRelativeToWorkingDirectory)) {
+      outputsMap.put(pathRelativeToWorkingDirectory, derivedArtifact);
+    }
     return derivedArtifact;
   }
 
diff --git a/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraphProvider.java b/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraphProvider.java
deleted file mode 100644
index 402147e..0000000
--- a/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraphProvider.java
+++ /dev/null
@@ -1,71 +0,0 @@
-// Copyright 2019 The Bazel Authors. All rights reserved.
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-//    http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-package com.google.devtools.build.lib.bazel.rules.ninja.actions;
-
-import com.google.common.collect.ImmutableSortedMap;
-import com.google.devtools.build.lib.actions.Artifact;
-import com.google.devtools.build.lib.analysis.TransitiveInfoProvider;
-import com.google.devtools.build.lib.collect.nestedset.NestedSet;
-import com.google.devtools.build.lib.concurrent.ThreadSafety.Immutable;
-import com.google.devtools.build.lib.vfs.PathFragment;
-import javax.annotation.Nullable;
-
-/**
- * Provider to pass the information between ninja_graph ({@link NinjaGraphRule}) and ninja_build
- * targets.
- */
-@Immutable
-public final class NinjaGraphProvider implements TransitiveInfoProvider {
-  private final PathFragment outputRoot;
-  private final PathFragment workingDirectory;
-  private final ImmutableSortedMap<PathFragment, NestedSet<Artifact>> phonyTargets;
-  private final ImmutableSortedMap<PathFragment, Artifact> outputsMap;
-
-  /**
-   * Constructor
-   *
-   * @param outputRoot name of output directory for Ninja actions under execroot
-   * @param workingDirectory relative path under execroot, the root for interpreting all paths
-   * @param phonyTargets mapping between phony target names and their input files
-   * @param outputsMap mapping between output paths in Ninja file and derived artifacts
-   */
-  NinjaGraphProvider(
-      PathFragment outputRoot,
-      PathFragment workingDirectory,
-      ImmutableSortedMap<PathFragment, NestedSet<Artifact>> phonyTargets,
-      ImmutableSortedMap<PathFragment, Artifact> outputsMap) {
-    this.outputRoot = outputRoot;
-    this.workingDirectory = workingDirectory;
-    this.phonyTargets = phonyTargets;
-    this.outputsMap = outputsMap;
-  }
-
-  public PathFragment getOutputRoot() {
-    return outputRoot;
-  }
-
-  public PathFragment getWorkingDirectory() {
-    return workingDirectory;
-  }
-
-  @Nullable
-  public NestedSet<Artifact> getPhonyArtifacts(PathFragment fragment) {
-    return phonyTargets.get(fragment);
-  }
-
-  public Artifact getUsualArtifact(PathFragment fragment) {
-    return outputsMap.get(fragment);
-  }
-}
diff --git a/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraphRule.java b/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraphRule.java
index 9634415..fa25d43 100644
--- a/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraphRule.java
+++ b/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaGraphRule.java
@@ -28,13 +28,13 @@
 import com.google.devtools.build.lib.packages.BuildType;
 import com.google.devtools.build.lib.packages.RuleClass;
 import com.google.devtools.build.lib.packages.RuleClass.Builder.RuleClassType;
+import com.google.devtools.build.lib.packages.Type;
 import com.google.devtools.build.lib.syntax.Sequence;
 import com.google.devtools.build.lib.syntax.StarlarkThread;
 import com.google.devtools.build.lib.util.FileTypeSet;
 
 /**
- * The rule that parses the Ninja graph and creates the provider {@link NinjaGraphProvider}
- * describing it.
+ * The rule that parses the Ninja graph and creates {@link NinjaAction} actions.
  *
  * <p>Important aspect is relation to non-symlinked-under-execroot-directories: {@link
  * com.google.devtools.build.lib.skylarkbuildapi.WorkspaceGlobalsApi#dontSymlinkDirectoriesInExecroot(Sequence,
@@ -55,9 +55,14 @@
             attr("srcs", LABEL_LIST)
                 .allowedFileTypes(FileTypeSet.ANY_FILE)
                 .setDoc("Source files requested by Ninja graph actions."))
-        .add(attr("main", LABEL).allowedFileTypes(FileTypeSet.ANY_FILE).setDoc("Main Ninja file."))
+        .add(
+            attr("main", LABEL)
+                .allowedFileTypes(FileTypeSet.ANY_FILE)
+                .mandatory()
+                .setDoc("Main Ninja file."))
         .add(
             attr("output_root", STRING)
+                .mandatory()
                 .setDoc(
                     "<p>Directory under workspace, where all the intermediate and output artifacts"
                         + " will be created.</p><p>Must not be symlinked to the execroot. For"
@@ -87,6 +92,12 @@
                         + " output of each dependency will be used as an input to the Ninja"
                         + " action,which refers to the corresponding path.")
                 .value(ImmutableMap.of()))
+        .add(
+            attr("output_groups", Type.STRING_LIST_DICT)
+                .setDoc(
+                    "Mapping of output groups to the list of output paths in the Ninja file. "
+                        + "Only the output paths mentioned in this attribute will be built."
+                        + " Phony target names may be specified as the output paths."))
         .build();
   }
 
diff --git a/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaRulesModule.java b/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaRulesModule.java
new file mode 100644
index 0000000..e688452
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/lib/bazel/rules/ninja/actions/NinjaRulesModule.java
@@ -0,0 +1,26 @@
+// Copyright 2020 The Bazel Authors. All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package com.google.devtools.build.lib.bazel.rules.ninja.actions;
+
+import com.google.devtools.build.lib.analysis.ConfiguredRuleClassProvider;
+import com.google.devtools.build.lib.runtime.BlazeModule;
+
+/** Module for Ninja execution. */
+public class NinjaRulesModule extends BlazeModule {
+  @Override
+  public void initializeRuleClasses(ConfiguredRuleClassProvider.Builder builder) {
+    builder.addRuleDefinition(new NinjaGraphRule());
+  }
+}
diff --git a/src/test/java/com/google/devtools/build/lib/bazel/rules/ninja/NinjaGraphTest.java b/src/test/java/com/google/devtools/build/lib/bazel/rules/ninja/NinjaGraphTest.java
index f86547d..79460f6 100644
--- a/src/test/java/com/google/devtools/build/lib/bazel/rules/ninja/NinjaGraphTest.java
+++ b/src/test/java/com/google/devtools/build/lib/bazel/rules/ninja/NinjaGraphTest.java
@@ -28,7 +28,6 @@
 import com.google.devtools.build.lib.analysis.configuredtargets.RuleConfiguredTarget;
 import com.google.devtools.build.lib.analysis.util.BuildViewTestCase;
 import com.google.devtools.build.lib.bazel.rules.ninja.actions.NinjaAction;
-import com.google.devtools.build.lib.bazel.rules.ninja.actions.NinjaGraphProvider;
 import com.google.devtools.build.lib.bazel.rules.ninja.actions.NinjaGraphRule;
 import com.google.devtools.build.lib.testutil.TestRuleClassProvider;
 import com.google.devtools.build.lib.vfs.PathFragment;
@@ -77,11 +76,6 @@
             " working_directory = 'build_config',",
             " main = 'build_config/build.ninja',",
             " output_root_inputs = ['input.txt'])");
-    NinjaGraphProvider provider = configuredTarget.getProvider(NinjaGraphProvider.class);
-    assertThat(provider).isNotNull();
-    assertThat(provider.getOutputRoot()).isEqualTo(PathFragment.create("build_config"));
-    assertThat(provider.getWorkingDirectory()).isEqualTo(PathFragment.create("build_config"));
-
     assertThat(configuredTarget).isInstanceOf(RuleConfiguredTarget.class);
     RuleConfiguredTarget ninjaConfiguredTarget = (RuleConfiguredTarget) configuredTarget;
     ImmutableList<ActionAnalysisMetadata> actions = ninjaConfiguredTarget.getActions();
@@ -135,11 +129,6 @@
             " working_directory = 'build_config',",
             " main = 'build_config/build.ninja',",
             " output_root_inputs = ['input.txt'])");
-    NinjaGraphProvider provider = configuredTarget.getProvider(NinjaGraphProvider.class);
-    assertThat(provider).isNotNull();
-    assertThat(provider.getOutputRoot()).isEqualTo(PathFragment.create("build_config"));
-    assertThat(provider.getWorkingDirectory()).isEqualTo(PathFragment.create("build_config"));
-
     assertThat(configuredTarget).isInstanceOf(RuleConfiguredTarget.class);
     RuleConfiguredTarget ninjaConfiguredTarget = (RuleConfiguredTarget) configuredTarget;
     ImmutableList<ActionAnalysisMetadata> actions = ninjaConfiguredTarget.getActions();
@@ -208,11 +197,6 @@
             " working_directory = 'build_config',",
             " main = 'build_config/build.ninja',",
             " output_root_inputs = ['a.txt', 'b.txt', 'c.txt', 'd.txt', 'e.txt'])");
-    NinjaGraphProvider provider = configuredTarget.getProvider(NinjaGraphProvider.class);
-    assertThat(provider).isNotNull();
-    assertThat(provider.getOutputRoot()).isEqualTo(PathFragment.create("build_config"));
-    assertThat(provider.getWorkingDirectory()).isEqualTo(PathFragment.create("build_config"));
-
     assertThat(configuredTarget).isInstanceOf(RuleConfiguredTarget.class);
     RuleConfiguredTarget ninjaConfiguredTarget = (RuleConfiguredTarget) configuredTarget;
     ImmutableList<ActionAnalysisMetadata> actions = ninjaConfiguredTarget.getActions();
@@ -296,11 +280,6 @@
             " working_directory = 'build_config',",
             " main = 'build_config/build.ninja',",
             " deps_mapping = {'placeholder': ':input.txt'})");
-    NinjaGraphProvider provider = configuredTarget.getProvider(NinjaGraphProvider.class);
-    assertThat(provider).isNotNull();
-    assertThat(provider.getOutputRoot()).isEqualTo(PathFragment.create("build_config"));
-    assertThat(provider.getWorkingDirectory()).isEqualTo(PathFragment.create("build_config"));
-
     assertThat(configuredTarget).isInstanceOf(RuleConfiguredTarget.class);
     RuleConfiguredTarget ninjaConfiguredTarget = (RuleConfiguredTarget) configuredTarget;
     ImmutableList<ActionAnalysisMetadata> actions = ninjaConfiguredTarget.getActions();
diff --git a/src/test/java/com/google/devtools/build/lib/blackbox/tests/BUILD b/src/test/java/com/google/devtools/build/lib/blackbox/tests/BUILD
index 52b504d..40d58a5 100644
--- a/src/test/java/com/google/devtools/build/lib/blackbox/tests/BUILD
+++ b/src/test/java/com/google/devtools/build/lib/blackbox/tests/BUILD
@@ -51,6 +51,17 @@
     ],
 )
 
+java_test(
+    name = "NinjaBlackBoxTest",
+    size = "small",
+    srcs = ["NinjaBlackBoxTest.java"],
+    tags = ["black_box_test"],
+    deps = [
+        ":common_deps",
+        "//src/main/java/com/google/devtools/build/lib:bazel-ninja",
+    ],
+)
+
 test_suite(
     name = "black_box_tests",
     tags = ["black_box_test"],
diff --git a/src/test/java/com/google/devtools/build/lib/blackbox/tests/NinjaBlackBoxTest.java b/src/test/java/com/google/devtools/build/lib/blackbox/tests/NinjaBlackBoxTest.java
new file mode 100644
index 0000000..2ff1af6
--- /dev/null
+++ b/src/test/java/com/google/devtools/build/lib/blackbox/tests/NinjaBlackBoxTest.java
@@ -0,0 +1,204 @@
+// Copyright 2020 The Bazel Authors. All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package com.google.devtools.build.lib.blackbox.tests;
+
+import static com.google.common.truth.Truth.assertThat;
+
+import com.google.devtools.build.lib.blackbox.framework.BuilderRunner;
+import com.google.devtools.build.lib.blackbox.framework.ProcessResult;
+import com.google.devtools.build.lib.blackbox.junit.AbstractBlackBoxTest;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import org.junit.Test;
+
+/** Integration test for Ninja execution functionality. */
+public class NinjaBlackBoxTest extends AbstractBlackBoxTest {
+  @Test
+  public void testOneTarget() throws Exception {
+    context()
+        .write(
+            WORKSPACE,
+            "workspace(name = 'test')",
+            "dont_symlink_directories_in_execroot(paths = ['build_dir'])");
+    context().write("build_dir/input.txt", "World");
+    context()
+        .write(
+            "build_dir/build.ninja",
+            "rule echo",
+            "  command = echo \"Hello $$(cat ${in})!\" > ${out}",
+            "build hello.txt: echo input.txt");
+    context()
+        .write(
+            "BUILD",
+            "ninja_graph(name = 'graph', output_root = 'build_dir',",
+            " working_directory = 'build_dir',",
+            " main = 'build_dir/build.ninja',",
+            " output_root_inputs = ['input.txt'],",
+            " output_groups = {'group': ['hello.txt']})");
+
+    BuilderRunner bazel = context().bazel().withFlags("--experimental_ninja_actions");
+    assertConfigured(bazel.build("//:graph"));
+    Path path = context().resolveExecRootPath(bazel, "build_dir/hello.txt");
+    assertThat(path.toFile().exists()).isTrue();
+    assertThat(Files.readAllLines(path)).containsExactly("Hello World!");
+
+    // React to input change.
+    context().write("build_dir/input.txt", "Sun");
+    assertNothingConfigured(bazel.build("//:graph"));
+    assertThat(Files.readAllLines(path)).containsExactly("Hello Sun!");
+
+    // React to Ninja file change.
+    context()
+        .write(
+            "build_dir/build.ninja",
+            "rule echo",
+            "  command = echo \"Hello $$(cat ${in}):)\" > ${out}",
+            "build hello.txt: echo input.txt");
+    assertConfigured(bazel.build("//:graph"));
+    assertThat(Files.readAllLines(path)).containsExactly("Hello Sun:)");
+  }
+
+  @Test
+  public void testWithoutExperimentalFlag() throws Exception {
+    context()
+        .write(
+            WORKSPACE,
+            "workspace(name = 'test')",
+            "dont_symlink_directories_in_execroot(paths = ['build_dir'])");
+    context().write("build_dir/input.txt", "World");
+    context()
+        .write(
+            "build_dir/build.ninja",
+            "rule echo",
+            "  command = echo \"Hello $$(cat ${in})!\" > ${out}",
+            "build hello.txt: echo input.txt");
+    context()
+        .write(
+            "BUILD",
+            "ninja_graph(name = 'graph', output_root = 'build_dir',",
+            " working_directory = 'build_dir',",
+            " main = 'build_dir/build.ninja',",
+            " output_root_inputs = ['input.txt'],",
+            " output_groups = {'group': ['hello.txt']})");
+
+    BuilderRunner bazel = context().bazel();
+    ProcessResult result = bazel.shouldFail().build("//:graph");
+    assertThat(result.errString())
+        .contains("name 'dont_symlink_directories_in_execroot' is not defined");
+    assertThat(result.errString()).contains("FAILED: Build did NOT complete successfully");
+  }
+
+  @Test
+  public void testWithoutMainNinja() throws Exception {
+    context()
+        .write(
+            WORKSPACE,
+            "workspace(name = 'test')",
+            "dont_symlink_directories_in_execroot(paths = ['build_dir'])");
+    context().write("build_dir/input.txt", "World");
+    context()
+        .write(
+            "build_dir/build.ninja",
+            "rule echo",
+            "  command = echo \"Hello $$(cat ${in})!\" > ${out}",
+            "build hello.txt: echo input.txt");
+    context()
+        .write(
+            "BUILD",
+            "ninja_graph(name = 'graph', output_root = 'build_dir',",
+            " working_directory = 'build_dir',",
+            " output_root_inputs = ['input.txt'],",
+            " output_groups = {'group': ['hello.txt']})");
+
+    BuilderRunner bazel = context().bazel().withFlags("--experimental_ninja_actions");
+    ProcessResult result = bazel.shouldFail().build("//:graph");
+    assertThat(result.errString())
+        .contains("//:graph: missing value for mandatory attribute 'main' in 'ninja_graph' rule");
+    assertThat(result.errString()).contains("FAILED: Build did NOT complete successfully");
+  }
+
+  @Test
+  public void testSourceFileIsMissing() throws Exception {
+    context()
+        .write(
+            WORKSPACE,
+            "workspace(name = 'test')",
+            "dont_symlink_directories_in_execroot(paths = ['build_dir'])");
+    context().write("input.txt", "World");
+    context()
+        .write(
+            "build_dir/build.ninja",
+            "rule echo",
+            "  command = echo \"Hello $$(cat ${in})!\" > ${out}",
+            "build hello.txt: echo ../input.txt");
+    context()
+        .write(
+            "BUILD",
+            "ninja_graph(name = 'graph', output_root = 'build_dir',",
+            " working_directory = 'build_dir',",
+            " main = 'build_dir/build.ninja',",
+            " output_groups = {'group': ['hello.txt']})");
+
+    BuilderRunner bazel = context().bazel().withFlags("--experimental_ninja_actions");
+    ProcessResult result = bazel.shouldFail().build("//:graph");
+    assertThat(result.errString())
+        .contains(
+            "in ninja_graph rule //:graph: Ninja actions are allowed to create outputs only "
+                + "under output_root, path '../input.txt' is not allowed.");
+    assertThat(result.errString()).contains("FAILED: Build did NOT complete successfully");
+  }
+
+  @Test
+  public void testSourceFileIsMissingUnderOutputRoot() throws Exception {
+    context()
+        .write(
+            WORKSPACE,
+            "workspace(name = 'test')",
+            "dont_symlink_directories_in_execroot(paths = ['build_dir'])");
+    context().write("input.txt", "World");
+    context()
+        .write(
+            "build_dir/build.ninja",
+            "rule echo",
+            "  command = echo \"Hello $$(cat ${in})!\" > ${out}",
+            "build hello.txt: echo build_dir/input.txt");
+    context()
+        .write(
+            "BUILD",
+            "ninja_graph(name = 'graph', output_root = 'build_dir',",
+            " working_directory = 'build_dir',",
+            " main = 'build_dir/build.ninja',",
+            " output_groups = {'group': ['hello.txt']})");
+
+    BuilderRunner bazel = context().bazel().withFlags("--experimental_ninja_actions");
+    ProcessResult result = bazel.shouldFail().build("//:graph");
+    assertThat(result.errString())
+        .contains(
+            "in ninja_graph rule //:graph: The following artifacts do not have a generating "
+                + "action in Ninja file: build_dir/build_dir/input.txt");
+    assertThat(result.errString()).contains("FAILED: Build did NOT complete successfully");
+  }
+
+  private static void assertNothingConfigured(ProcessResult result) {
+    assertThat(result.errString())
+        .contains("INFO: Analyzed target //:graph (0 packages loaded, 0 targets configured).");
+  }
+
+  private static void assertConfigured(ProcessResult result) {
+    assertThat(result.errString())
+        .doesNotContain(
+            "INFO: Analyzed target //:graph (0 packages loaded, 0 targets configured).");
+  }
+}