Browse Source

HADOOP-13789. Hadoop Common includes generated test protos in both jar and test-jar. Contributed by Sean Busbey.

Andrew Wang 8 years ago
parent
commit
e1c6ef2efa
17 changed files with 454 additions and 256 deletions
  1. 1 5
      hadoop-common-project/hadoop-common/pom.xml
  2. 0 2
      hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
  3. 0 2
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  4. 0 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml
  5. 0 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/pom.xml
  6. 7 221
      hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocMojo.java
  7. 283 0
      hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocRunner.java
  8. 61 0
      hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocTestMojo.java
  9. 98 0
      hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/package-info.java
  10. 0 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
  11. 0 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
  12. 0 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
  13. 0 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
  14. 0 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
  15. 0 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
  16. 2 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
  17. 2 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml

+ 1 - 5
hadoop-common-project/hadoop-common/pom.xml

@@ -369,7 +369,6 @@
           </execution>
           <execution>
             <id>compile-protoc</id>
-            <phase>generate-sources</phase>
             <goals>
               <goal>protoc</goal>
             </goals>
@@ -397,14 +396,12 @@
                   <include>GenericRefreshProtocol.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
           <execution>
             <id>compile-test-protoc</id>
-            <phase>generate-test-sources</phase>
             <goals>
-              <goal>protoc</goal>
+              <goal>test-protoc</goal>
             </goals>
             <configuration>
               <protocVersion>${protobuf.version}</protocVersion>
@@ -419,7 +416,6 @@
                   <include>test_rpc_service.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-test-sources/java</output>
             </configuration>
           </execution>
           <execution>

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs-client/pom.xml

@@ -138,7 +138,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
         <executions>
           <execution>
             <id>compile-protoc</id>
-            <phase>generate-sources</phase>
             <goals>
               <goal>protoc</goal>
             </goals>
@@ -164,7 +163,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                   <include>ReconfigurationProtocol.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
         </executions>

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -325,7 +325,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
         <executions>
           <execution>
             <id>compile-protoc</id>
-            <phase>generate-sources</phase>
             <goals>
               <goal>protoc</goal>
             </goals>
@@ -352,7 +351,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                   <include>fsimage.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
           <execution>

+ 0 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml

@@ -61,7 +61,6 @@
         <executions>
           <execution>
             <id>compile-protoc</id>
-            <phase>generate-sources</phase>
             <goals>
               <goal>protoc</goal>
             </goals>
@@ -82,7 +81,6 @@
                   <include>MRClientProtocol.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
         </executions>

+ 0 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/pom.xml

@@ -59,7 +59,6 @@
         <executions>
           <execution>
             <id>compile-protoc</id>
-            <phase>generate-sources</phase>
             <goals>
               <goal>protoc</goal>
             </goals>
@@ -76,7 +75,6 @@
                   <include>ShuffleHandlerRecovery.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
         </executions>

+ 7 - 221
hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocMojo.java

@@ -13,8 +13,6 @@
  */
 package org.apache.hadoop.maven.plugin.protoc;
 
-import org.apache.hadoop.maven.plugin.util.Exec;
-import org.apache.hadoop.maven.plugin.util.FileSetUtils;
 import org.apache.maven.model.FileSet;
 import org.apache.maven.plugin.AbstractMojo;
 import org.apache.maven.plugin.MojoExecutionException;
@@ -22,22 +20,13 @@ import org.apache.maven.plugins.annotations.LifecyclePhase;
 import org.apache.maven.plugins.annotations.Mojo;
 import org.apache.maven.plugins.annotations.Parameter;
 import org.apache.maven.project.MavenProject;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.type.TypeReference;
 
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
 import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.zip.CRC32;
 
+/**
+ * Mojo to generate java classes from .proto files using protoc.
+ * See package info for examples of use in a maven pom.
+ */
 @Mojo(name="protoc", defaultPhase = LifecyclePhase.GENERATE_SOURCES)
 public class ProtocMojo extends AbstractMojo {
 
@@ -63,213 +52,10 @@ public class ProtocMojo extends AbstractMojo {
       "${project.build.directory}/hadoop-maven-plugins-protoc-checksums.json")
   private String checksumPath;
 
-  /**
-   * Compares include and source file checksums against previously computed
-   * checksums stored in a json file in the build directory.
-   */
-  public class ChecksumComparator {
-
-    private final Map<String, Long> storedChecksums;
-    private final Map<String, Long> computedChecksums;
-
-    private final File checksumFile;
-
-    ChecksumComparator(String checksumPath) throws IOException {
-      checksumFile = new File(checksumPath);
-      // Read in the checksums
-      if (checksumFile.exists()) {
-        ObjectMapper mapper = new ObjectMapper();
-        storedChecksums = mapper
-            .readValue(checksumFile, new TypeReference<Map<String, Long>>() {
-            });
-      } else {
-        storedChecksums = new HashMap<>(0);
-      }
-      computedChecksums = new HashMap<>();
-    }
-
-    public boolean hasChanged(File file) throws IOException {
-      if (!file.exists()) {
-        throw new FileNotFoundException(
-            "Specified protoc include or source does not exist: " + file);
-      }
-      if (file.isDirectory()) {
-        return hasDirectoryChanged(file);
-      } else if (file.isFile()) {
-        return hasFileChanged(file);
-      } else {
-        throw new IOException("Not a file or directory: " + file);
-      }
-    }
-
-    private boolean hasDirectoryChanged(File directory) throws IOException {
-      File[] listing = directory.listFiles();
-      boolean changed = false;
-      if (listing == null) {
-        // not changed.
-        return false;
-      }
-      // Do not exit early, since we need to compute and save checksums
-      // for each file within the directory.
-      for (File f : listing) {
-        if (f.isDirectory()) {
-          if (hasDirectoryChanged(f)) {
-            changed = true;
-          }
-        } else if (f.isFile()) {
-          if (hasFileChanged(f)) {
-            changed = true;
-          }
-        } else {
-          getLog().debug("Skipping entry that is not a file or directory: "
-              + f);
-        }
-      }
-      return changed;
-    }
-
-    private boolean hasFileChanged(File file) throws IOException {
-      long computedCsum = computeChecksum(file);
-
-      // Return if the generated csum matches the stored csum
-      Long storedCsum = storedChecksums.get(file.getCanonicalPath());
-      if (storedCsum == null || storedCsum.longValue() != computedCsum) {
-        // It has changed.
-        return true;
-      }
-      return false;
-    }
-
-    private long computeChecksum(File file) throws IOException {
-      // If we've already computed the csum, reuse the computed value
-      final String canonicalPath = file.getCanonicalPath();
-      if (computedChecksums.containsKey(canonicalPath)) {
-        return computedChecksums.get(canonicalPath);
-      }
-      // Compute the csum for the file
-      CRC32 crc = new CRC32();
-      byte[] buffer = new byte[1024*64];
-      try (BufferedInputStream in =
-          new BufferedInputStream(new FileInputStream(file))) {
-        while (true) {
-          int read = in.read(buffer);
-          if (read <= 0) {
-            break;
-          }
-          crc.update(buffer, 0, read);
-        }
-      }
-      // Save it in the generated map and return
-      final long computedCsum = crc.getValue();
-      computedChecksums.put(canonicalPath, computedCsum);
-      return crc.getValue();
-    }
-
-    public void writeChecksums() throws IOException {
-      ObjectMapper mapper = new ObjectMapper();
-      try (BufferedOutputStream out = new BufferedOutputStream(
-          new FileOutputStream(checksumFile))) {
-        mapper.writeValue(out, computedChecksums);
-        getLog().info("Wrote protoc checksums to file " + checksumFile);
-      }
-    }
-  }
-
   public void execute() throws MojoExecutionException {
-    try {
-      List<String> command = new ArrayList<String>();
-      command.add(protocCommand);
-      command.add("--version");
-      Exec exec = new Exec(this);
-      List<String> out = new ArrayList<String>();
-      if (exec.run(command, out) == 127) {
-        getLog().error("protoc, not found at: " + protocCommand);
-        throw new MojoExecutionException("protoc failure");
-      } else {
-        if (out.isEmpty()) {
-          getLog().error("stdout: " + out);
-          throw new MojoExecutionException(
-              "'protoc --version' did not return a version");
-        } else {
-          if (!out.get(0).endsWith(protocVersion)) {
-            throw new MojoExecutionException(
-                "protoc version is '" + out.get(0) + "', expected version is '"
-                    + protocVersion + "'");
-          }
-        }
-      }
-      if (!output.mkdirs()) {
-        if (!output.exists()) {
-          throw new MojoExecutionException(
-              "Could not create directory: " + output);
-        }
-      }
-
-      // Whether the import or source protoc files have changed.
-      ChecksumComparator comparator = new ChecksumComparator(checksumPath);
-      boolean importsChanged = false;
-
-      command = new ArrayList<String>();
-      command.add(protocCommand);
-      command.add("--java_out=" + output.getCanonicalPath());
-      if (imports != null) {
-        for (File i : imports) {
-          if (comparator.hasChanged(i)) {
-            importsChanged = true;
-          }
-          command.add("-I" + i.getCanonicalPath());
-        }
-      }
-      // Filter to generate classes for just the changed source files.
-      List<File> changedSources = new ArrayList<>();
-      boolean sourcesChanged = false;
-      for (File f : FileSetUtils.convertFileSetToFiles(source)) {
-        // Need to recompile if the source has changed, or if any import has
-        // changed.
-        if (comparator.hasChanged(f) || importsChanged) {
-          sourcesChanged = true;
-          changedSources.add(f);
-          command.add(f.getCanonicalPath());
-        }
-      }
-
-      if (!sourcesChanged && !importsChanged) {
-        getLog().info("No changes detected in protoc files, skipping "
-            + "generation.");
-      } else {
-        if (getLog().isDebugEnabled()) {
-          StringBuilder b = new StringBuilder();
-          b.append("Generating classes for the following protoc files: [");
-          String prefix = "";
-          for (File f : changedSources) {
-            b.append(prefix);
-            b.append(f.toString());
-            prefix = ", ";
-          }
-          b.append("]");
-          getLog().debug(b.toString());
-        }
-
-        exec = new Exec(this);
-        out = new ArrayList<String>();
-        List<String> err = new ArrayList<>();
-        if (exec.run(command, out, err) != 0) {
-          getLog().error("protoc compiler error");
-          for (String s : out) {
-            getLog().error(s);
-          }
-          for (String s : err) {
-            getLog().error(s);
-          }
-          throw new MojoExecutionException("protoc failure");
-        }
-        // Write the new checksum file on success.
-        comparator.writeChecksums();
-      }
-    } catch (Throwable ex) {
-      throw new MojoExecutionException(ex.toString(), ex);
-    }
-    project.addCompileSourceRoot(output.getAbsolutePath());
+    final ProtocRunner protoc = new ProtocRunner(project, imports, output,
+        source, protocCommand, protocVersion, checksumPath, this, false);
+    protoc.execute();
   }
 
 }

+ 283 - 0
hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocRunner.java

@@ -0,0 +1,283 @@
+/*
+ * 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 org.apache.hadoop.maven.plugin.protoc;
+
+import org.apache.hadoop.maven.plugin.util.Exec;
+import org.apache.hadoop.maven.plugin.util.FileSetUtils;
+import org.apache.maven.model.FileSet;
+import org.apache.maven.plugin.AbstractMojo;
+import org.apache.maven.plugin.MojoExecutionException;
+import org.apache.maven.project.MavenProject;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.type.TypeReference;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.zip.CRC32;
+
+/**
+ * Common execution for both the main and test protoc mojos.
+ */
+public class ProtocRunner {
+
+  private final MavenProject project;
+  private final File[] imports;
+  private final File output;
+  private final FileSet source;
+  private final String protocCommand;
+  private final String protocVersion;
+  private final String checksumPath;
+  private final boolean test;
+  private final AbstractMojo mojo;
+
+  @SuppressWarnings("checkstyle:parameternumber")
+  public ProtocRunner(final MavenProject project, final File[] imports,
+      final File output, final FileSet source, final String protocCommand,
+      final String protocVersion, final String checksumPath,
+      final AbstractMojo mojo, final boolean test) {
+    this.project = project;
+    this.imports = Arrays.copyOf(imports, imports.length);
+    this.output = output;
+    this.source = source;
+    this.protocCommand = protocCommand;
+    this.protocVersion = protocVersion;
+    this.checksumPath = checksumPath;
+    this.mojo = mojo;
+    this.test = test;
+  }
+
+  /**
+   * Compares include and source file checksums against previously computed
+   * checksums stored in a json file in the build directory.
+   */
+  public class ChecksumComparator {
+
+    private final Map<String, Long> storedChecksums;
+    private final Map<String, Long> computedChecksums;
+
+    private final File checksumFile;
+
+    ChecksumComparator(String checksumPath) throws IOException {
+      checksumFile = new File(checksumPath);
+      // Read in the checksums
+      if (checksumFile.exists()) {
+        ObjectMapper mapper = new ObjectMapper();
+        storedChecksums = mapper
+            .readValue(checksumFile, new TypeReference<Map<String, Long>>() {
+            });
+      } else {
+        storedChecksums = new HashMap<>(0);
+      }
+      computedChecksums = new HashMap<>();
+    }
+
+    public boolean hasChanged(File file) throws IOException {
+      if (!file.exists()) {
+        throw new FileNotFoundException(
+            "Specified protoc include or source does not exist: " + file);
+      }
+      if (file.isDirectory()) {
+        return hasDirectoryChanged(file);
+      } else if (file.isFile()) {
+        return hasFileChanged(file);
+      } else {
+        throw new IOException("Not a file or directory: " + file);
+      }
+    }
+
+    private boolean hasDirectoryChanged(File directory) throws IOException {
+      File[] listing = directory.listFiles();
+      boolean changed = false;
+      if (listing == null) {
+        // not changed.
+        return false;
+      }
+      // Do not exit early, since we need to compute and save checksums
+      // for each file within the directory.
+      for (File f : listing) {
+        if (f.isDirectory()) {
+          if (hasDirectoryChanged(f)) {
+            changed = true;
+          }
+        } else if (f.isFile()) {
+          if (hasFileChanged(f)) {
+            changed = true;
+          }
+        } else {
+          mojo.getLog().debug("Skipping entry that is not a file or directory: "
+              + f);
+        }
+      }
+      return changed;
+    }
+
+    private boolean hasFileChanged(File file) throws IOException {
+      long computedCsum = computeChecksum(file);
+
+      // Return if the generated csum matches the stored csum
+      Long storedCsum = storedChecksums.get(file.getCanonicalPath());
+      if (storedCsum == null || storedCsum.longValue() != computedCsum) {
+        // It has changed.
+        return true;
+      }
+      return false;
+    }
+
+    private long computeChecksum(File file) throws IOException {
+      // If we've already computed the csum, reuse the computed value
+      final String canonicalPath = file.getCanonicalPath();
+      if (computedChecksums.containsKey(canonicalPath)) {
+        return computedChecksums.get(canonicalPath);
+      }
+      // Compute the csum for the file
+      CRC32 crc = new CRC32();
+      byte[] buffer = new byte[1024*64];
+      try (BufferedInputStream in =
+          new BufferedInputStream(new FileInputStream(file))) {
+        while (true) {
+          int read = in.read(buffer);
+          if (read <= 0) {
+            break;
+          }
+          crc.update(buffer, 0, read);
+        }
+      }
+      // Save it in the generated map and return
+      final long computedCsum = crc.getValue();
+      computedChecksums.put(canonicalPath, computedCsum);
+      return crc.getValue();
+    }
+
+    public void writeChecksums() throws IOException {
+      ObjectMapper mapper = new ObjectMapper();
+      try (BufferedOutputStream out = new BufferedOutputStream(
+          new FileOutputStream(checksumFile))) {
+        mapper.writeValue(out, computedChecksums);
+        mojo.getLog().info("Wrote protoc checksums to file " + checksumFile);
+      }
+    }
+  }
+
+  public void execute() throws MojoExecutionException {
+    try {
+      List<String> command = new ArrayList<String>();
+      command.add(protocCommand);
+      command.add("--version");
+      Exec exec = new Exec(mojo);
+      List<String> out = new ArrayList<String>();
+      if (exec.run(command, out) == 127) {
+        mojo.getLog().error("protoc, not found at: " + protocCommand);
+        throw new MojoExecutionException("protoc failure");
+      } else {
+        if (out.isEmpty()) {
+          mojo.getLog().error("stdout: " + out);
+          throw new MojoExecutionException(
+              "'protoc --version' did not return a version");
+        } else {
+          if (!out.get(0).endsWith(protocVersion)) {
+            throw new MojoExecutionException(
+                "protoc version is '" + out.get(0) + "', expected version is '"
+                    + protocVersion + "'");
+          }
+        }
+      }
+      if (!output.mkdirs()) {
+        if (!output.exists()) {
+          throw new MojoExecutionException(
+              "Could not create directory: " + output);
+        }
+      }
+
+      // Whether the import or source protoc files have changed.
+      ChecksumComparator comparator = new ChecksumComparator(checksumPath);
+      boolean importsChanged = false;
+
+      command = new ArrayList<String>();
+      command.add(protocCommand);
+      command.add("--java_out=" + output.getCanonicalPath());
+      if (imports != null) {
+        for (File i : imports) {
+          if (comparator.hasChanged(i)) {
+            importsChanged = true;
+          }
+          command.add("-I" + i.getCanonicalPath());
+        }
+      }
+      // Filter to generate classes for just the changed source files.
+      List<File> changedSources = new ArrayList<>();
+      boolean sourcesChanged = false;
+      for (File f : FileSetUtils.convertFileSetToFiles(source)) {
+        // Need to recompile if the source has changed, or if any import has
+        // changed.
+        if (comparator.hasChanged(f) || importsChanged) {
+          sourcesChanged = true;
+          changedSources.add(f);
+          command.add(f.getCanonicalPath());
+        }
+      }
+
+      if (!sourcesChanged && !importsChanged) {
+        mojo.getLog().info("No changes detected in protoc files, skipping "
+            + "generation.");
+      } else {
+        if (mojo.getLog().isDebugEnabled()) {
+          StringBuilder b = new StringBuilder();
+          b.append("Generating classes for the following protoc files: [");
+          String prefix = "";
+          for (File f : changedSources) {
+            b.append(prefix);
+            b.append(f.toString());
+            prefix = ", ";
+          }
+          b.append("]");
+          mojo.getLog().debug(b.toString());
+        }
+
+        exec = new Exec(mojo);
+        out = new ArrayList<String>();
+        List<String> err = new ArrayList<>();
+        if (exec.run(command, out, err) != 0) {
+          mojo.getLog().error("protoc compiler error");
+          for (String s : out) {
+            mojo.getLog().error(s);
+          }
+          for (String s : err) {
+            mojo.getLog().error(s);
+          }
+          throw new MojoExecutionException("protoc failure");
+        }
+        // Write the new checksum file on success.
+        comparator.writeChecksums();
+      }
+    } catch (Throwable ex) {
+      throw new MojoExecutionException(ex.toString(), ex);
+    }
+    if(test) {
+      project.addTestCompileSourceRoot(output.getAbsolutePath());
+    } else {
+      project.addCompileSourceRoot(output.getAbsolutePath());
+    }
+  }
+
+}

+ 61 - 0
hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocTestMojo.java

@@ -0,0 +1,61 @@
+/*
+ * 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 org.apache.hadoop.maven.plugin.protoc;
+
+import org.apache.maven.model.FileSet;
+import org.apache.maven.plugin.AbstractMojo;
+import org.apache.maven.plugin.MojoExecutionException;
+import org.apache.maven.plugins.annotations.LifecyclePhase;
+import org.apache.maven.plugins.annotations.Mojo;
+import org.apache.maven.plugins.annotations.Parameter;
+import org.apache.maven.project.MavenProject;
+
+import java.io.File;
+
+/**
+ * Mojo to generate java test classes from .proto files using protoc.
+ * See package info for examples of use in a maven pom.
+ */
+@Mojo(name="test-protoc", defaultPhase = LifecyclePhase.GENERATE_TEST_SOURCES)
+public class ProtocTestMojo extends AbstractMojo {
+
+  @Parameter(defaultValue="${project}", readonly=true)
+  private MavenProject project;
+
+  @Parameter
+  private File[] imports;
+
+  @Parameter(defaultValue=
+      "${project.build.directory}/generated-test-sources/java")
+  private File output;
+
+  @Parameter(required=true)
+  private FileSet source;
+
+  @Parameter(defaultValue="protoc")
+  private String protocCommand;
+
+  @Parameter(required=true)
+  private String protocVersion;
+
+  @Parameter(defaultValue =
+      "${project.build.directory}/hadoop-maven-plugins-protoc-checksums.json")
+  private String checksumPath;
+
+  public void execute() throws MojoExecutionException {
+    final ProtocRunner protoc = new ProtocRunner(project, imports, output,
+        source, protocCommand, protocVersion, checksumPath, this, true);
+    protoc.execute();
+  }
+}

+ 98 - 0
hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/package-info.java

@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/**
+ * <p>Plugins to generate Java files based on protocol buffers with the protoc
+ * command.
+ *
+ * <p>For generated files intended for primary build artifacts use like:
+ * <pre>
+ *  &lt;plugins&gt;
+ *    ... SNIP ...
+ *    &lt;plugin&gt;
+ *      &lt;groupId&gt;org.apache.hadoop&lt;/groupId&gt;
+ *      &lt;artifactId&gt;hadoop-maven-plugins&lt;/artifactId&gt;
+ *      &lt;executions&gt;
+ *        ... SNIP ...
+ *        &lt;execution&gt;
+ *          &lt;id&gt;compile-protoc&lt;/id&gt;
+ *          &lt;goals&gt;
+ *            &lt;goal&gt;protoc&lt;/goal&gt;
+ *          &lt;/goals&gt;
+ *          &lt;configuration&gt;
+ *            &lt;protocVersion&gt;${protobuf.version}&lt;/protocVersion&gt;
+ *            &lt;protocCommand&gt;${protoc.path}&lt;/protocCommand&gt;
+ *            &lt;imports&gt;
+ *              &lt;param&gt;${basedir}/src/main/proto&lt;/param&gt;
+ *            &lt;/imports&gt;
+ *            &lt;source&gt;
+ *              &lt;directory&gt;${basedir}/src/main/proto&lt;/directory&gt;
+ *              &lt;includes&gt;
+ *                &lt;include&gt;HAServiceProtocol.proto&lt;/include&gt;
+ *                ... SNIP ...
+ *                &lt;include&gt;RefreshCallQueueProtocol.proto&lt;/include&gt;
+ *                &lt;include&gt;GenericRefreshProtocol.proto&lt;/include&gt;
+ *              &lt;/includes&gt;
+ *            &lt;/source&gt;
+ *          &lt;/configuration&gt;
+ *        &lt;/execution&gt;
+ *        ... SNIP ...
+ *      &lt;/executions&gt;
+ *      ... SNIP ...
+ *    &lt;/plugin&gt;
+ *  &lt;/plugins&gt;
+ * </pre>
+ *
+ * For generated files intended only for test, use like:
+ * <pre>
+ *  &lt;plugins&gt;
+ *    ... SNIP ...
+ *    &lt;plugin&gt;
+ *      &lt;groupId&gt;org.apache.hadoop&lt;/groupId&gt;
+ *      &lt;artifactId&gt;hadoop-maven-plugins&lt;/artifactId&gt;
+ *      &lt;executions&gt;
+ *        ... SNIP ...
+ *        &lt;execution&gt;
+ *          &lt;id&gt;compile-test-protoc&lt;/id&gt;
+ *          &lt;goals&gt;
+ *            &lt;goal&gt;test-protoc&lt;/goal&gt;
+ *          &lt;/goals&gt;
+ *          &lt;configuration&gt;
+ *            &lt;protocVersion&gt;${protobuf.version}&lt;/protocVersion&gt;
+ *            &lt;protocCommand&gt;${protoc.path}&lt;/protocCommand&gt;
+ *            &lt;imports&gt;
+ *              &lt;param&gt;${basedir}/src/test/proto&lt;/param&gt;
+ *            &lt;/imports&gt;
+ *            &lt;source&gt;
+ *              &lt;directory&gt;${basedir}/src/test/proto&lt;/directory&gt;
+ *              &lt;includes&gt;
+ *                &lt;include&gt;test.proto&lt;/include&gt;
+ *                &lt;include&gt;test_rpc_service.proto&lt;/include&gt;
+ *              &lt;/includes&gt;
+ *            &lt;/source&gt;
+ *          &lt;/configuration&gt;
+ *        &lt;/execution&gt;
+ *        ... SNIP ...
+ *      &lt;/executions&gt;
+ *      ... SNIP ...
+ *    &lt;/plugin&gt;
+ *  &lt;/plugins&gt;
+ * </pre>
+ *
+ */
+package org.apache.hadoop.maven.plugin.protoc;

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml

@@ -100,7 +100,6 @@
         <executions>
           <execution>
             <id>compile-protoc</id>
-            <phase>generate-sources</phase>
             <goals>
               <goal>protoc</goal>
             </goals>
@@ -128,7 +127,6 @@
                   <include>server/SCM_Admin_protocol.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
         </executions>

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml

@@ -154,7 +154,6 @@
         <executions>
           <execution>
             <id>compile-protoc</id>
-            <phase>generate-sources</phase>
             <goals>
               <goal>protoc</goal>
             </goals>
@@ -172,7 +171,6 @@
                   <include>test_amrm_token.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
         </executions>

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml

@@ -262,7 +262,6 @@
           </execution>
           <execution>
             <id>compile-protoc</id>
-            <phase>generate-sources</phase>
             <goals>
               <goal>protoc</goal>
             </goals>
@@ -280,7 +279,6 @@
                   <include>yarn_security_token.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
           <execution>

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml

@@ -195,7 +195,6 @@
         <executions>
           <execution>
             <id>compile-protoc</id>
-            <phase>generate-sources</phase>
             <goals>
               <goal>protoc</goal>
             </goals>
@@ -215,7 +214,6 @@
                   <include>yarn_server_timelineserver_recovery.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
         </executions>

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml

@@ -129,7 +129,6 @@
         <executions>
           <execution>
             <id>compile-protoc</id>
-            <phase>generate-sources</phase>
             <goals>
               <goal>protoc</goal>
             </goals>
@@ -153,7 +152,6 @@
                   <include>collectornodemanager_protocol.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
         </executions>

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml

@@ -291,7 +291,6 @@
         <executions>
           <execution>
             <id>compile-protoc</id>
-            <phase>generate-sources</phase>
             <goals>
               <goal>protoc</goal>
             </goals>
@@ -311,7 +310,6 @@
                   <include>LocalizationProtocol.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
         </executions>

+ 2 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml

@@ -287,7 +287,6 @@
         <executions>
           <execution>
             <id>compile-protoc</id>
-            <phase>generate-sources</phase>
             <goals>
               <goal>protoc</goal>
             </goals>
@@ -307,14 +306,12 @@
                   <include>yarn_server_resourcemanager_recovery.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
           <execution>
             <id>compile-test-protoc</id>
-            <phase>generate-sources</phase>
             <goals>
-              <goal>protoc</goal>
+              <goal>test-protoc</goal>
             </goals>
             <configuration>
               <protocVersion>${protobuf.version}</protocVersion>
@@ -327,10 +324,9 @@
               <source>
                 <directory>${basedir}/src/test/proto</directory>
                 <includes>
-                          <include>test_client_tokens.proto</include>
+                  <include>test_client_tokens.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
         </executions>

+ 2 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml

@@ -135,10 +135,9 @@
         <artifactId>hadoop-maven-plugins</artifactId>
         <executions>
           <execution>
-            <id>compile-protoc</id>
-            <phase>generate-sources</phase>
+            <id>compile-test-protoc</id>
             <goals>
-              <goal>protoc</goal>
+              <goal>test-protoc</goal>
             </goals>
             <configuration>
               <protocVersion>${protobuf.version}</protocVersion>
@@ -154,7 +153,6 @@
                   <include>test_token.proto</include>
                 </includes>
               </source>
-              <output>${project.build.directory}/generated-sources/java</output>
             </configuration>
           </execution>
         </executions>